浏览代码

Merge trunk into HDFS-1073. Some tests failing due to merge -- will be addressed in followup commit.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1073@1143523 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 14 年之前
父节点
当前提交
45da9c0dea
共有 95 个文件被更改,包括 2075 次插入1042 次删除
  1. 48 0
      hdfs/CHANGES.txt
  2. 3 4
      hdfs/src/java/org/apache/hadoop/hdfs/BlockReader.java
  3. 26 30
      hdfs/src/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java
  4. 102 53
      hdfs/src/java/org/apache/hadoop/hdfs/DFSClient.java
  5. 7 11
      hdfs/src/java/org/apache/hadoop/hdfs/DFSInputStream.java
  6. 13 19
      hdfs/src/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  7. 21 13
      hdfs/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java
  8. 2 1
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/DSQuotaExceededException.java
  9. 102 3
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
  10. 32 91
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
  11. 50 38
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
  12. 4 4
      hdfs/src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  13. 14 13
      hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  14. 17 10
      hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
  15. 103 73
      hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  16. 6 4
      hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  17. 5 2
      hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  18. 6 5
      hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
  19. 2 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
  20. 38 34
      hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  21. 57 0
      hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java
  22. 3 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
  23. 6 5
      hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
  24. 2 3
      hdfs/src/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
  25. 6 0
      hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  26. 11 10
      hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  27. 46 56
      hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  28. 15 9
      hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
  29. 1 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
  30. 106 102
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
  31. 1 0
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/DecommissionManager.java
  32. 22 10
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
  33. 18 11
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  34. 24 2
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  35. 1 0
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  36. 6 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  37. 1 0
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  38. 3 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  39. 98 82
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  40. 1 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
  41. 4 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/Host2NodesMap.java
  42. 6 4
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  43. 14 3
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  44. 9 5
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  45. 10 5
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  46. 1 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  47. 4 3
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  48. 30 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  49. 3 3
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  50. 4 4
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  51. 1 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  52. 54 76
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java
  53. 8 3
      hdfs/src/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java
  54. 1 1
      hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj
  55. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/cli/testHDFSConf.xml
  56. 3 3
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java
  57. 27 12
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java
  58. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java
  59. 9 15
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestByteRangeInputStream.java
  60. 36 17
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSPermission.java
  61. 6 3
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java
  62. 71 11
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgrade.java
  63. 30 28
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
  64. 68 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestQuota.java
  65. 9 5
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/UpgradeUtilities.java
  66. 8 6
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java
  67. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java
  68. 2 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
  69. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
  70. 9 9
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
  71. 4 3
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlocks.java
  72. 2 2
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
  73. 12 11
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
  74. 11 18
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
  75. 4 4
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
  76. 1 0
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
  77. 2 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
  78. 2 0
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
  79. 66 0
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  80. 3 2
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestComputeInvalidateWork.java
  81. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
  82. 7 9
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
  83. 3 2
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHost2NodesMap.java
  84. 6 6
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
  85. 4 3
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java
  86. 5 4
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java
  87. 59 0
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestSecondaryWebUi.java
  88. 91 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
  89. 86 11
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java
  90. 149 8
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStreamFile.java
  91. 74 0
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
  92. 9 8
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
  93. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/net/TestNetworkTopology.java
  94. 13 7
      hdfs/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java
  95. 0 10
      hdfs/src/webapps/hdfs/block_info_xml.jsp

+ 48 - 0
hdfs/CHANGES.txt

@@ -291,6 +291,8 @@ Trunk (unreleased changes)
     (todd)
     (todd)
 
 
     HDFS-2055. Add hflush support to libhdfs. (Travis Crawford via eli)
     HDFS-2055. Add hflush support to libhdfs. (Travis Crawford via eli)
+   
+    HDFS-2083. Query JMX statistics over http via JMXJsonServlet. (tanping)
 
 
   IMPROVEMENTS
   IMPROVEMENTS
 
 
@@ -519,6 +521,25 @@ Trunk (unreleased changes)
     HDFS-1568. Improve the log messages in DataXceiver.  (Joey Echeverria via
     HDFS-1568. Improve the log messages in DataXceiver.  (Joey Echeverria via
     szetszwo)
     szetszwo)
 
 
+    HDFS-2100. Improve TestStorageRestore. (atm)
+
+    HDFS-2092. Remove some object references to Configuration in DFSClient.
+    (Bharath Mundlapudi via szetszwo)
+
+    HDFS-2087. Declare methods in DataTransferProtocol interface, and change
+    Sender and Receiver to implement the interface.  (szetszwo)
+
+    HDFS-1723. quota errors messages should use the same scale. (Jim Plush via
+    atm)
+
+    HDFS-2110. StreamFile and ByteRangeInputStream cleanup. (eli)
+
+    HDFS-2107. Move block management code from o.a.h.h.s.namenode to a new
+    package o.a.h.h.s.blockmanagement.  (szetszwo)
+
+    HDFS-2109. Store uMask as member variable to DFSClient.Conf.  (Bharath
+    Mundlapudi via szetszwo)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-1458. Improve checkpoint performance by avoiding unnecessary image
     HDFS-1458. Improve checkpoint performance by avoiding unnecessary image
@@ -534,8 +555,16 @@ Trunk (unreleased changes)
 
 
     HDFS-2056. Update fetchdt usage. (Tanping Wang via jitendra)
     HDFS-2056. Update fetchdt usage. (Tanping Wang via jitendra)
 
 
+    HDFS-2118. Couple dfs data dir improvements. (eli)
+
   BUG FIXES
   BUG FIXES
 
 
+    HDFS-2011. Removal and restoration of storage directories on checkpointing
+    failure doesn't work properly. (Ravi Prakash via mattf)
+
+    HDFS-1955. FSImage.doUpgrade() was made too fault-tolerant by HDFS-1826.
+    (mattf)
+
     HDFS-2061. Two minor bugs in BlockManager block report processing. (mattf)
     HDFS-2061. Two minor bugs in BlockManager block report processing. (mattf)
 
 
     HDFS-1449. Fix test failures - ExtendedBlock must return 
     HDFS-1449. Fix test failures - ExtendedBlock must return 
@@ -762,6 +791,23 @@ Trunk (unreleased changes)
     HDFS-1734. 'Chunk size to view' option is not working in Name Node UI.
     HDFS-1734. 'Chunk size to view' option is not working in Name Node UI.
     (Uma Maheswara Rao G via jitendra)
     (Uma Maheswara Rao G via jitendra)
 
 
+    HDFS-2086. If the include hosts list contains host names, after restarting
+    namenode, data nodes registration is denied.  Contributed by Tanping Wang.
+
+    HDFS-2082. SecondaryNameNode web interface doesn't show the right info. (atm)
+
+    HDFS-1321. If service port and main port are the same, there is no clear
+    log message explaining the issue. (Jim Plush via atm)
+
+    HDFS-1381. HDFS javadocs hard-code references to dfs.namenode.name.dir and
+    dfs.datanode.data.dir parameters (Jim Plush via atm)
+
+    HDFS-2053. Bug in INodeDirectory#computeContentSummary warning.
+    (Michael Noll via eli)
+
+    HDFS-1990. Fix  resource leaks in BlockReceiver.close().  (Uma Maheswara
+    Rao G via szetszwo)
+
 Release 0.22.0 - Unreleased
 Release 0.22.0 - Unreleased
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -819,6 +865,8 @@ Release 0.22.0 - Unreleased
     HDFS-528. Add ability for safemode to wait for a minimum number of 
     HDFS-528. Add ability for safemode to wait for a minimum number of 
     live datanodes (Todd Lipcon via eli)
     live datanodes (Todd Lipcon via eli)
 
 
+    HDFS-1753. Resource Leak in StreamFile. (Uma Maheswara Rao G via eli)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-1304. Add a new unit test for HftpFileSystem.open(..).  (szetszwo)
     HDFS-1304. Add a new unit test for HftpFileSystem.open(..).  (szetszwo)

+ 3 - 4
hdfs/src/java/org/apache/hadoop/hdfs/BlockReader.java

@@ -404,10 +404,9 @@ public class BlockReader extends FSInputChecker {
                                      String clientName)
                                      String clientName)
                                      throws IOException {
                                      throws IOException {
     // in and out will be closed when sock is closed (by the caller)
     // in and out will be closed when sock is closed (by the caller)
-    Sender.opReadBlock(
-        new DataOutputStream(new BufferedOutputStream(
-            NetUtils.getOutputStream(sock,HdfsConstants.WRITE_TIMEOUT))),
-        block, startOffset, len, clientName, blockToken);
+    final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
+          NetUtils.getOutputStream(sock, HdfsConstants.WRITE_TIMEOUT)));
+    new Sender(out).readBlock(block, blockToken, clientName, startOffset, len);
     
     
     //
     //
     // Get bytes in block, set streams
     // Get bytes in block, set streams

+ 26 - 30
hdfs/src/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java

@@ -26,7 +26,6 @@ import java.net.URL;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.hdfs.server.namenode.StreamFile;
 import org.apache.hadoop.hdfs.server.namenode.StreamFile;
 
 
-
 /**
 /**
  * To support HTTP byte streams, a new connection to an HTTP server needs to be
  * To support HTTP byte streams, a new connection to an HTTP server needs to be
  * created each time. This class hides the complexity of those multiple 
  * created each time. This class hides the complexity of those multiple 
@@ -60,7 +59,9 @@ class ByteRangeInputStream extends FSInputStream {
     }  
     }  
   }
   }
   
   
-  
+  enum StreamStatus {
+    NORMAL, SEEK
+  }
   protected InputStream in;
   protected InputStream in;
   protected URLOpener originalURL;
   protected URLOpener originalURL;
   protected URLOpener resolvedURL;
   protected URLOpener resolvedURL;
@@ -68,9 +69,7 @@ class ByteRangeInputStream extends FSInputStream {
   protected long currentPos = 0;
   protected long currentPos = 0;
   protected long filelength;
   protected long filelength;
 
 
-  protected int status = STATUS_SEEK;
-  protected static final int STATUS_NORMAL = 0;
-  protected static final int STATUS_SEEK = 1;
+  StreamStatus status = StreamStatus.SEEK;
 
 
   ByteRangeInputStream(final URL url) {
   ByteRangeInputStream(final URL url) {
     this(new URLOpener(url), new URLOpener(null));
     this(new URLOpener(url), new URLOpener(null));
@@ -82,18 +81,19 @@ class ByteRangeInputStream extends FSInputStream {
   }
   }
   
   
   private InputStream getInputStream() throws IOException {
   private InputStream getInputStream() throws IOException {
-    if (status != STATUS_NORMAL) {
+    if (status != StreamStatus.NORMAL) {
       
       
       if (in != null) {
       if (in != null) {
         in.close();
         in.close();
         in = null;
         in = null;
       }
       }
       
       
-      // use the original url  if no resolved url exists (e.g., if it's 
-      // the first time a request is made)
-      final URLOpener o = resolvedURL.getURL() == null? originalURL: resolvedURL;
+      // Use the original url if no resolved url exists, eg. if
+      // it's the first time a request is made.
+      final URLOpener opener =
+        (resolvedURL.getURL() == null) ? originalURL : resolvedURL;
 
 
-      final HttpURLConnection connection = o.openConnection();
+      final HttpURLConnection connection = opener.openConnection();
       try {
       try {
         connection.setRequestMethod("GET");
         connection.setRequestMethod("GET");
         if (startPos != 0) {
         if (startPos != 0) {
@@ -101,36 +101,35 @@ class ByteRangeInputStream extends FSInputStream {
         }
         }
         connection.connect();
         connection.connect();
         final String cl = connection.getHeaderField(StreamFile.CONTENT_LENGTH);
         final String cl = connection.getHeaderField(StreamFile.CONTENT_LENGTH);
-        filelength = cl == null? -1: Long.parseLong(cl);
+        filelength = (cl == null) ? -1 : Long.parseLong(cl);
         if (HftpFileSystem.LOG.isDebugEnabled()) {
         if (HftpFileSystem.LOG.isDebugEnabled()) {
           HftpFileSystem.LOG.debug("filelength = " + filelength);
           HftpFileSystem.LOG.debug("filelength = " + filelength);
         }
         }
         in = connection.getInputStream();
         in = connection.getInputStream();
-      } catch(IOException ioe) {
+      } catch (IOException ioe) {
         HftpFileSystem.throwIOExceptionFromConnection(connection, ioe);
         HftpFileSystem.throwIOExceptionFromConnection(connection, ioe);
       }
       }
       
       
-      if (startPos != 0 && connection.getResponseCode() != 206) {
-        // we asked for a byte range but did not receive a partial content
+      int respCode = connection.getResponseCode();
+      if (startPos != 0 && respCode != HttpURLConnection.HTTP_PARTIAL) {
+        // We asked for a byte range but did not receive a partial content
         // response...
         // response...
-        throw new IOException("206 expected, but received "
-                              + connection.getResponseCode());
-      } else if(startPos == 0 && connection.getResponseCode() != 200) {
-        // we asked for all bytes from the beginning but didn't receive a 200
+        throw new IOException("HTTP_PARTIAL expected, received " + respCode);
+      } else if (startPos == 0 && respCode != HttpURLConnection.HTTP_OK) {
+        // We asked for all bytes from the beginning but didn't receive a 200
         // response (none of the other 2xx codes are valid here)
         // response (none of the other 2xx codes are valid here)
-        throw new IOException("200 expected, but received "
-                              + connection.getResponseCode());
+        throw new IOException("HTTP_OK expected, received " + respCode);
       }
       }
-      
+
       resolvedURL.setURL(connection.getURL());
       resolvedURL.setURL(connection.getURL());
-      status = STATUS_NORMAL;
+      status = StreamStatus.NORMAL;
     }
     }
     
     
     return in;
     return in;
   }
   }
   
   
-  private void update(final boolean isEOF, final int n
-      ) throws IOException {
+  private void update(final boolean isEOF, final int n)
+      throws IOException {
     if (!isEOF) {
     if (!isEOF) {
       currentPos += n;
       currentPos += n;
     } else if (currentPos < filelength) {
     } else if (currentPos < filelength) {
@@ -154,7 +153,7 @@ class ByteRangeInputStream extends FSInputStream {
     if (pos != currentPos) {
     if (pos != currentPos) {
       startPos = pos;
       startPos = pos;
       currentPos = pos;
       currentPos = pos;
-      status = STATUS_SEEK;
+      status = StreamStatus.SEEK;
     }
     }
   }
   }
 
 
@@ -162,7 +161,7 @@ class ByteRangeInputStream extends FSInputStream {
    * Return the current offset from the start of the file
    * Return the current offset from the start of the file
    */
    */
   public long getPos() throws IOException {
   public long getPos() throws IOException {
-    return currentPos; // keep total count?
+    return currentPos;
   }
   }
 
 
   /**
   /**
@@ -172,7 +171,4 @@ class ByteRangeInputStream extends FSInputStream {
   public boolean seekToNewSource(long targetPos) throws IOException {
   public boolean seekToNewSource(long targetPos) throws IOException {
     return false;
     return false;
   }
   }
-
-}
-
-
+}

+ 102 - 53
hdfs/src/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -128,17 +128,87 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   static Random r = new Random();
   static Random r = new Random();
   final String clientName;
   final String clientName;
   Configuration conf;
   Configuration conf;
-  long defaultBlockSize;
-  private short defaultReplication;
   SocketFactory socketFactory;
   SocketFactory socketFactory;
-  int socketTimeout;
-  final int writePacketSize;
   final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
   final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
   final FileSystem.Statistics stats;
   final FileSystem.Statistics stats;
   final int hdfsTimeout;    // timeout value for a DFS operation.
   final int hdfsTimeout;    // timeout value for a DFS operation.
   final LeaseRenewer leaserenewer;
   final LeaseRenewer leaserenewer;
-
   final SocketCache socketCache;
   final SocketCache socketCache;
+  final Conf dfsClientConf;
+
+  /**
+   * DFSClient configuration 
+   */
+  static class Conf {
+    final int maxBlockAcquireFailures;
+    final int confTime;
+    final int ioBufferSize;
+    final int bytesPerChecksum;
+    final int writePacketSize;
+    final int socketTimeout;
+    final int socketCacheCapacity;
+    /** Wait time window (in msec) if BlockMissingException is caught */
+    final int timeWindow;
+    final int nCachedConnRetry;
+    final int nBlockWriteRetry;
+    final int nBlockWriteLocateFollowingRetry;
+    final long defaultBlockSize;
+    final long prefetchSize;
+    final short defaultReplication;
+    final String taskId;
+    final FsPermission uMask;
+
+    Conf(Configuration conf) {
+      maxBlockAcquireFailures = conf.getInt(
+          DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
+          DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
+      confTime = conf.getInt(
+          DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
+          HdfsConstants.WRITE_TIMEOUT);
+      ioBufferSize = conf.getInt(
+          CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
+          CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
+      bytesPerChecksum = conf.getInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY,
+          DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT);
+      socketTimeout = conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY,
+          HdfsConstants.READ_TIMEOUT);
+      /** dfs.write.packet.size is an internal config variable */
+      writePacketSize = conf.getInt(
+          DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
+          DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
+      defaultBlockSize = conf.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
+          DEFAULT_BLOCK_SIZE);
+      defaultReplication = (short) conf.getInt(
+          DFSConfigKeys.DFS_REPLICATION_KEY,
+          DFSConfigKeys.DFS_REPLICATION_DEFAULT);
+      taskId = conf.get("mapreduce.task.attempt.id", "NONMAPREDUCE");
+      socketCacheCapacity = conf.getInt(
+          DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY,
+          DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT);
+      prefetchSize = conf.getLong(
+          DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY,
+          10 * defaultBlockSize);
+      timeWindow = conf
+          .getInt(DFSConfigKeys.DFS_CLIENT_RETRY_WINDOW_BASE, 3000);
+      nCachedConnRetry = conf.getInt(
+          DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_KEY,
+          DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT);
+      nBlockWriteRetry = conf.getInt(
+          DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY,
+          DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT);
+      nBlockWriteLocateFollowingRetry = conf
+          .getInt(
+              DFSConfigKeys
+              .DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY,
+              DFSConfigKeys
+              .DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT);
+      uMask = FsPermission.getUMask(conf);
+    }
+  }
+ 
+  Conf getConf() {
+    return dfsClientConf;
+  }
   
   
   /**
   /**
    * A map from file names to {@link DFSOutputStream} objects
    * A map from file names to {@link DFSOutputStream} objects
@@ -257,16 +327,11 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   DFSClient(InetSocketAddress nameNodeAddr, ClientProtocol rpcNamenode,
   DFSClient(InetSocketAddress nameNodeAddr, ClientProtocol rpcNamenode,
       Configuration conf, FileSystem.Statistics stats)
       Configuration conf, FileSystem.Statistics stats)
     throws IOException {
     throws IOException {
+    // Copy only the required DFSClient configuration
+    this.dfsClientConf = new Conf(conf);
     this.conf = conf;
     this.conf = conf;
     this.stats = stats;
     this.stats = stats;
-    this.socketTimeout = 
-      conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 
-                  HdfsConstants.READ_TIMEOUT);
     this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
     this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
-    // dfs.write.packet.size is an internal config variable
-    this.writePacketSize = 
-      conf.getInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 
-                  DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
     this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
     this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
 
 
     // The hdfsTimeout is currently the same as the ipc timeout 
     // The hdfsTimeout is currently the same as the ipc timeout 
@@ -275,19 +340,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     final String authority = nameNodeAddr == null? "null":
     final String authority = nameNodeAddr == null? "null":
         nameNodeAddr.getHostName() + ":" + nameNodeAddr.getPort();
         nameNodeAddr.getHostName() + ":" + nameNodeAddr.getPort();
     this.leaserenewer = LeaseRenewer.getInstance(authority, ugi, this);
     this.leaserenewer = LeaseRenewer.getInstance(authority, ugi, this);
-    
-    String taskId = conf.get("mapreduce.task.attempt.id", "NONMAPREDUCE");
-    this.clientName = leaserenewer.getClientName(taskId);
-
-    defaultBlockSize = conf.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
-    defaultReplication = (short) 
-      conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 
-                  DFSConfigKeys.DFS_REPLICATION_DEFAULT);
-    
-    this.socketCache = new SocketCache(
-        conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY,
-            DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT));
-
+    this.clientName = leaserenewer.getClientName(dfsClientConf.taskId);
+    this.socketCache = new SocketCache(dfsClientConf.socketCacheCapacity);
     if (nameNodeAddr != null && rpcNamenode == null) {
     if (nameNodeAddr != null && rpcNamenode == null) {
       this.rpcNamenode = createRPCNamenode(nameNodeAddr, conf, ugi);
       this.rpcNamenode = createRPCNamenode(nameNodeAddr, conf, ugi);
       this.namenode = createNamenode(this.rpcNamenode);
       this.namenode = createNamenode(this.rpcNamenode);
@@ -306,8 +360,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * to retrieve block locations when reading.
    * to retrieve block locations when reading.
    */
    */
   int getMaxBlockAcquireFailures() {
   int getMaxBlockAcquireFailures() {
-    return conf.getInt(DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
-                       DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
+    return dfsClientConf.maxBlockAcquireFailures;
   }
   }
 
 
   /**
   /**
@@ -315,18 +368,14 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @param numNodes the number of nodes in the pipeline.
    * @param numNodes the number of nodes in the pipeline.
    */
    */
   int getDatanodeWriteTimeout(int numNodes) {
   int getDatanodeWriteTimeout(int numNodes) {
-    int confTime =
-        conf.getInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
-                    HdfsConstants.WRITE_TIMEOUT);
-
-    return (confTime > 0) ?
-      (confTime + HdfsConstants.WRITE_TIMEOUT_EXTENSION * numNodes) : 0;
+    return (dfsClientConf.confTime > 0) ?
+      (dfsClientConf.confTime + HdfsConstants.WRITE_TIMEOUT_EXTENSION * numNodes) : 0;
   }
   }
 
 
   int getDatanodeReadTimeout(int numNodes) {
   int getDatanodeReadTimeout(int numNodes) {
-    return socketTimeout > 0 ?
+    return dfsClientConf.socketTimeout > 0 ?
         (HdfsConstants.READ_TIMEOUT_EXTENSION * numNodes +
         (HdfsConstants.READ_TIMEOUT_EXTENSION * numNodes +
-        socketTimeout) : 0;
+            dfsClientConf.socketTimeout) : 0;
   }
   }
   
   
   int getHdfsTimeout() {
   int getHdfsTimeout() {
@@ -430,7 +479,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @return the default block size in bytes
    * @return the default block size in bytes
    */
    */
   public long getDefaultBlockSize() {
   public long getDefaultBlockSize() {
-    return defaultBlockSize;
+    return dfsClientConf.defaultBlockSize;
   }
   }
     
     
   /**
   /**
@@ -528,7 +577,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   }
   }
   
   
   public short getDefaultReplication() {
   public short getDefaultReplication() {
-    return defaultReplication;
+    return dfsClientConf.defaultReplication;
   }
   }
 
 
   /**
   /**
@@ -583,7 +632,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   
   
   public DFSInputStream open(String src) 
   public DFSInputStream open(String src) 
       throws IOException, UnresolvedLinkException {
       throws IOException, UnresolvedLinkException {
-    return open(src, conf.getInt("io.file.buffer.size", 4096), true, null);
+    return open(src, dfsClientConf.ioBufferSize, true, null);
   }
   }
 
 
   /**
   /**
@@ -629,7 +678,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    */
    */
   public OutputStream create(String src, boolean overwrite) 
   public OutputStream create(String src, boolean overwrite) 
       throws IOException {
       throws IOException {
-    return create(src, overwrite, defaultReplication, defaultBlockSize, null);
+    return create(src, overwrite, dfsClientConf.defaultReplication,
+        dfsClientConf.defaultBlockSize, null);
   }
   }
     
     
   /**
   /**
@@ -639,7 +689,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   public OutputStream create(String src, 
   public OutputStream create(String src, 
                              boolean overwrite,
                              boolean overwrite,
                              Progressable progress) throws IOException {
                              Progressable progress) throws IOException {
-    return create(src, overwrite, defaultReplication, defaultBlockSize, progress);
+    return create(src, overwrite, dfsClientConf.defaultReplication,
+        dfsClientConf.defaultBlockSize, progress);
   }
   }
     
     
   /**
   /**
@@ -660,7 +711,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   public OutputStream create(String src, boolean overwrite, short replication,
   public OutputStream create(String src, boolean overwrite, short replication,
       long blockSize, Progressable progress) throws IOException {
       long blockSize, Progressable progress) throws IOException {
     return create(src, overwrite, replication, blockSize, progress,
     return create(src, overwrite, replication, blockSize, progress,
-        conf.getInt("io.file.buffer.size", 4096));
+        dfsClientConf.ioBufferSize);
   }
   }
 
 
   /**
   /**
@@ -740,14 +791,13 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     if (permission == null) {
     if (permission == null) {
       permission = FsPermission.getDefault();
       permission = FsPermission.getDefault();
     }
     }
-    FsPermission masked = permission.applyUMask(FsPermission.getUMask(conf));
+    FsPermission masked = permission.applyUMask(dfsClientConf.uMask);
     if(LOG.isDebugEnabled()) {
     if(LOG.isDebugEnabled()) {
       LOG.debug(src + ": masked=" + masked);
       LOG.debug(src + ": masked=" + masked);
     }
     }
-    final DFSOutputStream result = new DFSOutputStream(this, src, masked,
-        flag, createParent, replication, blockSize, progress, buffersize,
-        conf.getInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 
-                    DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT));
+    final DFSOutputStream result = new DFSOutputStream(this, src, masked, flag,
+        createParent, replication, blockSize, progress, buffersize,
+        dfsClientConf.bytesPerChecksum);
     leaserenewer.put(src, result, this);
     leaserenewer.put(src, result, this);
     return result;
     return result;
   }
   }
@@ -808,7 +858,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       throws IOException {
       throws IOException {
     try {
     try {
       FsPermission dirPerm = 
       FsPermission dirPerm = 
-          FsPermission.getDefault().applyUMask(FsPermission.getUMask(conf)); 
+          FsPermission.getDefault().applyUMask(dfsClientConf.uMask); 
       namenode.createSymlink(target, link, dirPerm, createParent);
       namenode.createSymlink(target, link, dirPerm, createParent);
     } catch (RemoteException re) {
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
       throw re.unwrapRemoteException(AccessControlException.class,
@@ -851,8 +901,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
                                      UnresolvedPathException.class);
                                      UnresolvedPathException.class);
     }
     }
     return new DFSOutputStream(this, src, buffersize, progress,
     return new DFSOutputStream(this, src, buffersize, progress,
-        lastBlock, stat, conf.getInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 
-                                     DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT));
+        lastBlock, stat, dfsClientConf.bytesPerChecksum);
   }
   }
   
   
   /**
   /**
@@ -1061,7 +1110,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    */
    */
   public MD5MD5CRC32FileChecksum getFileChecksum(String src) throws IOException {
   public MD5MD5CRC32FileChecksum getFileChecksum(String src) throws IOException {
     checkOpen();
     checkOpen();
-    return getFileChecksum(src, namenode, socketFactory, socketTimeout);    
+    return getFileChecksum(src, namenode, socketFactory, dfsClientConf.socketTimeout);    
   }
   }
 
 
   /**
   /**
@@ -1117,7 +1166,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
                 + Op.BLOCK_CHECKSUM + ", block=" + block);
                 + Op.BLOCK_CHECKSUM + ", block=" + block);
           }
           }
           // get block MD5
           // get block MD5
-          Sender.opBlockChecksum(out, block, lb.getBlockToken());
+          new Sender(out).blockChecksum(block, lb.getBlockToken());
 
 
           final BlockOpResponseProto reply =
           final BlockOpResponseProto reply =
             BlockOpResponseProto.parseFrom(HdfsProtoUtil.vintPrefixed(in));
             BlockOpResponseProto.parseFrom(HdfsProtoUtil.vintPrefixed(in));
@@ -1377,7 +1426,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     if (permission == null) {
     if (permission == null) {
       permission = FsPermission.getDefault();
       permission = FsPermission.getDefault();
     }
     }
-    FsPermission masked = permission.applyUMask(FsPermission.getUMask(conf));
+    FsPermission masked = permission.applyUMask(dfsClientConf.uMask);
     if(LOG.isDebugEnabled()) {
     if(LOG.isDebugEnabled()) {
       LOG.debug(src + ": masked=" + masked);
       LOG.debug(src + ": masked=" + masked);
     }
     }
@@ -1404,7 +1453,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     checkOpen();
     checkOpen();
     if (absPermission == null) {
     if (absPermission == null) {
       absPermission = 
       absPermission = 
-        FsPermission.getDefault().applyUMask(FsPermission.getUMask(conf));
+        FsPermission.getDefault().applyUMask(dfsClientConf.uMask);
     } 
     } 
 
 
     if(LOG.isDebugEnabled()) {
     if(LOG.isDebugEnabled()) {

+ 7 - 11
hdfs/src/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -83,7 +83,7 @@ public class DFSInputStream extends FSInputStream {
    * capped at maxBlockAcquireFailures
    * capped at maxBlockAcquireFailures
    */
    */
   private int failures = 0;
   private int failures = 0;
-  private int timeWindow = 3000; // wait time window (in msec) if BlockMissingException is caught
+  private int timeWindow; 
 
 
   /* XXX Use of CocurrentHashMap is temp fix. Need to fix 
   /* XXX Use of CocurrentHashMap is temp fix. Need to fix 
    * parallel accesses to DFSInputStream (through ptreads) properly */
    * parallel accesses to DFSInputStream (through ptreads) properly */
@@ -106,13 +106,9 @@ public class DFSInputStream extends FSInputStream {
     this.buffersize = buffersize;
     this.buffersize = buffersize;
     this.src = src;
     this.src = src;
     this.socketCache = dfsClient.socketCache;
     this.socketCache = dfsClient.socketCache;
-    prefetchSize = this.dfsClient.conf.getLong(DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY,
-        10 * dfsClient.defaultBlockSize);
-    timeWindow = this.dfsClient.conf.getInt(
-        DFSConfigKeys.DFS_CLIENT_RETRY_WINDOW_BASE, timeWindow);
-    nCachedConnRetry = this.dfsClient.conf.getInt(
-        DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_KEY,
-        DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT);
+    prefetchSize = dfsClient.getConf().prefetchSize;
+    timeWindow = dfsClient.getConf().timeWindow;
+    nCachedConnRetry = dfsClient.getConf().nCachedConnRetry;
     openInfo();
     openInfo();
   }
   }
 
 
@@ -163,7 +159,7 @@ public class DFSInputStream extends FSInputStream {
       
       
       try {
       try {
         cdp = DFSClient.createClientDatanodeProtocolProxy(
         cdp = DFSClient.createClientDatanodeProtocolProxy(
-        datanode, dfsClient.conf, dfsClient.socketTimeout, locatedblock);
+        datanode, dfsClient.conf, dfsClient.getConf().socketTimeout, locatedblock);
         
         
         final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
         final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
         
         
@@ -771,8 +767,8 @@ public class DFSInputStream extends FSInputStream {
         // disaster.
         // disaster.
         sock.setTcpNoDelay(true);
         sock.setTcpNoDelay(true);
 
 
-        NetUtils.connect(sock, dnAddr, dfsClient.socketTimeout);
-        sock.setSoTimeout(dfsClient.socketTimeout);
+        NetUtils.connect(sock, dnAddr, dfsClient.getConf().socketTimeout);
+        sock.setSoTimeout(dfsClient.getConf().socketTimeout);
       }
       }
 
 
       try {
       try {

+ 13 - 19
hdfs/src/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -103,7 +103,6 @@ import org.apache.hadoop.util.StringUtils;
 ****************************************************************/
 ****************************************************************/
 class DFSOutputStream extends FSOutputSummer implements Syncable {
 class DFSOutputStream extends FSOutputSummer implements Syncable {
   private final DFSClient dfsClient;
   private final DFSClient dfsClient;
-  private Configuration conf;
   private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB
   private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB
   private Socket s;
   private Socket s;
   // closed is accessed by different threads under different locks.
   // closed is accessed by different threads under different locks.
@@ -355,7 +354,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
         // that expected size of of a packet, then create 
         // that expected size of of a packet, then create 
         // smaller size packet.
         // smaller size packet.
         //
         //
-        computePacketChunkSize(Math.min(dfsClient.writePacketSize, freeInLastBlock), 
+        computePacketChunkSize(Math.min(dfsClient.getConf().writePacketSize, freeInLastBlock), 
             bytesPerChecksum);
             bytesPerChecksum);
       }
       }
 
 
@@ -426,8 +425,8 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
                 && dataQueue.size() == 0 && 
                 && dataQueue.size() == 0 && 
                 (stage != BlockConstructionStage.DATA_STREAMING || 
                 (stage != BlockConstructionStage.DATA_STREAMING || 
                  stage == BlockConstructionStage.DATA_STREAMING && 
                  stage == BlockConstructionStage.DATA_STREAMING && 
-                 now - lastPacket < dfsClient.socketTimeout/2)) || doSleep ) {
-              long timeout = dfsClient.socketTimeout/2 - (now-lastPacket);
+                 now - lastPacket < dfsClient.getConf().socketTimeout/2)) || doSleep ) {
+              long timeout = dfsClient.getConf().socketTimeout/2 - (now-lastPacket);
               timeout = timeout <= 0 ? 1000 : timeout;
               timeout = timeout <= 0 ? 1000 : timeout;
               timeout = (stage == BlockConstructionStage.DATA_STREAMING)?
               timeout = (stage == BlockConstructionStage.DATA_STREAMING)?
                  timeout : 1000;
                  timeout : 1000;
@@ -847,8 +846,8 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
             DataNode.SMALL_BUFFER_SIZE));
             DataNode.SMALL_BUFFER_SIZE));
 
 
         //send the TRANSFER_BLOCK request
         //send the TRANSFER_BLOCK request
-        Sender.opTransferBlock(out, block,
-            dfsClient.clientName, targets, blockToken);
+        new Sender(out).transferBlock(block, blockToken, dfsClient.clientName,
+            targets);
 
 
         //ack
         //ack
         in = new DataInputStream(NetUtils.getInputStream(sock));
         in = new DataInputStream(NetUtils.getInputStream(sock));
@@ -953,8 +952,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
     private DatanodeInfo[] nextBlockOutputStream(String client) throws IOException {
     private DatanodeInfo[] nextBlockOutputStream(String client) throws IOException {
       LocatedBlock lb = null;
       LocatedBlock lb = null;
       DatanodeInfo[] nodes = null;
       DatanodeInfo[] nodes = null;
-      int count = conf.getInt(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY,
-                              DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT);
+      int count = dfsClient.getConf().nBlockWriteRetry;
       boolean success = false;
       boolean success = false;
       do {
       do {
         hasError = false;
         hasError = false;
@@ -1021,10 +1019,9 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
         blockReplyStream = new DataInputStream(NetUtils.getInputStream(s));
         blockReplyStream = new DataInputStream(NetUtils.getInputStream(s));
 
 
         // send the request
         // send the request
-        Sender.opWriteBlock(out, block,
-            nodes.length, recoveryFlag ? stage.getRecoveryStage() : stage, newGS, 
-            block.getNumBytes(), bytesSent, dfsClient.clientName, null, nodes,
-            accessToken);
+        new Sender(out).writeBlock(block, accessToken, dfsClient.clientName,
+            nodes, null, recoveryFlag? stage.getRecoveryStage() : stage, 
+            nodes.length, block.getNumBytes(), bytesSent, newGS);
         checksum.writeHeader(out);
         checksum.writeHeader(out);
         out.flush();
         out.flush();
 
 
@@ -1079,9 +1076,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
     private LocatedBlock locateFollowingBlock(long start,
     private LocatedBlock locateFollowingBlock(long start,
         DatanodeInfo[] excludedNodes) 
         DatanodeInfo[] excludedNodes) 
         throws IOException, UnresolvedLinkException {
         throws IOException, UnresolvedLinkException {
-      int retries = 
-        conf.getInt(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY,
-                    DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT);
+      int retries = dfsClient.getConf().nBlockWriteLocateFollowingRetry;
       long sleeptime = 400;
       long sleeptime = 400;
       while (true) {
       while (true) {
         long localstart = System.currentTimeMillis();
         long localstart = System.currentTimeMillis();
@@ -1201,7 +1196,6 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
       int bytesPerChecksum, short replication) throws IOException {
       int bytesPerChecksum, short replication) throws IOException {
     super(new PureJavaCrc32(), bytesPerChecksum, 4);
     super(new PureJavaCrc32(), bytesPerChecksum, 4);
     this.dfsClient = dfsClient;
     this.dfsClient = dfsClient;
-    this.conf = dfsClient.conf;
     this.src = src;
     this.src = src;
     this.blockSize = blockSize;
     this.blockSize = blockSize;
     this.blockReplication = replication;
     this.blockReplication = replication;
@@ -1232,7 +1226,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
       throws IOException {
       throws IOException {
     this(dfsClient, src, blockSize, progress, bytesPerChecksum, replication);
     this(dfsClient, src, blockSize, progress, bytesPerChecksum, replication);
 
 
-    computePacketChunkSize(dfsClient.writePacketSize, bytesPerChecksum);
+    computePacketChunkSize(dfsClient.getConf().writePacketSize, bytesPerChecksum);
 
 
     try {
     try {
       dfsClient.namenode.create(
       dfsClient.namenode.create(
@@ -1269,7 +1263,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
       bytesCurBlock = lastBlock.getBlockSize();
       bytesCurBlock = lastBlock.getBlockSize();
       streamer = new DataStreamer(lastBlock, stat, bytesPerChecksum);
       streamer = new DataStreamer(lastBlock, stat, bytesPerChecksum);
     } else {
     } else {
-      computePacketChunkSize(dfsClient.writePacketSize, bytesPerChecksum);
+      computePacketChunkSize(dfsClient.getConf().writePacketSize, bytesPerChecksum);
       streamer = new DataStreamer();
       streamer = new DataStreamer();
     }
     }
     streamer.start();
     streamer.start();
@@ -1385,7 +1379,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
       }
       }
 
 
       if (!appendChunk) {
       if (!appendChunk) {
-        int psize = Math.min((int)(blockSize-bytesCurBlock), dfsClient.writePacketSize);
+        int psize = Math.min((int)(blockSize-bytesCurBlock), dfsClient.getConf().writePacketSize);
         computePacketChunkSize(psize, bytesPerChecksum);
         computePacketChunkSize(psize, bytesPerChecksum);
       }
       }
       //
       //

+ 21 - 13
hdfs/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -70,9 +70,8 @@ import org.xml.sax.XMLReader;
 import org.xml.sax.helpers.DefaultHandler;
 import org.xml.sax.helpers.DefaultHandler;
 import org.xml.sax.helpers.XMLReaderFactory;
 import org.xml.sax.helpers.XMLReaderFactory;
 
 
-
-
-/** An implementation of a protocol for accessing filesystems over HTTP.
+/**
+ * An implementation of a protocol for accessing filesystems over HTTP.
  * The following implementation provides a limited, read-only interface
  * The following implementation provides a limited, read-only interface
  * to a filesystem over HTTP.
  * to a filesystem over HTTP.
  * @see org.apache.hadoop.hdfs.server.namenode.ListPathsServlet
  * @see org.apache.hadoop.hdfs.server.namenode.ListPathsServlet
@@ -245,17 +244,26 @@ public class HftpFileSystem extends FileSystem {
     } 
     } 
   }
   }
 
 
-
-  /* 
-    Construct URL pointing to file on namenode
-  */
-  URL getNamenodeFileURL(Path f) throws IOException {
-    return getNamenodeURL("/data" + f.toUri().getPath(), "ugi=" + getUgiParameter());
+  /**
+   * Return a URL pointing to given path on the namenode.
+   *
+   * @param p path to obtain the URL for
+   * @return namenode URL referring to the given path
+   * @throws IOException on error constructing the URL
+   */
+  URL getNamenodeFileURL(Path p) throws IOException {
+    return getNamenodeURL("/data" + p.toUri().getPath(),
+                          "ugi=" + getUgiParameter());
   }
   }
 
 
-  /* 
-    Construct URL pointing to namenode. 
-  */
+  /**
+   * Return a URL pointing to given path on the namenode.
+   *
+   * @param path to obtain the URL for
+   * @param query string to append to the path
+   * @return namenode URL referring to the given path
+   * @throws IOException on error constructing the URL
+   */
   URL getNamenodeURL(String path, String query) throws IOException {
   URL getNamenodeURL(String path, String query) throws IOException {
     try {
     try {
       final URL url = new URI("http", null, nnAddr.getHostName(),
       final URL url = new URI("http", null, nnAddr.getHostName(),
@@ -305,7 +313,7 @@ public class HftpFileSystem extends FileSystem {
     try {
     try {
       connection.setRequestMethod("GET");
       connection.setRequestMethod("GET");
       connection.connect();
       connection.connect();
-    } catch(IOException ioe) {
+    } catch (IOException ioe) {
       throwIOExceptionFromConnection(connection, ioe);
       throwIOExceptionFromConnection(connection, ioe);
     }
     }
     return connection;
     return connection;

+ 2 - 1
hdfs/src/java/org/apache/hadoop/hdfs/protocol/DSQuotaExceededException.java

@@ -41,7 +41,8 @@ public class DSQuotaExceededException extends QuotaExceededException {
     String msg = super.getMessage();
     String msg = super.getMessage();
     if (msg == null) {
     if (msg == null) {
       return "The DiskSpace quota" + (pathName==null?"":(" of " + pathName)) + 
       return "The DiskSpace quota" + (pathName==null?"":(" of " + pathName)) + 
-          " is exceeded: quota=" + quota + " diskspace consumed=" + StringUtils.humanReadableInt(count);
+          " is exceeded: quota=" + StringUtils.humanReadableInt(quota) + 
+          " diskspace consumed=" + StringUtils.humanReadableInt(count);
     } else {
     } else {
       return msg;
       return msg;
     }
     }

+ 102 - 3
hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java

@@ -17,10 +17,16 @@
  */
  */
 package org.apache.hadoop.hdfs.protocol.datatransfer;
 package org.apache.hadoop.hdfs.protocol.datatransfer;
 
 
+import java.io.IOException;
+
 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.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
 
 
 /**
 /**
  * Transfer data to/from datanode using a streaming protocol.
  * Transfer data to/from datanode using a streaming protocol.
@@ -35,8 +41,101 @@ public interface DataTransferProtocol {
    * when protocol changes. It is not very obvious. 
    * when protocol changes. It is not very obvious. 
    */
    */
   /*
   /*
-   * Version 27:
-   *    Move DataTransferProtocol and the inner classes to a package.
+   * Version 28:
+   *    Declare methods in DataTransferProtocol interface.
+   */
+  public static final int DATA_TRANSFER_VERSION = 28;
+
+  /** 
+   * Read a block.
+   * 
+   * @param blk the block being read.
+   * @param blockToken security token for accessing the block.
+   * @param clientName client's name.
+   * @param blockOffset offset of the block.
+   * @param length maximum number of bytes for this read.
+   */
+  public void readBlock(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken,
+      final String clientName,
+      final long blockOffset,
+      final long length) throws IOException;
+
+  /**
+   * Write a block to a datanode pipeline.
+   * 
+   * @param blk the block being written.
+   * @param blockToken security token for accessing the block.
+   * @param clientName client's name.
+   * @param targets target datanodes in the pipeline.
+   * @param source source datanode.
+   * @param stage pipeline stage.
+   * @param pipelineSize the size of the pipeline.
+   * @param minBytesRcvd minimum number of bytes received.
+   * @param maxBytesRcvd maximum number of bytes received.
+   * @param latestGenerationStamp the latest generation stamp of the block.
+   */
+  public void writeBlock(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken,
+      final String clientName,
+      final DatanodeInfo[] targets,
+      final DatanodeInfo source,
+      final BlockConstructionStage stage,
+      final int pipelineSize,
+      final long minBytesRcvd,
+      final long maxBytesRcvd,
+      final long latestGenerationStamp) throws IOException;
+
+  /**
+   * Transfer a block to another datanode.
+   * The block stage must be
+   * either {@link BlockConstructionStage#TRANSFER_RBW}
+   * or {@link BlockConstructionStage#TRANSFER_FINALIZED}.
+   * 
+   * @param blk the block being transferred.
+   * @param blockToken security token for accessing the block.
+   * @param clientName client's name.
+   * @param targets target datanodes.
+   */
+  public void transferBlock(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken,
+      final String clientName,
+      final DatanodeInfo[] targets) throws IOException;
+
+  /**
+   * Receive a block from a source datanode
+   * and then notifies the namenode
+   * to remove the copy from the original datanode.
+   * Note that the source datanode and the original datanode can be different.
+   * It is used for balancing purpose.
+   * 
+   * @param blk the block being replaced.
+   * @param blockToken security token for accessing the block.
+   * @param delHint the hint for deleting the block in the original datanode.
+   * @param source the source datanode for receiving the block.
+   */
+  public void replaceBlock(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken,
+      final String delHint,
+      final DatanodeInfo source) throws IOException;
+
+  /**
+   * Copy a block. 
+   * It is used for balancing purpose.
+   * 
+   * @param blk the block being copied.
+   * @param blockToken security token for accessing the block.
+   */
+  public void copyBlock(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken) throws IOException;
+
+  /**
+   * Get block checksum (MD5 of CRC32).
+   * 
+   * @param blk a block.
+   * @param blockToken security token for accessing the block.
+   * @throws IOException
    */
    */
-  public static final int DATA_TRANSFER_VERSION = 27;
+  public void blockChecksum(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken) throws IOException;
 }
 }

+ 32 - 91
hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java

@@ -27,23 +27,26 @@ import java.io.IOException;
 
 
 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.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.security.token.Token;
 
 
 /** Receiver */
 /** Receiver */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
-public abstract class Receiver {
+public abstract class Receiver implements DataTransferProtocol {
+  protected final DataInputStream in;
+
+  /** Create a receiver for DataTransferProtocol with a socket. */
+  protected Receiver(final DataInputStream in) {
+    this.in = in;
+  }
+
   /** Read an Op.  It also checks protocol version. */
   /** Read an Op.  It also checks protocol version. */
-  protected final Op readOp(DataInputStream in) throws IOException {
+  protected final Op readOp() throws IOException {
     final short version = in.readShort();
     final short version = in.readShort();
     if (version != DataTransferProtocol.DATA_TRANSFER_VERSION) {
     if (version != DataTransferProtocol.DATA_TRANSFER_VERSION) {
       throw new IOException( "Version Mismatch (Expected: " +
       throw new IOException( "Version Mismatch (Expected: " +
@@ -54,11 +57,10 @@ public abstract class Receiver {
   }
   }
 
 
   /** Process op by the corresponding method. */
   /** Process op by the corresponding method. */
-  protected final void processOp(Op op, DataInputStream in
-      ) throws IOException {
+  protected final void processOp(Op op) throws IOException {
     switch(op) {
     switch(op) {
     case READ_BLOCK:
     case READ_BLOCK:
-      opReadBlock(in);
+      opReadBlock();
       break;
       break;
     case WRITE_BLOCK:
     case WRITE_BLOCK:
       opWriteBlock(in);
       opWriteBlock(in);
@@ -81,121 +83,60 @@ public abstract class Receiver {
   }
   }
 
 
   /** Receive OP_READ_BLOCK */
   /** Receive OP_READ_BLOCK */
-  private void opReadBlock(DataInputStream in) throws IOException {
+  private void opReadBlock() throws IOException {
     OpReadBlockProto proto = OpReadBlockProto.parseFrom(vintPrefixed(in));
     OpReadBlockProto proto = OpReadBlockProto.parseFrom(vintPrefixed(in));
-    
-    ExtendedBlock b = fromProto(
-        proto.getHeader().getBaseHeader().getBlock());
-    Token<BlockTokenIdentifier> token = fromProto(
-        proto.getHeader().getBaseHeader().getToken());
-
-    opReadBlock(in, b, proto.getOffset(), proto.getLen(),
-        proto.getHeader().getClientName(), token);
+    readBlock(fromProto(proto.getHeader().getBaseHeader().getBlock()),
+        fromProto(proto.getHeader().getBaseHeader().getToken()),
+        proto.getHeader().getClientName(),
+        proto.getOffset(),
+        proto.getLen());
   }
   }
-  /**
-   * Abstract OP_READ_BLOCK method. Read a block.
-   */
-  protected abstract void opReadBlock(DataInputStream in, ExtendedBlock blk,
-      long offset, long length, String client,
-      Token<BlockTokenIdentifier> blockToken) throws IOException;
   
   
   /** Receive OP_WRITE_BLOCK */
   /** Receive OP_WRITE_BLOCK */
   private void opWriteBlock(DataInputStream in) throws IOException {
   private void opWriteBlock(DataInputStream in) throws IOException {
     final OpWriteBlockProto proto = OpWriteBlockProto.parseFrom(vintPrefixed(in));
     final OpWriteBlockProto proto = OpWriteBlockProto.parseFrom(vintPrefixed(in));
-    opWriteBlock(in,
-        fromProto(proto.getHeader().getBaseHeader().getBlock()),
-        proto.getPipelineSize(),
-        fromProto(proto.getStage()),
-        proto.getLatestGenerationStamp(),
-        proto.getMinBytesRcvd(), proto.getMaxBytesRcvd(),
+    writeBlock(fromProto(proto.getHeader().getBaseHeader().getBlock()),
+        fromProto(proto.getHeader().getBaseHeader().getToken()),
         proto.getHeader().getClientName(),
         proto.getHeader().getClientName(),
-        fromProto(proto.getSource()),
         fromProtos(proto.getTargetsList()),
         fromProtos(proto.getTargetsList()),
-        fromProto(proto.getHeader().getBaseHeader().getToken()));
+        fromProto(proto.getSource()),
+        fromProto(proto.getStage()),
+        proto.getPipelineSize(),
+        proto.getMinBytesRcvd(), proto.getMaxBytesRcvd(),
+        proto.getLatestGenerationStamp());
   }
   }
 
 
-  /**
-   * Abstract OP_WRITE_BLOCK method. 
-   * Write a block.
-   */
-  protected abstract void opWriteBlock(DataInputStream in, ExtendedBlock blk,
-      int pipelineSize, BlockConstructionStage stage, long newGs,
-      long minBytesRcvd, long maxBytesRcvd, String client, DatanodeInfo src,
-      DatanodeInfo[] targets, Token<BlockTokenIdentifier> blockToken)
-      throws IOException;
-
   /** Receive {@link Op#TRANSFER_BLOCK} */
   /** Receive {@link Op#TRANSFER_BLOCK} */
   private void opTransferBlock(DataInputStream in) throws IOException {
   private void opTransferBlock(DataInputStream in) throws IOException {
     final OpTransferBlockProto proto =
     final OpTransferBlockProto proto =
       OpTransferBlockProto.parseFrom(vintPrefixed(in));
       OpTransferBlockProto.parseFrom(vintPrefixed(in));
-
-    opTransferBlock(in,
-        fromProto(proto.getHeader().getBaseHeader().getBlock()),
+    transferBlock(fromProto(proto.getHeader().getBaseHeader().getBlock()),
+        fromProto(proto.getHeader().getBaseHeader().getToken()),
         proto.getHeader().getClientName(),
         proto.getHeader().getClientName(),
-        fromProtos(proto.getTargetsList()),
-        fromProto(proto.getHeader().getBaseHeader().getToken()));
+        fromProtos(proto.getTargetsList()));
   }
   }
 
 
-  /**
-   * Abstract {@link Op#TRANSFER_BLOCK} method.
-   * For {@link BlockConstructionStage#TRANSFER_RBW}
-   * or {@link BlockConstructionStage#TRANSFER_FINALIZED}.
-   */
-  protected abstract void opTransferBlock(DataInputStream in, ExtendedBlock blk,
-      String client, DatanodeInfo[] targets,
-      Token<BlockTokenIdentifier> blockToken)
-      throws IOException;
-
   /** Receive OP_REPLACE_BLOCK */
   /** Receive OP_REPLACE_BLOCK */
   private void opReplaceBlock(DataInputStream in) throws IOException {
   private void opReplaceBlock(DataInputStream in) throws IOException {
     OpReplaceBlockProto proto = OpReplaceBlockProto.parseFrom(vintPrefixed(in));
     OpReplaceBlockProto proto = OpReplaceBlockProto.parseFrom(vintPrefixed(in));
-
-    opReplaceBlock(in,
-        fromProto(proto.getHeader().getBlock()),
+    replaceBlock(fromProto(proto.getHeader().getBlock()),
+        fromProto(proto.getHeader().getToken()),
         proto.getDelHint(),
         proto.getDelHint(),
-        fromProto(proto.getSource()),
-        fromProto(proto.getHeader().getToken()));
+        fromProto(proto.getSource()));
   }
   }
 
 
-  /**
-   * Abstract OP_REPLACE_BLOCK method.
-   * It is used for balancing purpose; send to a destination
-   */
-  protected abstract void opReplaceBlock(DataInputStream in,
-      ExtendedBlock blk, String delHint, DatanodeInfo src,
-      Token<BlockTokenIdentifier> blockToken) throws IOException;
-
   /** Receive OP_COPY_BLOCK */
   /** Receive OP_COPY_BLOCK */
   private void opCopyBlock(DataInputStream in) throws IOException {
   private void opCopyBlock(DataInputStream in) throws IOException {
     OpCopyBlockProto proto = OpCopyBlockProto.parseFrom(vintPrefixed(in));
     OpCopyBlockProto proto = OpCopyBlockProto.parseFrom(vintPrefixed(in));
-    
-    opCopyBlock(in,
-        fromProto(proto.getHeader().getBlock()),
+    copyBlock(fromProto(proto.getHeader().getBlock()),
         fromProto(proto.getHeader().getToken()));
         fromProto(proto.getHeader().getToken()));
   }
   }
 
 
-  /**
-   * Abstract OP_COPY_BLOCK method. It is used for balancing purpose; send to
-   * a proxy source.
-   */
-  protected abstract void opCopyBlock(DataInputStream in, ExtendedBlock blk,
-      Token<BlockTokenIdentifier> blockToken)
-      throws IOException;
-
   /** Receive OP_BLOCK_CHECKSUM */
   /** Receive OP_BLOCK_CHECKSUM */
   private void opBlockChecksum(DataInputStream in) throws IOException {
   private void opBlockChecksum(DataInputStream in) throws IOException {
     OpBlockChecksumProto proto = OpBlockChecksumProto.parseFrom(vintPrefixed(in));
     OpBlockChecksumProto proto = OpBlockChecksumProto.parseFrom(vintPrefixed(in));
     
     
-    opBlockChecksum(in,
-        fromProto(proto.getHeader().getBlock()),
+    blockChecksum(fromProto(proto.getHeader().getBlock()),
         fromProto(proto.getHeader().getToken()));
         fromProto(proto.getHeader().getToken()));
   }
   }
-
-  /**
-   * Abstract OP_BLOCK_CHECKSUM method.
-   * Get the checksum of a block 
-   */
-  protected abstract void opBlockChecksum(DataInputStream in,
-      ExtendedBlock blk, Token<BlockTokenIdentifier> blockToken)
-      throws IOException;
 }
 }

+ 50 - 38
hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java

@@ -44,7 +44,14 @@ import com.google.protobuf.Message;
 /** Sender */
 /** Sender */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
-public class Sender {
+public class Sender implements DataTransferProtocol {
+  private final DataOutputStream out;
+
+  /** Create a sender for DataTransferProtocol with a output stream. */
+  public Sender(final DataOutputStream out) {
+    this.out = out;    
+  }
+
   /** Initialize a operation. */
   /** Initialize a operation. */
   private static void op(final DataOutput out, final Op op
   private static void op(final DataOutput out, final Op op
       ) throws IOException {
       ) throws IOException {
@@ -59,79 +66,85 @@ public class Sender {
     out.flush();
     out.flush();
   }
   }
 
 
-  /** Send OP_READ_BLOCK */
-  public static void opReadBlock(DataOutputStream out, ExtendedBlock blk,
-      long blockOffset, long blockLen, String clientName,
-      Token<BlockTokenIdentifier> blockToken)
-      throws IOException {
+  @Override
+  public void readBlock(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken,
+      final String clientName,
+      final long blockOffset,
+      final long length) throws IOException {
 
 
     OpReadBlockProto proto = OpReadBlockProto.newBuilder()
     OpReadBlockProto proto = OpReadBlockProto.newBuilder()
       .setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName, blockToken))
       .setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName, blockToken))
       .setOffset(blockOffset)
       .setOffset(blockOffset)
-      .setLen(blockLen)
+      .setLen(length)
       .build();
       .build();
 
 
     send(out, Op.READ_BLOCK, proto);
     send(out, Op.READ_BLOCK, proto);
   }
   }
   
   
 
 
-  /** Send OP_WRITE_BLOCK */
-  public static void opWriteBlock(DataOutputStream out, ExtendedBlock blk,
-      int pipelineSize, BlockConstructionStage stage, long newGs,
-      long minBytesRcvd, long maxBytesRcvd, String client, DatanodeInfo src,
-      DatanodeInfo[] targets, Token<BlockTokenIdentifier> blockToken)
-      throws IOException {
-    ClientOperationHeaderProto header = DataTransferProtoUtil.buildClientHeader(blk, client,
-        blockToken);
+  @Override
+  public void writeBlock(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken,
+      final String clientName,
+      final DatanodeInfo[] targets,
+      final DatanodeInfo source,
+      final BlockConstructionStage stage,
+      final int pipelineSize,
+      final long minBytesRcvd,
+      final long maxBytesRcvd,
+      final long latestGenerationStamp) throws IOException {
+    ClientOperationHeaderProto header = DataTransferProtoUtil.buildClientHeader(
+        blk, clientName, blockToken);
     
     
     OpWriteBlockProto.Builder proto = OpWriteBlockProto.newBuilder()
     OpWriteBlockProto.Builder proto = OpWriteBlockProto.newBuilder()
       .setHeader(header)
       .setHeader(header)
-      .addAllTargets(
-          toProtos(targets, 1))
+      .addAllTargets(toProtos(targets, 1))
       .setStage(toProto(stage))
       .setStage(toProto(stage))
       .setPipelineSize(pipelineSize)
       .setPipelineSize(pipelineSize)
       .setMinBytesRcvd(minBytesRcvd)
       .setMinBytesRcvd(minBytesRcvd)
       .setMaxBytesRcvd(maxBytesRcvd)
       .setMaxBytesRcvd(maxBytesRcvd)
-      .setLatestGenerationStamp(newGs);
+      .setLatestGenerationStamp(latestGenerationStamp);
     
     
-    if (src != null) {
-      proto.setSource(toProto(src));
+    if (source != null) {
+      proto.setSource(toProto(source));
     }
     }
 
 
     send(out, Op.WRITE_BLOCK, proto.build());
     send(out, Op.WRITE_BLOCK, proto.build());
   }
   }
 
 
-  /** Send {@link Op#TRANSFER_BLOCK} */
-  public static void opTransferBlock(DataOutputStream out, ExtendedBlock blk,
-      String client, DatanodeInfo[] targets,
-      Token<BlockTokenIdentifier> blockToken) throws IOException {
+  @Override
+  public void transferBlock(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken,
+      final String clientName,
+      final DatanodeInfo[] targets) throws IOException {
     
     
     OpTransferBlockProto proto = OpTransferBlockProto.newBuilder()
     OpTransferBlockProto proto = OpTransferBlockProto.newBuilder()
       .setHeader(DataTransferProtoUtil.buildClientHeader(
       .setHeader(DataTransferProtoUtil.buildClientHeader(
-          blk, client, blockToken))
+          blk, clientName, blockToken))
       .addAllTargets(toProtos(targets, 0))
       .addAllTargets(toProtos(targets, 0))
       .build();
       .build();
 
 
     send(out, Op.TRANSFER_BLOCK, proto);
     send(out, Op.TRANSFER_BLOCK, proto);
   }
   }
 
 
-  /** Send OP_REPLACE_BLOCK */
-  public static void opReplaceBlock(DataOutputStream out,
-      ExtendedBlock blk, String delHint, DatanodeInfo src,
-      Token<BlockTokenIdentifier> blockToken) throws IOException {
+  @Override
+  public void replaceBlock(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken,
+      final String delHint,
+      final DatanodeInfo source) throws IOException {
     OpReplaceBlockProto proto = OpReplaceBlockProto.newBuilder()
     OpReplaceBlockProto proto = OpReplaceBlockProto.newBuilder()
       .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
       .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
       .setDelHint(delHint)
       .setDelHint(delHint)
-      .setSource(toProto(src))
+      .setSource(toProto(source))
       .build();
       .build();
     
     
     send(out, Op.REPLACE_BLOCK, proto);
     send(out, Op.REPLACE_BLOCK, proto);
   }
   }
 
 
-  /** Send OP_COPY_BLOCK */
-  public static void opCopyBlock(DataOutputStream out, ExtendedBlock blk,
-      Token<BlockTokenIdentifier> blockToken)
-      throws IOException {
+  @Override
+  public void copyBlock(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken) throws IOException {
     OpCopyBlockProto proto = OpCopyBlockProto.newBuilder()
     OpCopyBlockProto proto = OpCopyBlockProto.newBuilder()
       .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
       .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
       .build();
       .build();
@@ -139,10 +152,9 @@ public class Sender {
     send(out, Op.COPY_BLOCK, proto);
     send(out, Op.COPY_BLOCK, proto);
   }
   }
 
 
-  /** Send OP_BLOCK_CHECKSUM */
-  public static void opBlockChecksum(DataOutputStream out, ExtendedBlock blk,
-      Token<BlockTokenIdentifier> blockToken)
-      throws IOException {
+  @Override
+  public void blockChecksum(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken) throws IOException {
     OpBlockChecksumProto proto = OpBlockChecksumProto.newBuilder()
     OpBlockChecksumProto proto = OpBlockChecksumProto.newBuilder()
       .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
       .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
       .build();
       .build();

+ 4 - 4
hdfs/src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -61,10 +61,10 @@ 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.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.common.Util;
-import org.apache.hadoop.hdfs.server.namenode.BlockPlacementPolicy;
-import org.apache.hadoop.hdfs.server.namenode.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
@@ -348,8 +348,8 @@ public class Balancer {
     private void sendRequest(DataOutputStream out) throws IOException {
     private void sendRequest(DataOutputStream out) throws IOException {
       final ExtendedBlock eb = new ExtendedBlock(nnc.blockpoolID, block.getBlock());
       final ExtendedBlock eb = new ExtendedBlock(nnc.blockpoolID, block.getBlock());
       final Token<BlockTokenIdentifier> accessToken = nnc.getAccessToken(eb);
       final Token<BlockTokenIdentifier> accessToken = nnc.getAccessToken(eb);
-      Sender.opReplaceBlock(out, eb, source.getStorageID(), 
-          proxySource.getDatanode(), accessToken);
+      new Sender(out).replaceBlock(eb, accessToken,
+          source.getStorageID(), proxySource.getDatanode());
     }
     }
     
     
     /* Receive a block copy response from the input stream */ 
     /* Receive a block copy response from the input stream */ 

+ 14 - 13
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfo.java → hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -15,16 +15,17 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.util.LightWeightGSet;
 import org.apache.hadoop.hdfs.util.LightWeightGSet;
 
 
 /**
 /**
  * Internal class for block metadata.
  * Internal class for block metadata.
  */
  */
-class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
+public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   private INodeFile inode;
   private INodeFile inode;
 
 
   /** For implementing {@link LightWeightGSet.LinkedElement} interface */
   /** For implementing {@link LightWeightGSet.LinkedElement} interface */
@@ -44,12 +45,12 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * Construct an entry for blocksmap
    * Construct an entry for blocksmap
    * @param replication the block's replication factor
    * @param replication the block's replication factor
    */
    */
-  protected BlockInfo(int replication) {
+  public BlockInfo(int replication) {
     this.triplets = new Object[3*replication];
     this.triplets = new Object[3*replication];
     this.inode = null;
     this.inode = null;
   }
   }
   
   
-  protected BlockInfo(Block blk, int replication) {
+  public BlockInfo(Block blk, int replication) {
     super(blk);
     super(blk);
     this.triplets = new Object[3*replication];
     this.triplets = new Object[3*replication];
     this.inode = null;
     this.inode = null;
@@ -65,11 +66,11 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     this.inode = from.inode;
     this.inode = from.inode;
   }
   }
 
 
-  INodeFile getINode() {
+  public INodeFile getINode() {
     return inode;
     return inode;
   }
   }
 
 
-  void setINode(INodeFile inode) {
+  public void setINode(INodeFile inode) {
     this.inode = inode;
     this.inode = inode;
   }
   }
 
 
@@ -162,7 +163,7 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   /**
   /**
    * Add data-node this block belongs to.
    * Add data-node this block belongs to.
    */
    */
-  boolean addNode(DatanodeDescriptor node) {
+  public boolean addNode(DatanodeDescriptor node) {
     if(findDatanode(node) >= 0) // the node is already there
     if(findDatanode(node) >= 0) // the node is already there
       return false;
       return false;
     // find the last null node
     // find the last null node
@@ -176,7 +177,7 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   /**
   /**
    * Remove data-node from the block.
    * Remove data-node from the block.
    */
    */
-  boolean removeNode(DatanodeDescriptor node) {
+  public boolean removeNode(DatanodeDescriptor node) {
     int dnIndex = findDatanode(node);
     int dnIndex = findDatanode(node);
     if(dnIndex < 0) // the node is not found
     if(dnIndex < 0) // the node is not found
       return false;
       return false;
@@ -218,7 +219,7 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * If the head is null then form a new list.
    * If the head is null then form a new list.
    * @return current block as the new head of the list.
    * @return current block as the new head of the list.
    */
    */
-  BlockInfo listInsert(BlockInfo head, DatanodeDescriptor dn) {
+  public BlockInfo listInsert(BlockInfo head, DatanodeDescriptor dn) {
     int dnIndex = this.findDatanode(dn);
     int dnIndex = this.findDatanode(dn);
     assert dnIndex >= 0 : "Data node is not found: current";
     assert dnIndex >= 0 : "Data node is not found: current";
     assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
     assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
@@ -238,7 +239,7 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * @return the new head of the list or null if the list becomes
    * @return the new head of the list or null if the list becomes
    * empty after deletion.
    * empty after deletion.
    */
    */
-  BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
+  public BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
     if(head == null)
     if(head == null)
       return null;
       return null;
     int dnIndex = this.findDatanode(dn);
     int dnIndex = this.findDatanode(dn);
@@ -284,7 +285,7 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * to {@link BlockInfoUnderConstruction}.
    * to {@link BlockInfoUnderConstruction}.
    * @return {@link BlockUCState#COMPLETE}
    * @return {@link BlockUCState#COMPLETE}
    */
    */
-  BlockUCState getBlockUCState() {
+  public BlockUCState getBlockUCState() {
     return BlockUCState.COMPLETE;
     return BlockUCState.COMPLETE;
   }
   }
 
 
@@ -293,7 +294,7 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * 
    * 
    * @return true if the state of the block is {@link BlockUCState#COMPLETE}
    * @return true if the state of the block is {@link BlockUCState#COMPLETE}
    */
    */
-  boolean isComplete() {
+  public boolean isComplete() {
     return getBlockUCState().equals(BlockUCState.COMPLETE);
     return getBlockUCState().equals(BlockUCState.COMPLETE);
   }
   }
 
 
@@ -302,7 +303,7 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * 
    * 
    * @return BlockInfoUnderConstruction -  an under construction block.
    * @return BlockInfoUnderConstruction -  an under construction block.
    */
    */
-  BlockInfoUnderConstruction convertToBlockUnderConstruction(
+  public BlockInfoUnderConstruction convertToBlockUnderConstruction(
       BlockUCState s, DatanodeDescriptor[] targets) {
       BlockUCState s, DatanodeDescriptor[] targets) {
     if(isComplete()) {
     if(isComplete()) {
       return new BlockInfoUnderConstruction(
       return new BlockInfoUnderConstruction(

+ 17 - 10
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfoUnderConstruction.java → hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -24,12 +24,13 @@ import java.util.List;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 
 
 /**
 /**
  * Represents a block that is currently being constructed.<br>
  * Represents a block that is currently being constructed.<br>
  * This is usually the last block of a file opened for write or append.
  * This is usually the last block of a file opened for write or append.
  */
  */
-class BlockInfoUnderConstruction extends BlockInfo {
+public class BlockInfoUnderConstruction extends BlockInfo {
   /** Block state. See {@link BlockUCState} */
   /** Block state. See {@link BlockUCState} */
   private BlockUCState blockUCState;
   private BlockUCState blockUCState;
 
 
@@ -128,11 +129,14 @@ class BlockInfoUnderConstruction extends BlockInfo {
    * Create block and set its state to
    * Create block and set its state to
    * {@link BlockUCState#UNDER_CONSTRUCTION}.
    * {@link BlockUCState#UNDER_CONSTRUCTION}.
    */
    */
-  BlockInfoUnderConstruction(Block blk, int replication) {
+  public BlockInfoUnderConstruction(Block blk, int replication) {
     this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
     this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
   }
   }
 
 
-  BlockInfoUnderConstruction(Block blk, int replication,
+  /**
+   * Create a block that is currently being constructed.
+   */
+  public BlockInfoUnderConstruction(Block blk, int replication,
                              BlockUCState state,
                              BlockUCState state,
                              DatanodeDescriptor[] targets) {
                              DatanodeDescriptor[] targets) {
     super(blk, replication);
     super(blk, replication);
@@ -160,7 +164,8 @@ class BlockInfoUnderConstruction extends BlockInfo {
     return new BlockInfo(this);
     return new BlockInfo(this);
   }
   }
 
 
-  void setExpectedLocations(DatanodeDescriptor[] targets) {
+  /** Set expected locations */
+  public void setExpectedLocations(DatanodeDescriptor[] targets) {
     int numLocations = targets == null ? 0 : targets.length;
     int numLocations = targets == null ? 0 : targets.length;
     this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
     this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
     for(int i = 0; i < numLocations; i++)
     for(int i = 0; i < numLocations; i++)
@@ -172,7 +177,7 @@ class BlockInfoUnderConstruction extends BlockInfo {
    * Create array of expected replica locations
    * Create array of expected replica locations
    * (as has been assigned by chooseTargets()).
    * (as has been assigned by chooseTargets()).
    */
    */
-  DatanodeDescriptor[] getExpectedLocations() {
+  public DatanodeDescriptor[] getExpectedLocations() {
     int numLocations = replicas == null ? 0 : replicas.size();
     int numLocations = replicas == null ? 0 : replicas.size();
     DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocations];
     DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocations];
     for(int i = 0; i < numLocations; i++)
     for(int i = 0; i < numLocations; i++)
@@ -180,7 +185,8 @@ class BlockInfoUnderConstruction extends BlockInfo {
     return locations;
     return locations;
   }
   }
 
 
-  int getNumExpectedLocations() {
+  /** Get the number of expected locations */
+  public int getNumExpectedLocations() {
     return replicas == null ? 0 : replicas.size();
     return replicas == null ? 0 : replicas.size();
   }
   }
 
 
@@ -189,7 +195,7 @@ class BlockInfoUnderConstruction extends BlockInfo {
    * @see BlockUCState
    * @see BlockUCState
    */
    */
   @Override // BlockInfo
   @Override // BlockInfo
-  BlockUCState getBlockUCState() {
+  public BlockUCState getBlockUCState() {
     return blockUCState;
     return blockUCState;
   }
   }
 
 
@@ -197,7 +203,8 @@ class BlockInfoUnderConstruction extends BlockInfo {
     blockUCState = s;
     blockUCState = s;
   }
   }
 
 
-  long getBlockRecoveryId() {
+  /** Get block recovery ID */
+  public long getBlockRecoveryId() {
     return blockRecoveryId;
     return blockRecoveryId;
   }
   }
 
 
@@ -220,7 +227,7 @@ class BlockInfoUnderConstruction extends BlockInfo {
    * Find the first alive data-node starting from the previous primary and
    * Find the first alive data-node starting from the previous primary and
    * make it primary.
    * make it primary.
    */
    */
-  void initializeBlockRecovery(long recoveryId) {
+  public void initializeBlockRecovery(long recoveryId) {
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;
     blockRecoveryId = recoveryId;
     if (replicas.size() == 0) {
     if (replicas.size() == 0) {

+ 103 - 73
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java → hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
@@ -39,10 +39,14 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
 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;
+import org.apache.hadoop.hdfs.server.blockmanagement.UnderReplicatedBlocks.BlockIterator;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
-import org.apache.hadoop.hdfs.server.namenode.UnderReplicatedBlocks;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 
 
 /**
 /**
  * Keeps information related to the blocks stored in the Hadoop cluster.
  * Keeps information related to the blocks stored in the Hadoop cluster.
@@ -57,18 +61,43 @@ public class BlockManager {
 
 
   private final FSNamesystem namesystem;
   private final FSNamesystem namesystem;
 
 
-  volatile long pendingReplicationBlocksCount = 0L;
-  volatile long corruptReplicaBlocksCount = 0L;
-  volatile long underReplicatedBlocksCount = 0L;
-  volatile long scheduledReplicationBlocksCount = 0L;
-  volatile long excessBlocksCount = 0L;
-  volatile long pendingDeletionBlocksCount = 0L;
+  private volatile long pendingReplicationBlocksCount = 0L;
+  private volatile long corruptReplicaBlocksCount = 0L;
+  private volatile long underReplicatedBlocksCount = 0L;
+  public volatile long scheduledReplicationBlocksCount = 0L;
+  private volatile long excessBlocksCount = 0L;
+  private volatile long pendingDeletionBlocksCount = 0L;
 
 
-  //
-  // Mapping: Block -> { INode, datanodes, self ref }
-  // Updated only in response to client-sent information.
-  //
-  final BlocksMap blocksMap;
+  /** Used by metrics */
+  public long getPendingReplicationBlocksCount() {
+    return pendingReplicationBlocksCount;
+  }
+  /** Used by metrics */
+  public long getUnderReplicatedBlocksCount() {
+    return underReplicatedBlocksCount;
+  }
+  /** Used by metrics */
+  public long getCorruptReplicaBlocksCount() {
+    return corruptReplicaBlocksCount;
+  }
+  /** Used by metrics */
+  public long getScheduledReplicationBlocksCount() {
+    return scheduledReplicationBlocksCount;
+  }
+  /** Used by metrics */
+  public long getPendingDeletionBlocksCount() {
+    return pendingDeletionBlocksCount;
+  }
+  /** Used by metrics */
+  public long getExcessBlocksCount() {
+    return excessBlocksCount;
+  }
+
+  /**
+   * Mapping: Block -> { INode, datanodes, self ref }
+   * Updated only in response to client-sent information.
+   */
+  public final BlocksMap blocksMap;
 
 
   //
   //
   // Store blocks-->datanodedescriptor(s) map of corrupt replicas
   // Store blocks-->datanodedescriptor(s) map of corrupt replicas
@@ -90,24 +119,24 @@ public class BlockManager {
   // eventually remove these extras.
   // eventually remove these extras.
   // Mapping: StorageID -> TreeSet<Block>
   // Mapping: StorageID -> TreeSet<Block>
   //
   //
-  Map<String, Collection<Block>> excessReplicateMap =
+  public final Map<String, Collection<Block>> excessReplicateMap =
     new TreeMap<String, Collection<Block>>();
     new TreeMap<String, Collection<Block>>();
 
 
   //
   //
   // Store set of Blocks that need to be replicated 1 or more times.
   // Store set of Blocks that need to be replicated 1 or more times.
   // We also store pending replication-orders.
   // We also store pending replication-orders.
   //
   //
-  UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
+  public UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
   private PendingReplicationBlocks pendingReplications;
   private PendingReplicationBlocks pendingReplications;
 
 
   //  The maximum number of replicas allowed for a block
   //  The maximum number of replicas allowed for a block
-  int maxReplication;
+  public int maxReplication;
   //  How many outgoing replication streams a given node should have at one time
   //  How many outgoing replication streams a given node should have at one time
-  int maxReplicationStreams;
+  public int maxReplicationStreams;
   // Minimum copies needed or else write is disallowed
   // Minimum copies needed or else write is disallowed
-  int minReplication;
+  public int minReplication;
   // Default number of replicas
   // Default number of replicas
-  int defaultReplication;
+  public int defaultReplication;
   // How many entries are returned by getCorruptInodes()
   // How many entries are returned by getCorruptInodes()
   int maxCorruptFilesReturned;
   int maxCorruptFilesReturned;
   
   
@@ -121,9 +150,9 @@ public class BlockManager {
   Random r = new Random();
   Random r = new Random();
 
 
   // for block replicas placement
   // for block replicas placement
-  BlockPlacementPolicy replicator;
+  public BlockPlacementPolicy replicator;
 
 
-  BlockManager(FSNamesystem fsn, Configuration conf) throws IOException {
+  public BlockManager(FSNamesystem fsn, Configuration conf) throws IOException {
     this(fsn, conf, DEFAULT_INITIAL_MAP_CAPACITY);
     this(fsn, conf, DEFAULT_INITIAL_MAP_CAPACITY);
   }
   }
   
   
@@ -178,16 +207,16 @@ public class BlockManager {
     FSNamesystem.LOG.info("shouldCheckForEnoughRacks = " + shouldCheckForEnoughRacks);
     FSNamesystem.LOG.info("shouldCheckForEnoughRacks = " + shouldCheckForEnoughRacks);
   }
   }
 
 
-  void activate() {
+  public void activate() {
     pendingReplications.start();
     pendingReplications.start();
   }
   }
 
 
-  void close() {
+  public void close() {
     if (pendingReplications != null) pendingReplications.stop();
     if (pendingReplications != null) pendingReplications.stop();
     blocksMap.close();
     blocksMap.close();
   }
   }
 
 
-  void metaSave(PrintWriter out) {
+  public void metaSave(PrintWriter out) {
     //
     //
     // Dump contents of neededReplication
     // Dump contents of neededReplication
     //
     //
@@ -249,7 +278,7 @@ public class BlockManager {
    * @param block
    * @param block
    * @return true if the block has minimum replicas
    * @return true if the block has minimum replicas
    */
    */
-  boolean checkMinReplication(Block block) {
+  public boolean checkMinReplication(Block block) {
     return (countNodes(block).liveReplicas() >= minReplication);
     return (countNodes(block).liveReplicas() >= minReplication);
   }
   }
 
 
@@ -297,7 +326,7 @@ public class BlockManager {
    * @throws IOException if the block does not have at least a minimal number
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    * of replicas reported from data-nodes.
    */
    */
-  void commitOrCompleteLastBlock(INodeFileUnderConstruction fileINode, 
+  public void commitOrCompleteLastBlock(INodeFileUnderConstruction fileINode, 
       Block commitBlock) throws IOException {
       Block commitBlock) throws IOException {
     
     
     if(commitBlock == null)
     if(commitBlock == null)
@@ -362,7 +391,7 @@ public class BlockManager {
    * @param fileINode file
    * @param fileINode file
    * @return the last block locations if the block is partial or null otherwise
    * @return the last block locations if the block is partial or null otherwise
    */
    */
-  LocatedBlock convertLastBlockToUnderConstruction(
+  public LocatedBlock convertLastBlockToUnderConstruction(
       INodeFileUnderConstruction fileINode) throws IOException {
       INodeFileUnderConstruction fileINode) throws IOException {
     BlockInfo oldBlock = fileINode.getLastBlock();
     BlockInfo oldBlock = fileINode.getLastBlock();
     if(oldBlock == null ||
     if(oldBlock == null ||
@@ -393,7 +422,7 @@ public class BlockManager {
   /**
   /**
    * Get all valid locations of the block
    * Get all valid locations of the block
    */
    */
-  ArrayList<String> getValidLocations(Block block) {
+  public ArrayList<String> getValidLocations(Block block) {
     ArrayList<String> machineSet =
     ArrayList<String> machineSet =
       new ArrayList<String>(blocksMap.numNodes(block));
       new ArrayList<String>(blocksMap.numNodes(block));
     for(Iterator<DatanodeDescriptor> it =
     for(Iterator<DatanodeDescriptor> it =
@@ -407,7 +436,7 @@ public class BlockManager {
     return machineSet;
     return machineSet;
   }
   }
 
 
-  List<LocatedBlock> getBlockLocations(BlockInfo[] blocks, long offset,
+  public List<LocatedBlock> getBlockLocations(BlockInfo[] blocks, long offset,
       long length, int nrBlocksToReturn) throws IOException {
       long length, int nrBlocksToReturn) throws IOException {
     int curBlk = 0;
     int curBlk = 0;
     long curPos = 0, blkSize = 0;
     long curPos = 0, blkSize = 0;
@@ -436,11 +465,15 @@ public class BlockManager {
     return results;
     return results;
   }
   }
 
 
-  /** @param needBlockToken 
-   * @return a LocatedBlock for the given block */
-  LocatedBlock getBlockLocation(final BlockInfo blk, final long pos
+  /** @return a LocatedBlock for the given block */
+  public LocatedBlock getBlockLocation(final BlockInfo blk, final long pos
       ) throws IOException {
       ) throws IOException {
-    if (!blk.isComplete()) {
+    if (blk instanceof BlockInfoUnderConstruction) {
+      if (blk.isComplete()) {
+        throw new IOException(
+            "blk instanceof BlockInfoUnderConstruction && blk.isComplete()"
+            + ", blk=" + blk);
+      }
       final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)blk;
       final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)blk;
       final DatanodeDescriptor[] locations = uc.getExpectedLocations();
       final DatanodeDescriptor[] locations = uc.getExpectedLocations();
       return namesystem.createLocatedBlock(uc, locations, pos, false);
       return namesystem.createLocatedBlock(uc, locations, pos, false);
@@ -476,7 +509,7 @@ public class BlockManager {
    * Check whether the replication parameter is within the range
    * Check whether the replication parameter is within the range
    * determined by system configuration.
    * determined by system configuration.
    */
    */
-   void verifyReplication(String src,
+   public void verifyReplication(String src,
                           short replication,
                           short replication,
                           String clientName) throws IOException {
                           String clientName) throws IOException {
 
 
@@ -544,7 +577,7 @@ public class BlockManager {
    * @param b block
    * @param b block
    * @param dn datanode
    * @param dn datanode
    */
    */
-  void addToInvalidates(Block b, DatanodeInfo dn) {
+  public void addToInvalidates(Block b, DatanodeInfo dn) {
     addToInvalidates(b, dn, true);
     addToInvalidates(b, dn, true);
   }
   }
 
 
@@ -585,7 +618,7 @@ public class BlockManager {
     }
     }
   }
   }
 
 
-  void findAndMarkBlockAsCorrupt(Block blk,
+  public void findAndMarkBlockAsCorrupt(Block blk,
                                  DatanodeInfo dn) throws IOException {
                                  DatanodeInfo dn) throws IOException {
     BlockInfo storedBlock = getStoredBlock(blk);
     BlockInfo storedBlock = getStoredBlock(blk);
     if (storedBlock == null) {
     if (storedBlock == null) {
@@ -668,14 +701,14 @@ public class BlockManager {
     }
     }
   }
   }
 
 
-  void updateState() {
+  public void updateState() {
     pendingReplicationBlocksCount = pendingReplications.size();
     pendingReplicationBlocksCount = pendingReplications.size();
     underReplicatedBlocksCount = neededReplications.size();
     underReplicatedBlocksCount = neededReplications.size();
     corruptReplicaBlocksCount = corruptReplicas.size();
     corruptReplicaBlocksCount = corruptReplicas.size();
   }
   }
 
 
   /** Return number of under-replicated but not missing blocks */
   /** Return number of under-replicated but not missing blocks */
-  int getUnderReplicatedNotMissingBlocks() {
+  public int getUnderReplicatedNotMissingBlocks() {
     return neededReplications.getUnderReplicatedBlockCount();
     return neededReplications.getUnderReplicatedBlockCount();
   }
   }
   
   
@@ -684,7 +717,7 @@ public class BlockManager {
    * @param nodesToProcess number of datanodes to schedule deletion work
    * @param nodesToProcess number of datanodes to schedule deletion work
    * @return total number of block for deletion
    * @return total number of block for deletion
    */
    */
-  int computeInvalidateWork(int nodesToProcess) {
+  public int computeInvalidateWork(int nodesToProcess) {
     int numOfNodes = recentInvalidateSets.size();
     int numOfNodes = recentInvalidateSets.size();
     nodesToProcess = Math.min(numOfNodes, nodesToProcess);
     nodesToProcess = Math.min(numOfNodes, nodesToProcess);
 
 
@@ -724,7 +757,7 @@ public class BlockManager {
    *
    *
    * @return number of blocks scheduled for replication during this iteration.
    * @return number of blocks scheduled for replication during this iteration.
    */
    */
-  int computeReplicationWork(int blocksToProcess) throws IOException {
+  public int computeReplicationWork(int blocksToProcess) throws IOException {
     // Choose the blocks to be replicated
     // Choose the blocks to be replicated
     List<List<Block>> blocksToReplicate =
     List<List<Block>> blocksToReplicate =
       chooseUnderReplicatedBlocks(blocksToProcess);
       chooseUnderReplicatedBlocks(blocksToProcess);
@@ -1031,7 +1064,7 @@ public class BlockManager {
    * If there were any replication requests that timed out, reap them
    * If there were any replication requests that timed out, reap them
    * and put them back into the neededReplication queue
    * and put them back into the neededReplication queue
    */
    */
-  void processPendingReplications() {
+  public void processPendingReplications() {
     Block[] timedOutItems = pendingReplications.getTimedOutBlocks();
     Block[] timedOutItems = pendingReplications.getTimedOutBlocks();
     if (timedOutItems != null) {
     if (timedOutItems != null) {
       namesystem.writeLock();
       namesystem.writeLock();
@@ -1464,7 +1497,7 @@ public class BlockManager {
     short fileReplication = fileINode.getReplication();
     short fileReplication = fileINode.getReplication();
     if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
     if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
       neededReplications.remove(storedBlock, numCurrentReplica,
       neededReplications.remove(storedBlock, numCurrentReplica,
-          num.decommissionedReplicas, fileReplication);
+          num.decommissionedReplicas(), fileReplication);
     } else {
     } else {
       updateNeededReplications(storedBlock, curReplicaDelta, 0);
       updateNeededReplications(storedBlock, curReplicaDelta, 0);
     }
     }
@@ -1525,7 +1558,7 @@ public class BlockManager {
    * For each block in the name-node verify whether it belongs to any file,
    * For each block in the name-node verify whether it belongs to any file,
    * over or under replicated. Place it into the respective queue.
    * over or under replicated. Place it into the respective queue.
    */
    */
-  void processMisReplicatedBlocks() {
+  public void processMisReplicatedBlocks() {
     long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0;
     long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0;
     namesystem.writeLock();
     namesystem.writeLock();
     try {
     try {
@@ -1570,7 +1603,7 @@ public class BlockManager {
    * If there are any extras, call chooseExcessReplicates() to
    * If there are any extras, call chooseExcessReplicates() to
    * mark them in the excessReplicateMap.
    * mark them in the excessReplicateMap.
    */
    */
-  void processOverReplicatedBlock(Block block, short replication,
+  public void processOverReplicatedBlock(Block block, short replication,
       DatanodeDescriptor addedNode, DatanodeDescriptor delNodeHint) {
       DatanodeDescriptor addedNode, DatanodeDescriptor delNodeHint) {
     assert namesystem.hasWriteLock();
     assert namesystem.hasWriteLock();
     if (addedNode == delNodeHint) {
     if (addedNode == delNodeHint) {
@@ -1597,7 +1630,7 @@ public class BlockManager {
         addedNode, delNodeHint, replicator);
         addedNode, delNodeHint, replicator);
   }
   }
 
 
-  void addToExcessReplicate(DatanodeInfo dn, Block block) {
+  public void addToExcessReplicate(DatanodeInfo dn, Block block) {
     assert namesystem.hasWriteLock();
     assert namesystem.hasWriteLock();
     Collection<Block> excessBlocks = excessReplicateMap.get(dn.getStorageID());
     Collection<Block> excessBlocks = excessReplicateMap.get(dn.getStorageID());
     if (excessBlocks == null) {
     if (excessBlocks == null) {
@@ -1618,7 +1651,7 @@ public class BlockManager {
    * Modify (block-->datanode) map. Possibly generate replication tasks, if the
    * Modify (block-->datanode) map. Possibly generate replication tasks, if the
    * removed block is still valid.
    * removed block is still valid.
    */
    */
-  void removeStoredBlock(Block block, DatanodeDescriptor node) {
+  public void removeStoredBlock(Block block, DatanodeDescriptor node) {
     if(NameNode.stateChangeLog.isDebugEnabled()) {
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
       NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
           + block + " from " + node.getName());
           + block + " from " + node.getName());
@@ -1673,7 +1706,7 @@ public class BlockManager {
   /**
   /**
    * The given node is reporting that it received a certain block.
    * The given node is reporting that it received a certain block.
    */
    */
-  void addBlock(DatanodeDescriptor node, Block block, String delHint)
+  public void addBlock(DatanodeDescriptor node, Block block, String delHint)
       throws IOException {
       throws IOException {
     // decrement number of blocks scheduled to this datanode.
     // decrement number of blocks scheduled to this datanode.
     node.decBlocksScheduled();
     node.decBlocksScheduled();
@@ -1726,7 +1759,7 @@ public class BlockManager {
   /**
   /**
    * Return the number of nodes that are live and decommissioned.
    * Return the number of nodes that are live and decommissioned.
    */
    */
-  NumberReplicas countNodes(Block b) {
+  public NumberReplicas countNodes(Block b) {
     int count = 0;
     int count = 0;
     int live = 0;
     int live = 0;
     int corrupt = 0;
     int corrupt = 0;
@@ -1805,7 +1838,7 @@ public class BlockManager {
    * Return true if there are any blocks on this node that have not
    * Return true if there are any blocks on this node that have not
    * yet reached their replication factor. Otherwise returns false.
    * yet reached their replication factor. Otherwise returns false.
    */
    */
-  boolean isReplicationInProgress(DatanodeDescriptor srcNode) {
+  public boolean isReplicationInProgress(DatanodeDescriptor srcNode) {
     boolean status = false;
     boolean status = false;
     int underReplicatedBlocks = 0;
     int underReplicatedBlocks = 0;
     int decommissionOnlyReplicas = 0;
     int decommissionOnlyReplicas = 0;
@@ -1855,11 +1888,11 @@ public class BlockManager {
     return status;
     return status;
   }
   }
 
 
-  int getActiveBlockCount() {
+  public int getActiveBlockCount() {
     return blocksMap.size() - (int)pendingDeletionBlocksCount;
     return blocksMap.size() - (int)pendingDeletionBlocksCount;
   }
   }
 
 
-  DatanodeDescriptor[] getNodes(BlockInfo block) {
+  public DatanodeDescriptor[] getNodes(BlockInfo block) {
     DatanodeDescriptor[] nodes =
     DatanodeDescriptor[] nodes =
       new DatanodeDescriptor[block.numNodes()];
       new DatanodeDescriptor[block.numNodes()];
     Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
     Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
@@ -1869,22 +1902,22 @@ public class BlockManager {
     return nodes;
     return nodes;
   }
   }
 
 
-  int getTotalBlocks() {
+  public int getTotalBlocks() {
     return blocksMap.size();
     return blocksMap.size();
   }
   }
 
 
-  void removeBlock(Block block) {
+  public void removeBlock(Block block) {
     addToInvalidates(block);
     addToInvalidates(block);
     corruptReplicas.removeFromCorruptReplicasMap(block);
     corruptReplicas.removeFromCorruptReplicasMap(block);
     blocksMap.removeBlock(block);
     blocksMap.removeBlock(block);
   }
   }
 
 
-  BlockInfo getStoredBlock(Block block) {
+  public BlockInfo getStoredBlock(Block block) {
     return blocksMap.getStoredBlock(block);
     return blocksMap.getStoredBlock(block);
   }
   }
 
 
   /* updates a block in under replication queue */
   /* updates a block in under replication queue */
-  void updateNeededReplications(Block block, int curReplicasDelta,
+  public void updateNeededReplications(Block block, int curReplicasDelta,
       int expectedReplicasDelta) {
       int expectedReplicasDelta) {
     namesystem.writeLock();
     namesystem.writeLock();
     try {
     try {
@@ -1905,13 +1938,13 @@ public class BlockManager {
     }
     }
   }
   }
 
 
-  void checkReplication(Block block, int numExpectedReplicas) {
+  public void checkReplication(Block block, int numExpectedReplicas) {
     // filter out containingNodes that are marked for decommission.
     // filter out containingNodes that are marked for decommission.
     NumberReplicas number = countNodes(block);
     NumberReplicas number = countNodes(block);
     if (isNeededReplication(block, numExpectedReplicas, number.liveReplicas())) { 
     if (isNeededReplication(block, numExpectedReplicas, number.liveReplicas())) { 
       neededReplications.add(block,
       neededReplications.add(block,
                              number.liveReplicas(),
                              number.liveReplicas(),
-                             number.decommissionedReplicas,
+                             number.decommissionedReplicas(),
                              numExpectedReplicas);
                              numExpectedReplicas);
     }
     }
   }
   }
@@ -1926,11 +1959,8 @@ public class BlockManager {
     return fileINode.getReplication();
     return fileINode.getReplication();
   }
   }
 
 
-  /**
-   * Remove a datanode from the invalidatesSet
-   * @param n datanode
-   */
-  void removeFromInvalidates(String storageID) {
+  /** Remove a datanode from the invalidatesSet */
+  public void removeFromInvalidates(String storageID) {
     Collection<Block> blocks = recentInvalidateSets.remove(storageID);
     Collection<Block> blocks = recentInvalidateSets.remove(storageID);
     if (blocks != null) {
     if (blocks != null) {
       pendingDeletionBlocksCount -= blocks.size();
       pendingDeletionBlocksCount -= blocks.size();
@@ -1998,7 +2028,7 @@ public class BlockManager {
   //Returns the number of racks over which a given block is replicated
   //Returns the number of racks over which a given block is replicated
   //decommissioning/decommissioned nodes are not counted. corrupt replicas 
   //decommissioning/decommissioned nodes are not counted. corrupt replicas 
   //are also ignored
   //are also ignored
-  int getNumberOfRacks(Block b) {
+  public int getNumberOfRacks(Block b) {
     HashSet<String> rackSet = new HashSet<String>(0);
     HashSet<String> rackSet = new HashSet<String>(0);
     Collection<DatanodeDescriptor> corruptNodes = 
     Collection<DatanodeDescriptor> corruptNodes = 
                                   corruptReplicas.getNodes(b);
                                   corruptReplicas.getNodes(b);
@@ -2056,32 +2086,32 @@ public class BlockManager {
     }
     }
   }
   }
   
   
-  long getMissingBlocksCount() {
+  public long getMissingBlocksCount() {
     // not locking
     // not locking
     return this.neededReplications.getCorruptBlockSize();
     return this.neededReplications.getCorruptBlockSize();
   }
   }
 
 
-  BlockInfo addINode(BlockInfo block, INodeFile iNode) {
+  public BlockInfo addINode(BlockInfo block, INodeFile iNode) {
     return blocksMap.addINode(block, iNode);
     return blocksMap.addINode(block, iNode);
   }
   }
 
 
-  INodeFile getINode(Block b) {
+  public INodeFile getINode(Block b) {
     return blocksMap.getINode(b);
     return blocksMap.getINode(b);
   }
   }
 
 
-  void removeFromCorruptReplicasMap(Block block) {
+  public void removeFromCorruptReplicasMap(Block block) {
     corruptReplicas.removeFromCorruptReplicasMap(block);
     corruptReplicas.removeFromCorruptReplicasMap(block);
   }
   }
 
 
-  int numCorruptReplicas(Block block) {
+  public int numCorruptReplicas(Block block) {
     return corruptReplicas.numCorruptReplicas(block);
     return corruptReplicas.numCorruptReplicas(block);
   }
   }
 
 
-  void removeBlockFromMap(Block block) {
+  public void removeBlockFromMap(Block block) {
     blocksMap.removeBlock(block);
     blocksMap.removeBlock(block);
   }
   }
 
 
-  int getCapacity() {
+  public int getCapacity() {
     namesystem.readLock();
     namesystem.readLock();
     try {
     try {
       return blocksMap.getCapacity();
       return blocksMap.getCapacity();
@@ -2104,7 +2134,7 @@ public class BlockManager {
    * @return Up to numExpectedBlocks blocks from startingBlockId if it exists
    * @return Up to numExpectedBlocks blocks from startingBlockId if it exists
    *
    *
    */
    */
-  long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
+  public long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
                                    Long startingBlockId) {
                                    Long startingBlockId) {
     return corruptReplicas.getCorruptReplicaBlockIds(numExpectedBlocks,
     return corruptReplicas.getCorruptReplicaBlockIds(numExpectedBlocks,
                                                      startingBlockId);
                                                      startingBlockId);
@@ -2113,7 +2143,7 @@ public class BlockManager {
   /**
   /**
    * Return an iterator over the set of blocks for which there are no replicas.
    * Return an iterator over the set of blocks for which there are no replicas.
    */
    */
-  UnderReplicatedBlocks.BlockIterator getCorruptReplicaBlockIterator() {
+  public BlockIterator getCorruptReplicaBlockIterator() {
     return neededReplications
     return neededReplications
         .iterator(UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
         .iterator(UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
   }
   }

+ 6 - 4
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockPlacementPolicy.java → hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
@@ -26,6 +26,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 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.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
+import org.apache.hadoop.hdfs.server.namenode.FSInodeInfo;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -94,7 +96,7 @@ public abstract class BlockPlacementPolicy {
    * @return array of DatanodeDescriptor instances chosen as target
    * @return array of DatanodeDescriptor instances chosen as target
    * and sorted as a pipeline.
    * and sorted as a pipeline.
    */
    */
-  abstract DatanodeDescriptor[] chooseTarget(String srcPath,
+  public abstract DatanodeDescriptor[] chooseTarget(String srcPath,
                                              int numOfReplicas,
                                              int numOfReplicas,
                                              DatanodeDescriptor writer,
                                              DatanodeDescriptor writer,
                                              List<DatanodeDescriptor> chosenNodes,
                                              List<DatanodeDescriptor> chosenNodes,
@@ -222,11 +224,11 @@ public abstract class BlockPlacementPolicy {
    * @param numOfReplicas number of replicas wanted.
    * @param numOfReplicas number of replicas wanted.
    * @param writer the writer's machine, null if not in the cluster.
    * @param writer the writer's machine, null if not in the cluster.
    * @param blocksize size of the data to be written.
    * @param blocksize size of the data to be written.
-   * @param excludedNodes: datanodes that should not be considered as targets.
+   * @param excludedNodes datanodes that should not be considered as targets.
    * @return array of DatanodeDescriptor instances chosen as targets
    * @return array of DatanodeDescriptor instances chosen as targets
    * and sorted as a pipeline.
    * and sorted as a pipeline.
    */
    */
-  DatanodeDescriptor[] chooseTarget(String srcPath,
+  public DatanodeDescriptor[] chooseTarget(String srcPath,
                                     int numOfReplicas,
                                     int numOfReplicas,
                                     DatanodeDescriptor writer,
                                     DatanodeDescriptor writer,
                                     HashMap<Node, Node> excludedNodes,
                                     HashMap<Node, Node> excludedNodes,

+ 5 - 2
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockPlacementPolicyDefault.java → hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
@@ -32,6 +32,9 @@ 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.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
+import org.apache.hadoop.hdfs.server.namenode.FSInodeInfo;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.NodeBase;
@@ -89,7 +92,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
-  DatanodeDescriptor[] chooseTarget(String srcPath,
+  public DatanodeDescriptor[] chooseTarget(String srcPath,
                                     int numOfReplicas,
                                     int numOfReplicas,
                                     DatanodeDescriptor writer,
                                     DatanodeDescriptor writer,
                                     List<DatanodeDescriptor> chosenNodes,
                                     List<DatanodeDescriptor> chosenNodes,

+ 6 - 5
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java → hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java

@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import java.util.Iterator;
 import java.util.Iterator;
 
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.util.GSet;
 import org.apache.hadoop.hdfs.util.GSet;
 import org.apache.hadoop.hdfs.util.LightWeightGSet;
 import org.apache.hadoop.hdfs.util.LightWeightGSet;
 
 
@@ -28,7 +29,7 @@ import org.apache.hadoop.hdfs.util.LightWeightGSet;
  * block's metadata currently includes INode it belongs to and
  * block's metadata currently includes INode it belongs to and
  * the datanodes that store the block.
  * the datanodes that store the block.
  */
  */
-class BlocksMap {
+public class BlocksMap {
   private static class NodeIterator implements Iterator<DatanodeDescriptor> {
   private static class NodeIterator implements Iterator<DatanodeDescriptor> {
     private BlockInfo blockInfo;
     private BlockInfo blockInfo;
     private int nextIdx = 0;
     private int nextIdx = 0;
@@ -100,7 +101,7 @@ class BlocksMap {
   /**
   /**
    * Add block b belonging to the specified file inode to the map.
    * Add block b belonging to the specified file inode to the map.
    */
    */
-  BlockInfo addINode(BlockInfo b, INodeFile iNode) {
+  public BlockInfo addINode(BlockInfo b, INodeFile iNode) {
     BlockInfo info = blocks.get(b);
     BlockInfo info = blocks.get(b);
     if (info != b) {
     if (info != b) {
       info = b;
       info = b;
@@ -136,7 +137,7 @@ class BlocksMap {
    * Searches for the block in the BlocksMap and 
    * Searches for the block in the BlocksMap and 
    * returns Iterator that iterates through the nodes the block belongs to.
    * returns Iterator that iterates through the nodes the block belongs to.
    */
    */
-  Iterator<DatanodeDescriptor> nodeIterator(Block b) {
+  public Iterator<DatanodeDescriptor> nodeIterator(Block b) {
     return nodeIterator(blocks.get(b));
     return nodeIterator(blocks.get(b));
   }
   }
 
 
@@ -185,7 +186,7 @@ class BlocksMap {
   /**
   /**
    * Check if the block exists in map
    * Check if the block exists in map
    */
    */
-  boolean contains(Block block) {
+  public boolean contains(Block block) {
     return blocks.contains(block);
     return blocks.contains(block);
   }
   }
   
   

+ 2 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/CorruptReplicasMap.java → hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java

@@ -15,10 +15,11 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 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.namenode.NameNode;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
 
 
 import java.util.*;
 import java.util.*;

+ 38 - 34
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java → hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import java.io.DataInput;
 import java.io.DataInput;
 import java.io.IOException;
 import java.io.IOException;
@@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 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;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
@@ -44,7 +45,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   
   
   // 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.
-  DecommissioningStatus decommissioningStatus = new DecommissioningStatus();
+  public DecommissioningStatus decommissioningStatus = new DecommissioningStatus();
   
   
   /** Block and targets pair */
   /** Block and targets pair */
   @InterfaceAudience.Private
   @InterfaceAudience.Private
@@ -96,8 +97,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private int numBlocks = 0;
   private int numBlocks = 0;
   // isAlive == heartbeats.contains(this)
   // isAlive == heartbeats.contains(this)
   // This is an optimization, because contains takes O(n) time on Arraylist
   // This is an optimization, because contains takes O(n) time on Arraylist
-  protected boolean isAlive = false;
-  protected boolean needKeyUpdate = false;
+  public boolean isAlive = false;
+  public boolean needKeyUpdate = false;
 
 
   /** A queue of blocks to be replicated by this datanode */
   /** A queue of blocks to be replicated by this datanode */
   private BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
   private BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
@@ -204,7 +205,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * Add datanode to the block.
    * Add datanode to the block.
    * Add block to the head of the list of blocks belonging to the data-node.
    * Add block to the head of the list of blocks belonging to the data-node.
    */
    */
-  boolean addBlock(BlockInfo b) {
+  public boolean addBlock(BlockInfo b) {
     if(!b.addNode(this))
     if(!b.addNode(this))
       return false;
       return false;
     // add to the head of the data-node list
     // add to the head of the data-node list
@@ -217,7 +218,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * Remove block from the list of blocks belonging to the data-node.
    * Remove block from the list of blocks belonging to the data-node.
    * Remove datanode from the block.
    * Remove datanode from the block.
    */
    */
-  boolean removeBlock(BlockInfo b) {
+  public boolean removeBlock(BlockInfo b) {
     blockList = b.listRemove(blockList, this);
     blockList = b.listRemove(blockList, this);
     if ( b.removeNode(this) ) {
     if ( b.removeNode(this) ) {
       numBlocks--;
       numBlocks--;
@@ -242,7 +243,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * @param newBlock - a replacement block
    * @param newBlock - a replacement block
    * @return the new block
    * @return the new block
    */
    */
-  BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
+  public BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
     boolean done = removeBlock(oldBlock);
     boolean done = removeBlock(oldBlock);
     assert done : "Old block should belong to the data-node when replacing";
     assert done : "Old block should belong to the data-node when replacing";
     done = addBlock(newBlock);
     done = addBlock(newBlock);
@@ -250,7 +251,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return newBlock;
     return newBlock;
   }
   }
 
 
-  void resetBlocks() {
+  public void resetBlocks() {
     this.capacity = 0;
     this.capacity = 0;
     this.remaining = 0;
     this.remaining = 0;
     this.blockPoolUsed = 0;
     this.blockPoolUsed = 0;
@@ -268,7 +269,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
   /**
    * Updates stats from datanode heartbeat.
    * Updates stats from datanode heartbeat.
    */
    */
-  void updateHeartbeat(long capacity, long dfsUsed, long remaining,
+  public void updateHeartbeat(long capacity, long dfsUsed, long remaining,
       long blockPoolUsed, int xceiverCount, int volFailures) {
       long blockPoolUsed, int xceiverCount, int volFailures) {
     this.capacity = capacity;
     this.capacity = capacity;
     this.dfsUsed = dfsUsed;
     this.dfsUsed = dfsUsed;
@@ -283,7 +284,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
   /**
    * Iterates over the list of blocks belonging to the datanode.
    * Iterates over the list of blocks belonging to the datanode.
    */
    */
-  static class BlockIterator implements Iterator<BlockInfo> {
+  public static class BlockIterator implements Iterator<BlockInfo> {
     private BlockInfo current;
     private BlockInfo current;
     private DatanodeDescriptor node;
     private DatanodeDescriptor node;
       
       
@@ -307,7 +308,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
     }
   }
   }
 
 
-  Iterator<BlockInfo> getBlockIterator() {
+  public Iterator<BlockInfo> getBlockIterator() {
     return new BlockIterator(this.blockList, this);
     return new BlockIterator(this.blockList, this);
   }
   }
   
   
@@ -361,11 +362,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
     }
   }
   }
   
   
-  List<BlockTargetPair> getReplicationCommand(int maxTransfers) {
+  public List<BlockTargetPair> getReplicationCommand(int maxTransfers) {
     return replicateBlocks.poll(maxTransfers);
     return replicateBlocks.poll(maxTransfers);
   }
   }
 
 
-  BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
+  public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
     List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
     List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
     if(blocks == null)
       return null;
       return null;
@@ -375,7 +376,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
   /**
    * Remove the specified number of blocks to be invalidated
    * Remove the specified number of blocks to be invalidated
    */
    */
-  Block[] getInvalidateBlocks(int maxblocks) {
+  public Block[] getInvalidateBlocks(int maxblocks) {
     return getBlockArray(invalidateBlocks, maxblocks); 
     return getBlockArray(invalidateBlocks, maxblocks); 
   }
   }
 
 
@@ -418,7 +419,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
   }
 
 
   /** Serialization for FSEditLog */
   /** Serialization for FSEditLog */
-  void readFieldsFromFSEditLog(DataInput in) throws IOException {
+  public void readFieldsFromFSEditLog(DataInput in) throws IOException {
     this.name = DeprecatedUTF8.readString(in);
     this.name = DeprecatedUTF8.readString(in);
     this.storageID = DeprecatedUTF8.readString(in);
     this.storageID = DeprecatedUTF8.readString(in);
     this.infoPort = in.readShort() & 0x0000ffff;
     this.infoPort = in.readShort() & 0x0000ffff;
@@ -445,7 +446,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
   /**
    * Increments counter for number of blocks scheduled. 
    * Increments counter for number of blocks scheduled. 
    */
    */
-  void incBlocksScheduled() {
+  public void incBlocksScheduled() {
     currApproxBlocksScheduled++;
     currApproxBlocksScheduled++;
   }
   }
   
   
@@ -485,12 +486,13 @@ public class DatanodeDescriptor extends DatanodeInfo {
     // by DatanodeID
     // by DatanodeID
     return (this == obj) || super.equals(obj);
     return (this == obj) || super.equals(obj);
   }
   }
-  
-  class DecommissioningStatus {
-    int underReplicatedBlocks;
-    int decommissionOnlyReplicas;
-    int underReplicatedInOpenFiles;
-    long startTime;
+
+  /** Decommissioning status */
+  public class DecommissioningStatus {
+    private int underReplicatedBlocks;
+    private int decommissionOnlyReplicas;
+    private int underReplicatedInOpenFiles;
+    private long startTime;
     
     
     synchronized void set(int underRep,
     synchronized void set(int underRep,
         int onlyRep, int underConstruction) {
         int onlyRep, int underConstruction) {
@@ -501,32 +503,34 @@ public class DatanodeDescriptor extends DatanodeInfo {
       decommissionOnlyReplicas = onlyRep;
       decommissionOnlyReplicas = onlyRep;
       underReplicatedInOpenFiles = underConstruction;
       underReplicatedInOpenFiles = underConstruction;
     }
     }
-    
-    synchronized int getUnderReplicatedBlocks() {
+
+    /** @return the number of under-replicated blocks */
+    public synchronized int getUnderReplicatedBlocks() {
       if (isDecommissionInProgress() == false) {
       if (isDecommissionInProgress() == false) {
         return 0;
         return 0;
       }
       }
       return underReplicatedBlocks;
       return underReplicatedBlocks;
     }
     }
-    synchronized int getDecommissionOnlyReplicas() {
+    /** @return the number of decommission-only replicas */
+    public synchronized int getDecommissionOnlyReplicas() {
       if (isDecommissionInProgress() == false) {
       if (isDecommissionInProgress() == false) {
         return 0;
         return 0;
       }
       }
       return decommissionOnlyReplicas;
       return decommissionOnlyReplicas;
     }
     }
-
-    synchronized int getUnderReplicatedInOpenFiles() {
+    /** @return the number of under-replicated blocks in open files */
+    public synchronized int getUnderReplicatedInOpenFiles() {
       if (isDecommissionInProgress() == false) {
       if (isDecommissionInProgress() == false) {
         return 0;
         return 0;
       }
       }
       return underReplicatedInOpenFiles;
       return underReplicatedInOpenFiles;
     }
     }
-
-    synchronized void setStartTime(long time) {
+    /** Set start time */
+    public synchronized void setStartTime(long time) {
       startTime = time;
       startTime = time;
     }
     }
-    
-    synchronized long getStartTime() {
+    /** @return start time */
+    public synchronized long getStartTime() {
       if (isDecommissionInProgress() == false) {
       if (isDecommissionInProgress() == false) {
         return 0;
         return 0;
       }
       }
@@ -538,11 +542,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * Set the flag to indicate if this datanode is disallowed from communicating
    * Set the flag to indicate if this datanode is disallowed from communicating
    * with the namenode.
    * with the namenode.
    */
    */
-  void setDisallowed(boolean flag) {
+  public void setDisallowed(boolean flag) {
     disallowed = flag;
     disallowed = flag;
   }
   }
-  
-  boolean isDisallowed() {
+  /** Is the datanode disallowed from communicating with the namenode? */
+  public boolean isDisallowed() {
     return disallowed;
     return disallowed;
   }
   }
 
 

+ 57 - 0
hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.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.blockmanagement;
+
+/**
+ * A immutable object that stores the number of live replicas and
+ * the number of decommissined Replicas.
+ */
+public class NumberReplicas {
+  private int liveReplicas;
+  private int decommissionedReplicas;
+  private int corruptReplicas;
+  private int excessReplicas;
+
+  NumberReplicas() {
+    initialize(0, 0, 0, 0);
+  }
+
+  NumberReplicas(int live, int decommissioned, int corrupt, int excess) {
+    initialize(live, decommissioned, corrupt, excess);
+  }
+
+  void initialize(int live, int decommissioned, int corrupt, int excess) {
+    liveReplicas = live;
+    decommissionedReplicas = decommissioned;
+    corruptReplicas = corrupt;
+    excessReplicas = excess;
+  }
+
+  public int liveReplicas() {
+    return liveReplicas;
+  }
+  public int decommissionedReplicas() {
+    return decommissionedReplicas;
+  }
+  public int corruptReplicas() {
+    return corruptReplicas;
+  }
+  public int excessReplicas() {
+    return excessReplicas;
+  }
+} 

+ 3 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/PendingReplicationBlocks.java → hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java

@@ -15,9 +15,11 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 import org.apache.hadoop.util.*;
 import org.apache.hadoop.util.*;
 import java.io.*;
 import java.io.*;

+ 6 - 5
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/UnderReplicatedBlocks.java → hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java

@@ -15,17 +15,18 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import java.util.*;
 import java.util.*;
 
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 
 
 /* Class for keeping track of under replication blocks
 /* Class for keeping track of under replication blocks
  * Blocks have replication priority, with priority 0 indicating the highest
  * Blocks have replication priority, with priority 0 indicating the highest
  * Blocks have only one replicas has the highest
  * Blocks have only one replicas has the highest
  */
  */
-class UnderReplicatedBlocks implements Iterable<Block> {
+public class UnderReplicatedBlocks implements Iterable<Block> {
   static final int LEVEL = 5;
   static final int LEVEL = 5;
   static public final int QUEUE_WITH_CORRUPT_BLOCKS = 4;
   static public final int QUEUE_WITH_CORRUPT_BLOCKS = 4;
   private List<TreeSet<Block>> priorityQueues = new ArrayList<TreeSet<Block>>();
   private List<TreeSet<Block>> priorityQueues = new ArrayList<TreeSet<Block>>();
@@ -47,7 +48,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
   }
 
 
   /* Return the total number of under replication blocks */
   /* Return the total number of under replication blocks */
-  synchronized int size() {
+  public synchronized int size() {
     int size = 0;
     int size = 0;
     for (int i=0; i<LEVEL; i++) {
     for (int i=0; i<LEVEL; i++) {
       size += priorityQueues.get(i).size();
       size += priorityQueues.get(i).size();
@@ -70,7 +71,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
   }
   
   
   /* Check if a block is in the neededReplication queue */
   /* Check if a block is in the neededReplication queue */
-  synchronized boolean contains(Block block) {
+  public synchronized boolean contains(Block block) {
     for(TreeSet<Block> set:priorityQueues) {
     for(TreeSet<Block> set:priorityQueues) {
       if(set.contains(block)) { return true; }
       if(set.contains(block)) { return true; }
     }
     }
@@ -218,7 +219,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
     return new BlockIterator();
     return new BlockIterator();
   }
   }
   
   
-  class BlockIterator implements Iterator<Block> {
+  public class BlockIterator implements Iterator<Block> {
     private int level;
     private int level;
     private boolean isIteratorForLevel = false;
     private boolean isIteratorForLevel = false;
     private List<Iterator<Block>> iterators = new ArrayList<Iterator<Block>>();
     private List<Iterator<Block>> iterators = new ArrayList<Iterator<Block>>();

+ 2 - 3
hdfs/src/java/org/apache/hadoop/hdfs/server/common/JspHelper.java

@@ -26,8 +26,8 @@ import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.Socket;
 import java.net.URL;
 import java.net.URL;
 import java.net.URLEncoder;
 import java.net.URLEncoder;
-import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashMap;
@@ -45,14 +45,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
-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.ExtendedBlock;
 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.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.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.http.HtmlQuoting;
 import org.apache.hadoop.http.HtmlQuoting;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;

+ 6 - 0
hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -231,6 +231,9 @@ class BlockReceiver implements Closeable, FSConstants {
     } catch(IOException e) {
     } catch(IOException e) {
       ioe = e;
       ioe = e;
     }
     }
+    finally {
+      IOUtils.closeStream(checksumOut);
+    }
     // close block file
     // close block file
     try {
     try {
       if (out != null) {
       if (out != null) {
@@ -244,6 +247,9 @@ class BlockReceiver implements Closeable, FSConstants {
     } catch (IOException e) {
     } catch (IOException e) {
       ioe = e;
       ioe = e;
     }
     }
+    finally{
+      IOUtils.closeStream(out);
+    }
     // disk check
     // disk check
     if(ioe != null) {
     if(ioe != null) {
       datanode.checkDiskError(ioe);
       datanode.checkDiskError(ioe);

+ 11 - 10
hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -1977,8 +1977,8 @@ public class DataNode extends Configured
               EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE));
               EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE));
         }
         }
 
 
-        Sender.opWriteBlock(out,
-            b, 0, stage, 0, 0, 0, clientname, srcNode, targets, accessToken);
+        new Sender(out).writeBlock(b, accessToken, clientname, targets, srcNode,
+            stage, 0, 0, 0, 0);
 
 
         // send data & checksum
         // send data & checksum
         blockSender.sendBlock(out, baseStream, null);
         blockSender.sendBlock(out, baseStream, null);
@@ -2186,20 +2186,21 @@ public class DataNode extends Configured
         continue;
         continue;
       }
       }
       // drop any (illegal) authority in the URI for backwards compatibility
       // drop any (illegal) authority in the URI for backwards compatibility
-      File data = new File(dirURI.getPath());
+      File dir = new File(dirURI.getPath());
       try {
       try {
-        DiskChecker.checkDir(localFS, new Path(data.toURI()), permission);
-        dirs.add(data);
-      } catch (IOException e) {
-        LOG.warn("Invalid directory in: "
-                 + DFS_DATANODE_DATA_DIR_KEY + ": ", e);
-        invalidDirs.append("\"").append(data.getCanonicalPath()).append("\" ");
+        DiskChecker.checkDir(localFS, new Path(dir.toURI()), permission);
+        dirs.add(dir);
+      } catch (IOException ioe) {
+        LOG.warn("Invalid " + DFS_DATANODE_DATA_DIR_KEY + " "
+            + dir + " : ", ioe);
+        invalidDirs.append("\"").append(dir.getCanonicalPath()).append("\" ");
       }
       }
     }
     }
-    if (dirs.size() == 0)
+    if (dirs.size() == 0) {
       throw new IOException("All directories in "
       throw new IOException("All directories in "
           + DFS_DATANODE_DATA_DIR_KEY + " are invalid: "
           + DFS_DATANODE_DATA_DIR_KEY + " are invalid: "
           + invalidDirs);
           + invalidDirs);
+    }
     return dirs;
     return dirs;
   }
   }
 
 

+ 46 - 56
hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -85,7 +85,10 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
   private long opStartTime; //the start time of receiving an Op
   private long opStartTime; //the start time of receiving an Op
   
   
   public DataXceiver(Socket s, DataNode datanode, 
   public DataXceiver(Socket s, DataNode datanode, 
-      DataXceiverServer dataXceiverServer) {
+      DataXceiverServer dataXceiverServer) throws IOException {
+    super(new DataInputStream(new BufferedInputStream(
+        NetUtils.getInputStream(s), FSConstants.SMALL_BUFFER_SIZE)));
+
     this.s = s;
     this.s = s;
     this.isLocal = s.getInetAddress().equals(s.getLocalAddress());
     this.isLocal = s.getInetAddress().equals(s.getLocalAddress());
     this.datanode = datanode;
     this.datanode = datanode;
@@ -122,18 +125,14 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
   DataNode getDataNode() {return datanode;}
   DataNode getDataNode() {return datanode;}
 
 
   /**
   /**
-   * Read/write data from/to the DataXceiveServer.
+   * Read/write data from/to the DataXceiverServer.
    */
    */
   public void run() {
   public void run() {
     updateCurrentThreadName("Waiting for operation");
     updateCurrentThreadName("Waiting for operation");
 
 
-    DataInputStream in=null; 
     int opsProcessed = 0;
     int opsProcessed = 0;
     Op op = null;
     Op op = null;
     try {
     try {
-      in = new DataInputStream(
-          new BufferedInputStream(NetUtils.getInputStream(s), 
-                                  SMALL_BUFFER_SIZE));
       int stdTimeout = s.getSoTimeout();
       int stdTimeout = s.getSoTimeout();
 
 
       // We process requests in a loop, and stay around for a short timeout.
       // We process requests in a loop, and stay around for a short timeout.
@@ -145,7 +144,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
             assert socketKeepaliveTimeout > 0;
             assert socketKeepaliveTimeout > 0;
             s.setSoTimeout(socketKeepaliveTimeout);
             s.setSoTimeout(socketKeepaliveTimeout);
           }
           }
-          op = readOp(in);
+          op = readOp();
         } catch (InterruptedIOException ignored) {
         } catch (InterruptedIOException ignored) {
           // Time out while we wait for client rpc
           // Time out while we wait for client rpc
           break;
           break;
@@ -176,7 +175,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
         }
         }
 
 
         opStartTime = now();
         opStartTime = now();
-        processOp(op, in);
+        processOp(op);
         ++opsProcessed;
         ++opsProcessed;
       } while (!s.isClosed() && socketKeepaliveTimeout > 0);
       } while (!s.isClosed() && socketKeepaliveTimeout > 0);
     } catch (Throwable t) {
     } catch (Throwable t) {
@@ -196,13 +195,12 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
     }
     }
   }
   }
 
 
-  /**
-   * Read a block from the disk.
-   */
   @Override
   @Override
-  protected void opReadBlock(DataInputStream in, ExtendedBlock block,
-      long startOffset, long length, String clientName,
-      Token<BlockTokenIdentifier> blockToken) throws IOException {
+  public void readBlock(final ExtendedBlock block,
+      final Token<BlockTokenIdentifier> blockToken,
+      final String clientName,
+      final long blockOffset,
+      final long length) throws IOException {
     OutputStream baseStream = NetUtils.getOutputStream(s, 
     OutputStream baseStream = NetUtils.getOutputStream(s, 
         datanode.socketWriteTimeout);
         datanode.socketWriteTimeout);
     DataOutputStream out = new DataOutputStream(
     DataOutputStream out = new DataOutputStream(
@@ -225,7 +223,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
     updateCurrentThreadName("Sending block " + block);
     updateCurrentThreadName("Sending block " + block);
     try {
     try {
       try {
       try {
-        blockSender = new BlockSender(block, startOffset, length,
+        blockSender = new BlockSender(block, blockOffset, length,
             true, true, false, datanode, clientTraceFmt);
             true, true, false, datanode, clientTraceFmt);
       } catch(IOException e) {
       } catch(IOException e) {
         LOG.info("opReadBlock " + block + " received exception " + e);
         LOG.info("opReadBlock " + block + " received exception " + e);
@@ -284,16 +282,17 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
     datanode.metrics.incrReadsFromClient(isLocal);
     datanode.metrics.incrReadsFromClient(isLocal);
   }
   }
 
 
-  /**
-   * Write a block to disk.
-   */
   @Override
   @Override
-  protected void opWriteBlock(final DataInputStream in, final ExtendedBlock block, 
-      final int pipelineSize, final BlockConstructionStage stage,
-      final long newGs, final long minBytesRcvd, final long maxBytesRcvd,
-      final String clientname, final DatanodeInfo srcDataNode,
-      final DatanodeInfo[] targets, final Token<BlockTokenIdentifier> blockToken
-      ) throws IOException {
+  public void writeBlock(final ExtendedBlock block,
+      final Token<BlockTokenIdentifier> blockToken,
+      final String clientname,
+      final DatanodeInfo[] targets,
+      final DatanodeInfo srcDataNode,
+      final BlockConstructionStage stage,
+      final int pipelineSize,
+      final long minBytesRcvd,
+      final long maxBytesRcvd,
+      final long latestGenerationStamp) throws IOException {
     updateCurrentThreadName("Receiving block " + block + " client=" + clientname);
     updateCurrentThreadName("Receiving block " + block + " client=" + clientname);
     final boolean isDatanode = clientname.length() == 0;
     final boolean isDatanode = clientname.length() == 0;
     final boolean isClient = !isDatanode;
     final boolean isClient = !isDatanode;
@@ -308,7 +307,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
 
 
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("opWriteBlock: stage=" + stage + ", clientname=" + clientname 
       LOG.debug("opWriteBlock: stage=" + stage + ", clientname=" + clientname 
-      		+ "\n  block  =" + block + ", newGs=" + newGs
+      		+ "\n  block  =" + block + ", newGs=" + latestGenerationStamp
       		+ ", bytesRcvd=[" + minBytesRcvd + ", " + maxBytesRcvd + "]"
       		+ ", bytesRcvd=[" + minBytesRcvd + ", " + maxBytesRcvd + "]"
           + "\n  targets=" + Arrays.asList(targets)
           + "\n  targets=" + Arrays.asList(targets)
           + "; pipelineSize=" + pipelineSize + ", srcDataNode=" + srcDataNode
           + "; pipelineSize=" + pipelineSize + ", srcDataNode=" + srcDataNode
@@ -351,10 +350,10 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
         blockReceiver = new BlockReceiver(block, in, 
         blockReceiver = new BlockReceiver(block, in, 
             s.getRemoteSocketAddress().toString(),
             s.getRemoteSocketAddress().toString(),
             s.getLocalSocketAddress().toString(),
             s.getLocalSocketAddress().toString(),
-            stage, newGs, minBytesRcvd, maxBytesRcvd,
+            stage, latestGenerationStamp, minBytesRcvd, maxBytesRcvd,
             clientname, srcDataNode, datanode);
             clientname, srcDataNode, datanode);
       } else {
       } else {
-        datanode.data.recoverClose(block, newGs, minBytesRcvd);
+        datanode.data.recoverClose(block, latestGenerationStamp, minBytesRcvd);
       }
       }
 
 
       //
       //
@@ -380,9 +379,9 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
                          SMALL_BUFFER_SIZE));
                          SMALL_BUFFER_SIZE));
           mirrorIn = new DataInputStream(NetUtils.getInputStream(mirrorSock));
           mirrorIn = new DataInputStream(NetUtils.getInputStream(mirrorSock));
 
 
-          Sender.opWriteBlock(mirrorOut, originalBlock,
-              pipelineSize, stage, newGs, minBytesRcvd, maxBytesRcvd, clientname,
-              srcDataNode, targets, blockToken);
+          new Sender(mirrorOut).writeBlock(originalBlock, blockToken,
+              clientname, targets, srcDataNode, stage, pipelineSize,
+              minBytesRcvd, maxBytesRcvd, latestGenerationStamp);
 
 
           if (blockReceiver != null) { // send checksum header
           if (blockReceiver != null) { // send checksum header
             blockReceiver.writeChecksumHeader(mirrorOut);
             blockReceiver.writeChecksumHeader(mirrorOut);
@@ -464,7 +463,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
       // update its generation stamp
       // update its generation stamp
       if (isClient && 
       if (isClient && 
           stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
           stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
-        block.setGenerationStamp(newGs);
+        block.setGenerationStamp(latestGenerationStamp);
         block.setNumBytes(minBytesRcvd);
         block.setNumBytes(minBytesRcvd);
       }
       }
       
       
@@ -499,10 +498,10 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
   }
   }
 
 
   @Override
   @Override
-  protected void opTransferBlock(final DataInputStream in,
-      final ExtendedBlock blk, final String client,
-      final DatanodeInfo[] targets,
-      final Token<BlockTokenIdentifier> blockToken) throws IOException {
+  public void transferBlock(final ExtendedBlock blk,
+      final Token<BlockTokenIdentifier> blockToken,
+      final String clientName,
+      final DatanodeInfo[] targets) throws IOException {
     checkAccess(null, true, blk, blockToken,
     checkAccess(null, true, blk, blockToken,
         Op.TRANSFER_BLOCK, BlockTokenSecretManager.AccessMode.COPY);
         Op.TRANSFER_BLOCK, BlockTokenSecretManager.AccessMode.COPY);
 
 
@@ -511,19 +510,16 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
     final DataOutputStream out = new DataOutputStream(
     final DataOutputStream out = new DataOutputStream(
         NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
         NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
     try {
     try {
-      datanode.transferReplicaForPipelineRecovery(blk, targets, client);
+      datanode.transferReplicaForPipelineRecovery(blk, targets, clientName);
       writeResponse(Status.SUCCESS, out);
       writeResponse(Status.SUCCESS, out);
     } finally {
     } finally {
       IOUtils.closeStream(out);
       IOUtils.closeStream(out);
     }
     }
   }
   }
   
   
-  /**
-   * Get block checksum (MD5 of CRC32).
-   */
   @Override
   @Override
-  protected void opBlockChecksum(DataInputStream in, ExtendedBlock block,
-      Token<BlockTokenIdentifier> blockToken) throws IOException {
+  public void blockChecksum(final ExtendedBlock block,
+      final Token<BlockTokenIdentifier> blockToken) throws IOException {
     final DataOutputStream out = new DataOutputStream(
     final DataOutputStream out = new DataOutputStream(
         NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
         NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
     checkAccess(out, true, block, blockToken,
     checkAccess(out, true, block, blockToken,
@@ -572,12 +568,9 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
     datanode.metrics.addBlockChecksumOp(elapsed());
     datanode.metrics.addBlockChecksumOp(elapsed());
   }
   }
 
 
-  /**
-   * Read a block from the disk and then sends it to a destination.
-   */
   @Override
   @Override
-  protected void opCopyBlock(DataInputStream in, ExtendedBlock block,
-      Token<BlockTokenIdentifier> blockToken) throws IOException {
+  public void copyBlock(final ExtendedBlock block,
+      final Token<BlockTokenIdentifier> blockToken) throws IOException {
     updateCurrentThreadName("Copying block " + block);
     updateCurrentThreadName("Copying block " + block);
     // Read in the header
     // Read in the header
     if (datanode.isBlockTokenEnabled) {
     if (datanode.isBlockTokenEnabled) {
@@ -647,15 +640,12 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
     datanode.metrics.addCopyBlockOp(elapsed());
     datanode.metrics.addCopyBlockOp(elapsed());
   }
   }
 
 
-  /**
-   * Receive a block and write it to disk, it then notifies the namenode to
-   * remove the copy from the source.
-   */
   @Override
   @Override
-  protected void opReplaceBlock(DataInputStream in,
-      ExtendedBlock block, String sourceID, DatanodeInfo proxySource,
-      Token<BlockTokenIdentifier> blockToken) throws IOException {
-    updateCurrentThreadName("Replacing block " + block + " from " + sourceID);
+  public void replaceBlock(final ExtendedBlock block,
+      final Token<BlockTokenIdentifier> blockToken,
+      final String delHint,
+      final DatanodeInfo proxySource) throws IOException {
+    updateCurrentThreadName("Replacing block " + block + " from " + delHint);
 
 
     /* read header */
     /* read header */
     block.setNumBytes(dataXceiverServer.estimateBlockSize);
     block.setNumBytes(dataXceiverServer.estimateBlockSize);
@@ -699,7 +689,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
                      new BufferedOutputStream(baseStream, SMALL_BUFFER_SIZE));
                      new BufferedOutputStream(baseStream, SMALL_BUFFER_SIZE));
 
 
       /* send request to the proxy */
       /* send request to the proxy */
-      Sender.opCopyBlock(proxyOut, block, blockToken);
+      new Sender(proxyOut).copyBlock(block, blockToken);
 
 
       // receive the response from the proxy
       // receive the response from the proxy
       proxyReply = new DataInputStream(new BufferedInputStream(
       proxyReply = new DataInputStream(new BufferedInputStream(
@@ -727,7 +717,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
           dataXceiverServer.balanceThrottler, null);
           dataXceiverServer.balanceThrottler, null);
                     
                     
       // notify name node
       // notify name node
-      datanode.notifyNamenodeReceivedBlock(block, sourceID);
+      datanode.notifyNamenodeReceivedBlock(block, delHint);
 
 
       LOG.info("Moved block " + block + 
       LOG.info("Moved block " + block + 
           " from " + s.getRemoteSocketAddress());
           " from " + s.getRemoteSocketAddress());

+ 15 - 9
hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java

@@ -28,12 +28,13 @@ import java.util.Map;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.server.balancer.Balancer;
 import org.apache.hadoop.hdfs.server.balancer.Balancer;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 
 
 
 /**
 /**
@@ -128,22 +129,27 @@ class DataXceiverServer implements Runnable, FSConstants {
                    DFSConfigKeys.DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_DEFAULT));
                    DFSConfigKeys.DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_DEFAULT));
   }
   }
 
 
-  /**
-   */
+  @Override
   public void run() {
   public void run() {
     while (datanode.shouldRun) {
     while (datanode.shouldRun) {
       try {
       try {
         Socket s = ss.accept();
         Socket s = ss.accept();
         s.setTcpNoDelay(true);
         s.setTcpNoDelay(true);
-        new Daemon(datanode.threadGroup, 
-            new DataXceiver(s, datanode, this)).start();
+        final DataXceiver exciver;
+        try {
+          exciver = new DataXceiver(s, datanode, this);
+        } catch(IOException e) {
+          IOUtils.closeSocket(s);
+          throw e;
+        }
+        new Daemon(datanode.threadGroup, exciver).start();
       } catch (SocketTimeoutException ignored) {
       } catch (SocketTimeoutException ignored) {
         // wake up to see if should continue to run
         // wake up to see if should continue to run
       } catch (IOException ie) {
       } catch (IOException ie) {
-        LOG.warn(datanode.getMachineName() + ":DataXceiveServer: ", ie);
+        LOG.warn(datanode.getMachineName() + ":DataXceiverServer: ", ie);
       } catch (Throwable te) {
       } catch (Throwable te) {
         LOG.error(datanode.getMachineName()
         LOG.error(datanode.getMachineName()
-            + ":DataXceiveServer: Exiting due to: ", te);
+            + ":DataXceiverServer: Exiting due to: ", te);
         datanode.shouldRun = false;
         datanode.shouldRun = false;
       }
       }
     }
     }
@@ -151,7 +157,7 @@ class DataXceiverServer implements Runnable, FSConstants {
       ss.close();
       ss.close();
     } catch (IOException ie) {
     } catch (IOException ie) {
       LOG.warn(datanode.getMachineName()
       LOG.warn(datanode.getMachineName()
-          + ":DataXceiveServer: Close exception due to: ", ie);
+          + ":DataXceiverServer: Close exception due to: ", ie);
     }
     }
   }
   }
   
   
@@ -161,7 +167,7 @@ class DataXceiverServer implements Runnable, FSConstants {
     try {
     try {
       this.ss.close();
       this.ss.close();
     } catch (IOException ie) {
     } catch (IOException ie) {
-      LOG.warn(datanode.getMachineName() + ":DataXceiveServer.kill(): " 
+      LOG.warn(datanode.getMachineName() + ":DataXceiverServer.kill(): "
                               + StringUtils.stringifyException(ie));
                               + StringUtils.stringifyException(ie));
     }
     }
 
 

+ 1 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java

@@ -1150,7 +1150,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       conf.getInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
       conf.getInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
                   DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
                   DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
 
 
-    String[] dataDirs = conf.getStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
+    String[] dataDirs = conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
 
 
     int volsConfigured = (dataDirs == null) ? 0 : dataDirs.length;
     int volsConfigured = (dataDirs == null) ? 0 : dataDirs.length;
 
 

+ 106 - 102
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java

@@ -17,8 +17,12 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
+import java.net.URL;
+import java.net.URLConnection;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Iterator;
@@ -27,13 +31,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.Set;
 
 
-import javax.management.JMX;
-import javax.management.MBeanServerConnection;
 import javax.management.MalformedObjectNameException;
 import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.management.remote.JMXConnector;
-import javax.management.remote.JMXConnectorFactory;
-import javax.management.remote.JMXServiceURL;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -43,24 +41,27 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.type.TypeReference;
 import org.codehaus.jackson.type.TypeReference;
 import org.znerd.xmlenc.XMLOutputter;
 import org.znerd.xmlenc.XMLOutputter;
 
 
 /**
 /**
  * This class generates the data that is needed to be displayed on cluster web 
  * This class generates the data that is needed to be displayed on cluster web 
- * console by connecting to each namenode through JMX.
+ * console.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 class ClusterJspHelper {
 class ClusterJspHelper {
   private static final Log LOG = LogFactory.getLog(ClusterJspHelper.class);
   private static final Log LOG = LogFactory.getLog(ClusterJspHelper.class);
   public static final String OVERALL_STATUS = "overall-status";
   public static final String OVERALL_STATUS = "overall-status";
   public static final String DEAD = "Dead";
   public static final String DEAD = "Dead";
+  private static final String JMX_QRY = 
+    "/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo";
   
   
   /**
   /**
    * JSP helper function that generates cluster health report.  When 
    * JSP helper function that generates cluster health report.  When 
    * encountering exception while getting Namenode status, the exception will 
    * encountering exception while getting Namenode status, the exception will 
-   * be listed in the page with corresponding stack trace.
+   * be listed on the page with corresponding stack trace.
    */
    */
   ClusterStatus generateClusterHealthReport() {
   ClusterStatus generateClusterHealthReport() {
     ClusterStatus cs = new ClusterStatus();
     ClusterStatus cs = new ClusterStatus();
@@ -79,26 +80,24 @@ class ClusterJspHelper {
       NamenodeMXBeanHelper nnHelper = null;
       NamenodeMXBeanHelper nnHelper = null;
       try {
       try {
         nnHelper = new NamenodeMXBeanHelper(isa, conf);
         nnHelper = new NamenodeMXBeanHelper(isa, conf);
-        NamenodeStatus nn = nnHelper.getNamenodeStatus();
+        String mbeanProps= queryMbean(nnHelper.httpAddress, conf);
+        NamenodeStatus nn = nnHelper.getNamenodeStatus(mbeanProps);
         if (cs.clusterid.isEmpty() || cs.clusterid.equals("")) { // Set clusterid only once
         if (cs.clusterid.isEmpty() || cs.clusterid.equals("")) { // Set clusterid only once
-          cs.clusterid = nnHelper.getClusterId();
+          cs.clusterid = nnHelper.getClusterId(mbeanProps);
         }
         }
         cs.addNamenodeStatus(nn);
         cs.addNamenodeStatus(nn);
       } catch ( Exception e ) {
       } catch ( Exception e ) {
         // track exceptions encountered when connecting to namenodes
         // track exceptions encountered when connecting to namenodes
         cs.addException(isa.getHostName(), e);
         cs.addException(isa.getHostName(), e);
         continue;
         continue;
-      } finally {
-        if (nnHelper != null) {
-          nnHelper.cleanup();
-        }
-      }
+      } 
     }
     }
     return cs;
     return cs;
   }
   }
 
 
   /**
   /**
-   * Helper function that generates the decommissioning report.
+   * Helper function that generates the decommissioning report.  Connect to each
+   * Namenode over http via JmxJsonServlet to collect the data nodes status.
    */
    */
   DecommissionStatus generateDecommissioningReport() {
   DecommissionStatus generateDecommissioningReport() {
     String clusterid = "";
     String clusterid = "";
@@ -127,21 +126,18 @@ class ClusterJspHelper {
       NamenodeMXBeanHelper nnHelper = null;
       NamenodeMXBeanHelper nnHelper = null;
       try {
       try {
         nnHelper = new NamenodeMXBeanHelper(isa, conf);
         nnHelper = new NamenodeMXBeanHelper(isa, conf);
+        String mbeanProps= queryMbean(nnHelper.httpAddress, conf);
         if (clusterid.equals("")) {
         if (clusterid.equals("")) {
-          clusterid = nnHelper.getClusterId();
+          clusterid = nnHelper.getClusterId(mbeanProps);
         }
         }
-        nnHelper.getDecomNodeInfoForReport(statusMap);
+        nnHelper.getDecomNodeInfoForReport(statusMap, mbeanProps);
       } catch (Exception e) {
       } catch (Exception e) {
         // catch exceptions encountered while connecting to namenodes
         // catch exceptions encountered while connecting to namenodes
         String nnHost = isa.getHostName();
         String nnHost = isa.getHostName();
         decommissionExceptions.put(nnHost, e);
         decommissionExceptions.put(nnHost, e);
         unreportedNamenode.add(nnHost);
         unreportedNamenode.add(nnHost);
         continue;
         continue;
-      } finally {
-        if (nnHelper != null) {
-          nnHelper.cleanup();
-        }
-      }
+      } 
     }
     }
     updateUnknownStatus(statusMap, unreportedNamenode);
     updateUnknownStatus(statusMap, unreportedNamenode);
     getDecommissionNodeClusterState(statusMap);
     getDecommissionNodeClusterState(statusMap);
@@ -260,40 +256,20 @@ class ClusterJspHelper {
     }
     }
     return Integer.parseInt(address.split(":")[1]);
     return Integer.parseInt(address.split(":")[1]);
   }
   }
-
+  
   /**
   /**
-   * Class for connecting to Namenode over JMX and get attributes
-   * exposed by the MXBean.
+   * Class for connecting to Namenode over http via JmxJsonServlet 
+   * to get JMX attributes exposed by the MXBean.  
    */
    */
   static class NamenodeMXBeanHelper {
   static class NamenodeMXBeanHelper {
     private static final ObjectMapper mapper = new ObjectMapper();
     private static final ObjectMapper mapper = new ObjectMapper();
-    private final InetSocketAddress rpcAddress;
     private final String host;
     private final String host;
-    private final Configuration conf;
-    private final JMXConnector connector;
-    private final NameNodeMXBean mxbeanProxy;
+    private final String httpAddress;
     
     
     NamenodeMXBeanHelper(InetSocketAddress addr, Configuration conf)
     NamenodeMXBeanHelper(InetSocketAddress addr, Configuration conf)
         throws IOException, MalformedObjectNameException {
         throws IOException, MalformedObjectNameException {
-      this.rpcAddress = addr;
       this.host = addr.getHostName();
       this.host = addr.getHostName();
-      this.conf = conf;
-      int port = conf.getInt("dfs.namenode.jmxport", -1);
-      
-      JMXServiceURL jmxURL = new JMXServiceURL("service:jmx:rmi:///jndi/rmi://"
-          + host + ":" + port + "/jmxrmi");
-      connector = JMXConnectorFactory.connect(jmxURL);
-      mxbeanProxy = getNamenodeMxBean();
-    }
-    
-    private NameNodeMXBean getNamenodeMxBean()
-        throws IOException, MalformedObjectNameException {
-      // Get an MBeanServerConnection on the remote VM.
-      MBeanServerConnection remote = connector.getMBeanServerConnection();
-      ObjectName mxbeanName = new ObjectName(
-          "Hadoop:service=NameNode,name=NameNodeInfo");
-
-      return JMX.newMXBeanProxy(remote, mxbeanName, NameNodeMXBean.class);
+      this.httpAddress = DFSUtil.getInfoServer(addr, conf, false);
     }
     }
     
     
     /** Get the map corresponding to the JSON string */
     /** Get the map corresponding to the JSON string */
@@ -305,10 +281,9 @@ class ClusterJspHelper {
     }
     }
     
     
     /**
     /**
-     * Process JSON string returned from JMX connection to get the number of
-     * live datanodes.
+     * Get the number of live datanodes.
      * 
      * 
-     * @param json JSON output from JMX call that contains live node status.
+     * @param json JSON string that contains live node status.
      * @param nn namenode status to return information in
      * @param nn namenode status to return information in
      */
      */
     private static void getLiveNodeCount(String json, NamenodeStatus nn)
     private static void getLiveNodeCount(String json, NamenodeStatus nn)
@@ -333,11 +308,10 @@ class ClusterJspHelper {
     }
     }
   
   
     /**
     /**
-     * Count the number of dead datanode based on the JSON string returned from
-     * JMX call.
+     * Count the number of dead datanode.
      * 
      * 
      * @param nn namenode
      * @param nn namenode
-     * @param json JSON string returned from JMX call
+     * @param json JSON string
      */
      */
     private static void getDeadNodeCount(String json, NamenodeStatus nn)
     private static void getDeadNodeCount(String json, NamenodeStatus nn)
         throws IOException {
         throws IOException {
@@ -358,51 +332,53 @@ class ClusterJspHelper {
       }
       }
     }
     }
   
   
-    public String getClusterId() {
-      return mxbeanProxy.getClusterId();
+    public String getClusterId(String props) throws IOException {
+      return getProperty(props, "ClusterId").getTextValue();
     }
     }
     
     
-    public NamenodeStatus getNamenodeStatus()
-        throws IOException, MalformedObjectNameException {
+    public NamenodeStatus getNamenodeStatus(String props) throws IOException,
+        MalformedObjectNameException, NumberFormatException {
       NamenodeStatus nn = new NamenodeStatus();
       NamenodeStatus nn = new NamenodeStatus();
       nn.host = host;
       nn.host = host;
-      nn.filesAndDirectories = mxbeanProxy.getTotalFiles();
-      nn.capacity = mxbeanProxy.getTotal();
-      nn.free = mxbeanProxy.getFree();
-      nn.bpUsed = mxbeanProxy.getBlockPoolUsedSpace();
-      nn.nonDfsUsed = mxbeanProxy.getNonDfsUsedSpace();
-      nn.blocksCount = mxbeanProxy.getTotalBlocks();
-      nn.missingBlocksCount = mxbeanProxy.getNumberOfMissingBlocks();
-      nn.free = mxbeanProxy.getFree();
-      nn.httpAddress = DFSUtil.getInfoServer(rpcAddress, conf, false);
-      getLiveNodeCount(mxbeanProxy.getLiveNodes(), nn);
-      getDeadNodeCount(mxbeanProxy.getDeadNodes(), nn);
+      nn.filesAndDirectories = getProperty(props, "TotalFiles").getLongValue();
+      nn.capacity = getProperty(props, "Total").getLongValue();
+      nn.free = getProperty(props, "Free").getLongValue();
+      nn.bpUsed = getProperty(props, "BlockPoolUsedSpace").getLongValue();
+      nn.nonDfsUsed = getProperty(props, "NonDfsUsedSpace").getLongValue();
+      nn.blocksCount = getProperty(props, "TotalBlocks").getLongValue();
+      nn.missingBlocksCount = getProperty(props, "NumberOfMissingBlocks")
+          .getLongValue();
+      nn.httpAddress = httpAddress;
+      getLiveNodeCount(getProperty(props, "LiveNodes").getValueAsText(), nn);
+      getDeadNodeCount(getProperty(props, "DeadNodes").getValueAsText(), nn);
       return nn;
       return nn;
     }
     }
     
     
     /**
     /**
-     * Connect to namenode to get decommission node information.
+     * Get the decommission node information.
      * @param statusMap data node status map
      * @param statusMap data node status map
-     * @param connector JMXConnector
+     * @param props string
      */
      */
     private void getDecomNodeInfoForReport(
     private void getDecomNodeInfoForReport(
-        Map<String, Map<String, String>> statusMap) throws IOException,
-        MalformedObjectNameException {
-      getLiveNodeStatus(statusMap, host, mxbeanProxy.getLiveNodes());
-      getDeadNodeStatus(statusMap, host, mxbeanProxy.getDeadNodes());
-      getDecommissionNodeStatus(statusMap, host, mxbeanProxy.getDecomNodes());
+        Map<String, Map<String, String>> statusMap, String props)
+        throws IOException, MalformedObjectNameException {
+      getLiveNodeStatus(statusMap, host, getProperty(props, "LiveNodes")
+          .getValueAsText());
+      getDeadNodeStatus(statusMap, host, getProperty(props, "DeadNodes")
+          .getValueAsText());
+      getDecommissionNodeStatus(statusMap, host,
+          getProperty(props, "DecomNodes").getValueAsText());
     }
     }
   
   
     /**
     /**
-     * Process the JSON string returned from JMX call to get live datanode
-     * status. Store the information into datanode status map and
-     * Decommissionnode.
+     * Store the live datanode status information into datanode status map and
+     * DecommissionNode.
      * 
      * 
      * @param statusMap Map of datanode status. Key is datanode, value
      * @param statusMap Map of datanode status. Key is datanode, value
      *          is an inner map whose key is namenode, value is datanode status.
      *          is an inner map whose key is namenode, value is datanode status.
      *          reported by each namenode.
      *          reported by each namenode.
      * @param namenodeHost host name of the namenode
      * @param namenodeHost host name of the namenode
-     * @param decomnode update Decommissionnode with alive node status
+     * @param decomnode update DecommissionNode with alive node status
      * @param json JSON string contains datanode status
      * @param json JSON string contains datanode status
      * @throws IOException
      * @throws IOException
      */
      */
@@ -434,15 +410,14 @@ class ClusterJspHelper {
     }
     }
   
   
     /**
     /**
-     * Process the JSON string returned from JMX connection to get the dead
-     * datanode information. Store the information into datanode status map and
-     * Decommissionnode.
+     * Store the dead datanode information into datanode status map and
+     * DecommissionNode.
      * 
      * 
      * @param statusMap map with key being datanode, value being an
      * @param statusMap map with key being datanode, value being an
      *          inner map (key:namenode, value:decommisionning state).
      *          inner map (key:namenode, value:decommisionning state).
      * @param host datanode hostname
      * @param host datanode hostname
-     * @param decomnode
-     * @param json
+     * @param decomnode DecommissionNode
+     * @param json String
      * @throws IOException
      * @throws IOException
      */
      */
     private static void getDeadNodeStatus(
     private static void getDeadNodeStatus(
@@ -478,14 +453,13 @@ class ClusterJspHelper {
     }
     }
   
   
     /**
     /**
-     * We process the JSON string returned from JMX connection to get the
-     * decommisioning datanode information.
+     * Get the decommisioning datanode information.
      * 
      * 
      * @param dataNodeStatusMap map with key being datanode, value being an
      * @param dataNodeStatusMap map with key being datanode, value being an
      *          inner map (key:namenode, value:decommisionning state).
      *          inner map (key:namenode, value:decommisionning state).
      * @param host datanode
      * @param host datanode
-     * @param decomnode Decommissionnode
-     * @param json JSON string returned from JMX connection
+     * @param decomnode DecommissionNode
+     * @param json String
      */
      */
     private static void getDecommissionNodeStatus(
     private static void getDecommissionNodeStatus(
         Map<String, Map<String, String>> dataNodeStatusMap, String host,
         Map<String, Map<String, String>> dataNodeStatusMap, String host,
@@ -508,19 +482,6 @@ class ClusterJspHelper {
         dataNodeStatusMap.put(dn, nnStatus);
         dataNodeStatusMap.put(dn, nnStatus);
       }
       }
     }
     }
-  
-    
-    public void cleanup() {
-      if (connector != null) {
-        try {
-          connector.close();
-        } catch (Exception e) {
-          // log failure of close jmx connection
-          LOG.warn("Unable to close JMX connection. "
-              + StringUtils.stringifyException(e));
-        }
-      }
-    }
   }
   }
 
 
   /**
   /**
@@ -893,4 +854,47 @@ class ClusterJspHelper {
     doc.endTag(); // message
     doc.endTag(); // message
     doc.endTag(); // cluster
     doc.endTag(); // cluster
   }
   }
+  
+  /**
+   * Read in the content from a URL
+   * @param url URL To read
+   * @return the text from the output
+   * @throws IOException if something went wrong
+   */
+  private static String readOutput(URL url) throws IOException {
+    StringBuilder out = new StringBuilder();
+    URLConnection connection = url.openConnection();
+    BufferedReader in = new BufferedReader(
+                            new InputStreamReader(
+                            connection.getInputStream()));
+    String inputLine;
+    while ((inputLine = in.readLine()) != null) {
+      out.append(inputLine);
+    }
+    in.close();
+    return out.toString();
+  }
+
+  private static String queryMbean(String httpAddress, Configuration conf) 
+    throws IOException {
+    URL url = new URL("http://"+httpAddress+JMX_QRY);
+    return readOutput(url);
+  }
+  /**
+   * In order to query a namenode mxbean, a http connection in the form of
+   * "http://hostname/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo"
+   * is sent to namenode.  JMX attributes are exposed via JmxJsonServelet on 
+   * the namenode side.
+   */
+  private static JsonNode getProperty(String props, String propertyname)
+  throws IOException {
+    if (props == null || props.equals("") || propertyname == null 
+        || propertyname.equals("")) {
+      return null;
+    }
+    ObjectMapper m = new ObjectMapper();
+    JsonNode rootNode = m.readValue(props, JsonNode.class);
+    JsonNode jn = rootNode.get("beans").get(0).get(propertyname);
+    return jn;
+  }
 } 
 } 

+ 1 - 0
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/DecommissionManager.java

@@ -21,6 +21,7 @@ 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;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.util.CyclicIteration;
 import org.apache.hadoop.util.CyclicIteration;
 
 
 /**
 /**

+ 22 - 10
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java

@@ -154,19 +154,31 @@ class EditLogFileOutputStream extends EditLogOutputStream {
     
     
     // close should have been called after all pending transactions
     // close should have been called after all pending transactions
     // have been flushed & synced.
     // have been flushed & synced.
-    int bufSize = bufCurrent.size();
-    if (bufSize != 0) {
-      throw new IOException("FSEditStream has " + bufSize
-          + " bytes still to be flushed and cannot " + "be closed.");
+    // if already closed, just skip
+    if(bufCurrent != null)
+    {
+      int bufSize = bufCurrent.size();
+      if (bufSize != 0) {
+        throw new IOException("FSEditStream has " + bufSize
+            + " bytes still to be flushed and cannot " + "be closed.");
+      }
+      bufCurrent.close();
+      bufCurrent = null;
     }
     }
-    bufCurrent.close();
-    bufReady.close();
 
 
-    // remove the last INVALID marker from transaction log.
-    fc.truncate(fc.position());
-    fp.close();
+    if(bufReady != null) {
+      bufReady.close();
+      bufReady = null;
+    }
 
 
-    bufCurrent = bufReady = null;
+    // remove the last INVALID marker from transaction log.
+    if (fc != null && fc.isOpen()) {
+      fc.truncate(fc.position());
+      fc.close();
+    }
+    if (fp != null) {
+      fp.close();
+    }
     fp = null;
     fp = null;
   }
   }
   
   

+ 18 - 11
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -17,40 +17,47 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import static org.apache.hadoop.hdfs.server.common.Util.now;
+
 import java.io.Closeable;
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.concurrent.locks.Condition;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
+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.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
-import org.apache.hadoop.hdfs.protocol.FSLimitException.*;
+import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
+import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+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.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.util.ByteArray;
 import org.apache.hadoop.hdfs.util.ByteArray;
-import static org.apache.hadoop.hdfs.server.common.Util.now;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 
 /*************************************************
 /*************************************************
  * FSDirectory stores the filesystem directory state.
  * FSDirectory stores the filesystem directory state.
@@ -61,7 +68,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
  * and logged to disk.
  * and logged to disk.
  * 
  * 
  *************************************************/
  *************************************************/
-class FSDirectory implements Closeable {
+public class FSDirectory implements Closeable {
 
 
   INodeDirectoryWithQuota rootDir;
   INodeDirectoryWithQuota rootDir;
   FSImage fsImage;  
   FSImage fsImage;  
@@ -1337,7 +1344,7 @@ class FSDirectory implements Closeable {
    * @throws QuotaExceededException if the new count violates any quota limit
    * @throws QuotaExceededException if the new count violates any quota limit
    * @throws FileNotFound if path does not exist.
    * @throws FileNotFound if path does not exist.
    */
    */
-  void updateSpaceConsumed(String path, long nsDelta, long dsDelta)
+  public void updateSpaceConsumed(String path, long nsDelta, long dsDelta)
                                          throws QuotaExceededException,
                                          throws QuotaExceededException,
                                                 FileNotFoundException,
                                                 FileNotFoundException,
                                                 UnresolvedLinkException {
                                                 UnresolvedLinkException {

+ 24 - 2
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -17,6 +17,8 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import static org.apache.hadoop.hdfs.server.common.Util.now;
+
 import java.io.BufferedInputStream;
 import java.io.BufferedInputStream;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.File;
 import java.io.File;
@@ -33,12 +35,32 @@ import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.LogHeader;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.LogHeader;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.Reader;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.Reader;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ClearNSQuotaOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ConcatDeleteOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetNSQuotaOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
 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.FSEditLogOp.*;
 
 
 public class FSEditLogLoader {
 public class FSEditLogLoader {
   private final FSNamesystem fsNamesys;
   private final FSNamesystem fsNamesys;

+ 1 - 0
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 
 

+ 6 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -414,7 +414,12 @@ public class FSImage implements Closeable {
       LOG.info("Upgrade of " + sd.getRoot() + " is complete.");
       LOG.info("Upgrade of " + sd.getRoot() + " is complete.");
     }
     }
     isUpgradeFinalized = false;
     isUpgradeFinalized = false;
-    storage.reportErrorsOnDirectories(errorSDs);
+    if (!errorSDs.isEmpty()) {
+      storage.reportErrorsOnDirectories(errorSDs);
+      //during upgrade, it's a fatal error to fail any storage directory
+      throw new IOException("Upgrade failed in " + errorSDs.size()
+          + " storage directory(ies), previously logged.");
+    }
     storage.initializeDistributedUpgrade();
     storage.initializeDistributedUpgrade();
   }
   }
 
 

+ 1 - 0
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.MD5Hash;

+ 3 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -22,7 +22,6 @@ import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -33,6 +32,9 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 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;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;

+ 98 - 82
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -31,6 +31,7 @@ import java.io.PrintWriter;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ManagementFactory;
 import java.net.InetAddress;
 import java.net.InetAddress;
 import java.net.URI;
 import java.net.URI;
+import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
@@ -96,15 +97,20 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 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.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
+import org.apache.hadoop.hdfs.server.blockmanagement.UnderReplicatedBlocks;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.common.Util;
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor.BlockTargetPair;
 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.UnderReplicatedBlocks.BlockIterator;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
@@ -235,7 +241,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   // Stores the correct file name hierarchy
   // Stores the correct file name hierarchy
   //
   //
   public FSDirectory dir;
   public FSDirectory dir;
-  BlockManager blockManager;
+  public BlockManager blockManager;
   
   
   // Block pool ID used by this namenode
   // Block pool ID used by this namenode
   String blockPoolId;
   String blockPoolId;
@@ -270,10 +276,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * Stores a set of DatanodeDescriptor objects.
    * Stores a set of DatanodeDescriptor objects.
    * This is a subset of {@link #datanodeMap}, containing nodes that are 
    * This is a subset of {@link #datanodeMap}, containing nodes that are 
    * considered alive.
    * considered alive.
-   * The {@link HeartbeatMonitor} periodically checks for outdated entries,
+   * The HeartbeatMonitor periodically checks for out-dated entries,
    * and removes them from the list.
    * and removes them from the list.
    */
    */
-  ArrayList<DatanodeDescriptor> heartbeats = new ArrayList<DatanodeDescriptor>();
+  public ArrayList<DatanodeDescriptor> heartbeats = new ArrayList<DatanodeDescriptor>();
 
 
   public LeaseManager leaseManager = new LeaseManager(this); 
   public LeaseManager leaseManager = new LeaseManager(this); 
 
 
@@ -314,8 +320,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   private volatile SafeModeInfo safeMode;  // safe mode information
   private volatile SafeModeInfo safeMode;  // safe mode information
   private Host2NodesMap host2DataNodeMap = new Host2NodesMap();
   private Host2NodesMap host2DataNodeMap = new Host2NodesMap();
     
     
-  // datanode networktoplogy
-  NetworkTopology clusterMap = new NetworkTopology();
+  /** datanode network toplogy */
+  public NetworkTopology clusterMap = new NetworkTopology();
   private DNSToSwitchMapping dnsToSwitchMapping;
   private DNSToSwitchMapping dnsToSwitchMapping;
 
 
   private HostsFileReader hostsReader; 
   private HostsFileReader hostsReader; 
@@ -329,7 +335,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   private final GenerationStamp generationStamp = new GenerationStamp();
   private final GenerationStamp generationStamp = new GenerationStamp();
 
 
   // Ask Datanode only up to this many blocks to delete.
   // Ask Datanode only up to this many blocks to delete.
-  int blockInvalidateLimit = DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT;
+  public int blockInvalidateLimit = DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT;
 
 
   // precision of access times.
   // precision of access times.
   private long accessTimePrecision = 0;
   private long accessTimePrecision = 0;
@@ -472,23 +478,23 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   }
   }
 
 
   // utility methods to acquire and release read lock and write lock
   // utility methods to acquire and release read lock and write lock
-  void readLock() {
+  public void readLock() {
     this.fsLock.readLock().lock();
     this.fsLock.readLock().lock();
   }
   }
 
 
-  void readUnlock() {
+  public void readUnlock() {
     this.fsLock.readLock().unlock();
     this.fsLock.readLock().unlock();
   }
   }
 
 
-  void writeLock() {
+  public void writeLock() {
     this.fsLock.writeLock().lock();
     this.fsLock.writeLock().lock();
   }
   }
 
 
-  void writeUnlock() {
+  public void writeUnlock() {
     this.fsLock.writeLock().unlock();
     this.fsLock.writeLock().unlock();
   }
   }
 
 
-  boolean hasWriteLock() {
+  public boolean hasWriteLock() {
     return this.fsLock.isWriteLockedByCurrentThread();
     return this.fsLock.isWriteLockedByCurrentThread();
   }
   }
 
 
@@ -1014,7 +1020,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   }
   }
 
 
   /** Create a LocatedBlock. */
   /** Create a LocatedBlock. */
-  LocatedBlock createLocatedBlock(final Block b, final DatanodeInfo[] locations,
+  public LocatedBlock createLocatedBlock(final Block b, final DatanodeInfo[] locations,
       final long offset, final boolean corrupt) throws IOException {
       final long offset, final boolean corrupt) throws IOException {
     return new LocatedBlock(getExtendedBlock(b), locations, offset, corrupt);
     return new LocatedBlock(getExtendedBlock(b), locations, offset, corrupt);
   }
   }
@@ -3013,7 +3019,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * @return an array of datanode commands 
    * @return an array of datanode commands 
    * @throws IOException
    * @throws IOException
    */
    */
-  DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
+  public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
       long capacity, long dfsUsed, long remaining, long blockPoolUsed,
       long capacity, long dfsUsed, long remaining, long blockPoolUsed,
       int xceiverCount, int xmitsInProgress, int failedVolumes) 
       int xceiverCount, int xmitsInProgress, int failedVolumes) 
         throws IOException {
         throws IOException {
@@ -3521,7 +3527,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * If no such a node is available,
    * If no such a node is available,
    * then pick a node with least free space
    * then pick a node with least free space
    */
    */
-  void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess, 
+  public void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess, 
                               Block b, short replication,
                               Block b, short replication,
                               DatanodeDescriptor addedNode,
                               DatanodeDescriptor addedNode,
                               DatanodeDescriptor delNodeHint,
                               DatanodeDescriptor delNodeHint,
@@ -3785,9 +3791,19 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
             nodes.add(dn);
             nodes.add(dn);
           }
           }
           //Remove any form of the this datanode in include/exclude lists.
           //Remove any form of the this datanode in include/exclude lists.
-          mustList.remove(dn.getName());
-          mustList.remove(dn.getHost());
-          mustList.remove(dn.getHostName());
+          try {
+            InetAddress inet = InetAddress.getByName(dn.getHost());
+            // compare hostname(:port)
+            mustList.remove(inet.getHostName());
+            mustList.remove(inet.getHostName()+":"+dn.getPort());
+            // compare ipaddress(:port)
+            mustList.remove(inet.getHostAddress().toString());
+            mustList.remove(inet.getHostAddress().toString()+ ":" +dn.getPort());
+          } catch ( UnknownHostException e ) {
+            mustList.remove(dn.getName());
+            mustList.remove(dn.getHost());
+            LOG.warn(e);
+          }
         }
         }
       }
       }
       
       
@@ -3969,45 +3985,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
     return replication;
     return replication;
   }
   }
     
     
-  /**
-   * A immutable object that stores the number of live replicas and
-   * the number of decommissined Replicas.
-   */
-  static class NumberReplicas {
-    private int liveReplicas;
-    int decommissionedReplicas;
-    private int corruptReplicas;
-    private int excessReplicas;
-
-    NumberReplicas() {
-      initialize(0, 0, 0, 0);
-    }
-
-    NumberReplicas(int live, int decommissioned, int corrupt, int excess) {
-      initialize(live, decommissioned, corrupt, excess);
-    }
-
-    void initialize(int live, int decommissioned, int corrupt, int excess) {
-      liveReplicas = live;
-      decommissionedReplicas = decommissioned;
-      corruptReplicas = corrupt;
-      excessReplicas = excess;
-    }
-
-    int liveReplicas() {
-      return liveReplicas;
-    }
-    int decommissionedReplicas() {
-      return decommissionedReplicas;
-    }
-    int corruptReplicas() {
-      return corruptReplicas;
-    }
-    int excessReplicas() {
-      return excessReplicas;
-    }
-  } 
-
   /**
   /**
    * Change, if appropriate, the admin state of a datanode to 
    * Change, if appropriate, the admin state of a datanode to 
    * decommission completed. Return true if decommission is complete.
    * decommission completed. Return true if decommission is complete.
@@ -4032,23 +4009,62 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    */
    */
   private boolean inHostsList(DatanodeID node, String ipAddr) {
   private boolean inHostsList(DatanodeID node, String ipAddr) {
     Set<String> hostsList = hostsReader.getHosts();
     Set<String> hostsList = hostsReader.getHosts();
-    return (hostsList.isEmpty() || 
-            (ipAddr != null && hostsList.contains(ipAddr)) ||
-            hostsList.contains(node.getHost()) ||
-            hostsList.contains(node.getName()) || 
-            ((node instanceof DatanodeInfo) && 
-             hostsList.contains(((DatanodeInfo)node).getHostName())));
+     return checkInList(node, ipAddr, hostsList, false);
   }
   }
   
   
   private boolean inExcludedHostsList(DatanodeID node, String ipAddr) {
   private boolean inExcludedHostsList(DatanodeID node, String ipAddr) {
     Set<String> excludeList = hostsReader.getExcludedHosts();
     Set<String> excludeList = hostsReader.getExcludedHosts();
-    return  ((ipAddr != null && excludeList.contains(ipAddr)) ||
-            excludeList.contains(node.getHost()) ||
-            excludeList.contains(node.getName()) ||
-            ((node instanceof DatanodeInfo) && 
-             excludeList.contains(((DatanodeInfo)node).getHostName())));
+    return checkInList(node, ipAddr, excludeList, true);
   }
   }
 
 
+
+  /**
+   * Check if the given node (of DatanodeID or ipAddress) is in the (include or 
+   * exclude) list.  If ipAddress in null, check only based upon the given 
+   * DatanodeID.  If ipAddress is not null, the ipAddress should refers to the
+   * same host that given DatanodeID refers to.
+   * 
+   * @param node, DatanodeID, the host DatanodeID
+   * @param ipAddress, if not null, should refers to the same host
+   *                   that DatanodeID refers to
+   * @param hostsList, the list of hosts in the include/exclude file
+   * @param isExcludeList, boolean, true if this is the exclude list
+   * @return boolean, if in the list
+   */
+  private boolean checkInList(DatanodeID node, String ipAddress,
+      Set<String> hostsList, boolean isExcludeList) {
+    InetAddress iaddr = null;
+    try {
+      if (ipAddress != null) {
+        iaddr = InetAddress.getByName(ipAddress);
+      } else {
+        iaddr = InetAddress.getByName(node.getHost());
+      }
+    }catch (UnknownHostException e) {
+      LOG.warn("Unknown host in host list: "+ipAddress);
+      // can't resolve the host name.
+      if (isExcludeList){
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    // if include list is empty, host is in include list
+    if ( (!isExcludeList) && (hostsList.isEmpty()) ){
+      return true;
+    }
+    return // compare ipaddress(:port)
+    (hostsList.contains(iaddr.getHostAddress().toString()))
+        || (hostsList.contains(iaddr.getHostAddress().toString() + ":"
+            + node.getPort()))
+        // compare hostname(:port)
+        || (hostsList.contains(iaddr.getHostName()))
+        || (hostsList.contains(iaddr.getHostName() + ":" + node.getPort()))
+        || ((node instanceof DatanodeInfo) && hostsList
+            .contains(((DatanodeInfo) node).getHostName()));
+  }
+  
   /**
   /**
    * Rereads the config to get hosts and exclude list file names.
    * Rereads the config to get hosts and exclude list file names.
    * Rereads the files to update the hosts and exclude lists.  It
    * Rereads the files to update the hosts and exclude lists.  It
@@ -4626,7 +4642,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * Check whether the name node is in safe mode.
    * Check whether the name node is in safe mode.
    * @return true if safe mode is ON, false otherwise
    * @return true if safe mode is ON, false otherwise
    */
    */
-  boolean isInSafeMode() {
+  public boolean isInSafeMode() {
     // safeMode is volatile, and may be set to null at any time
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
     if (safeMode == null)
@@ -4637,7 +4653,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   /**
   /**
    * Check whether the name node is in startup mode.
    * Check whether the name node is in startup mode.
    */
    */
-  boolean isInStartupSafeMode() {
+  public boolean isInStartupSafeMode() {
     // safeMode is volatile, and may be set to null at any time
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
     if (safeMode == null)
@@ -4648,7 +4664,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   /**
   /**
    * Check whether replication queues are populated.
    * Check whether replication queues are populated.
    */
    */
-  boolean isPopulatingReplQueues() {
+  public boolean isPopulatingReplQueues() {
     // safeMode is volatile, and may be set to null at any time
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
     if (safeMode == null)
@@ -4660,7 +4676,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * Increment number of blocks that reached minimal replication.
    * Increment number of blocks that reached minimal replication.
    * @param replication current replication 
    * @param replication current replication 
    */
    */
-  void incrementSafeBlockCount(int replication) {
+  public void incrementSafeBlockCount(int replication) {
     // safeMode is volatile, and may be set to null at any time
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
     if (safeMode == null)
@@ -4671,7 +4687,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   /**
   /**
    * Decrement number of blocks that reached minimal replication.
    * Decrement number of blocks that reached minimal replication.
    */
    */
-  void decrementSafeBlockCount(Block b) {
+  public void decrementSafeBlockCount(Block b) {
     // safeMode is volatile, and may be set to null at any time
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null) // mostly true
     if (safeMode == null) // mostly true
@@ -4978,13 +4994,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   @Override // FSNamesystemMBean
   @Override // FSNamesystemMBean
   @Metric
   @Metric
   public long getPendingReplicationBlocks() {
   public long getPendingReplicationBlocks() {
-    return blockManager.pendingReplicationBlocksCount;
+    return blockManager.getPendingReplicationBlocksCount();
   }
   }
 
 
   @Override // FSNamesystemMBean
   @Override // FSNamesystemMBean
   @Metric
   @Metric
   public long getUnderReplicatedBlocks() {
   public long getUnderReplicatedBlocks() {
-    return blockManager.underReplicatedBlocksCount;
+    return blockManager.getUnderReplicatedBlocksCount();
   }
   }
 
 
   /** Return number of under-replicated but not missing blocks */
   /** Return number of under-replicated but not missing blocks */
@@ -4995,23 +5011,23 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   /** Returns number of blocks with corrupt replicas */
   /** Returns number of blocks with corrupt replicas */
   @Metric({"CorruptBlocks", "Number of blocks with corrupt replicas"})
   @Metric({"CorruptBlocks", "Number of blocks with corrupt replicas"})
   public long getCorruptReplicaBlocks() {
   public long getCorruptReplicaBlocks() {
-    return blockManager.corruptReplicaBlocksCount;
+    return blockManager.getCorruptReplicaBlocksCount();
   }
   }
 
 
   @Override // FSNamesystemMBean
   @Override // FSNamesystemMBean
   @Metric
   @Metric
   public long getScheduledReplicationBlocks() {
   public long getScheduledReplicationBlocks() {
-    return blockManager.scheduledReplicationBlocksCount;
+    return blockManager.getScheduledReplicationBlocksCount();
   }
   }
 
 
   @Metric
   @Metric
   public long getPendingDeletionBlocks() {
   public long getPendingDeletionBlocks() {
-    return blockManager.pendingDeletionBlocksCount;
+    return blockManager.getPendingDeletionBlocksCount();
   }
   }
 
 
   @Metric
   @Metric
   public long getExcessBlocks() {
   public long getExcessBlocks() {
-    return blockManager.excessBlocksCount;
+    return blockManager.getExcessBlocksCount();
   }
   }
   
   
   @Metric
   @Metric
@@ -5380,7 +5396,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   }
   }
 
 
   /** Get a datanode descriptor given corresponding storageID */
   /** Get a datanode descriptor given corresponding storageID */
-  DatanodeDescriptor getDatanode(String nodeID) {
+  public DatanodeDescriptor getDatanode(String nodeID) {
     assert hasReadOrWriteLock();
     assert hasReadOrWriteLock();
     return datanodeMap.get(nodeID);
     return datanodeMap.get(nodeID);
   }
   }
@@ -5444,7 +5460,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
       if (startBlockAfter != null) {
       if (startBlockAfter != null) {
         startBlockId = Block.filename2id(startBlockAfter);
         startBlockId = Block.filename2id(startBlockAfter);
       }
       }
-      BlockIterator blkIterator = blockManager.getCorruptReplicaBlockIterator();
+      UnderReplicatedBlocks.BlockIterator blkIterator = blockManager.getCorruptReplicaBlockIterator();
       while (blkIterator.hasNext()) {
       while (blkIterator.hasNext()) {
         Block blk = blkIterator.next();
         Block blk = blkIterator.next();
         INode inode = blockManager.getINode(blk);
         INode inode = blockManager.getINode(blk);

+ 1 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java

@@ -117,7 +117,7 @@ public class FileDataServlet extends DfsServlet {
               .getParameter(JspHelper.DELEGATION_PARAMETER_NAME);
               .getParameter(JspHelper.DELEGATION_PARAMETER_NAME);
 
 
           HdfsFileStatus info = nn.getFileInfo(path);
           HdfsFileStatus info = nn.getFileInfo(path);
-          if ((info != null) && !info.isDir()) {
+          if (info != null && !info.isDir()) {
             try {
             try {
               response.sendRedirect(createUri(path, info, ugi, nn, request,
               response.sendRedirect(createUri(path, info, ugi, nn, request,
                   delegationToken).toURL().toString());
                   delegationToken).toURL().toString());

+ 4 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/Host2NodesMap.java

@@ -17,10 +17,13 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
-import java.util.*;
+import java.util.HashMap;
+import java.util.Random;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+
 class Host2NodesMap {
 class Host2NodesMap {
   private HashMap<String, DatanodeDescriptor[]> map
   private HashMap<String, DatanodeDescriptor[]> map
     = new HashMap<String, DatanodeDescriptor[]>();
     = new HashMap<String, DatanodeDescriptor[]>();

+ 6 - 4
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -20,11 +20,13 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.List;
 import java.util.List;
 
 
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 
 
 /**
 /**
@@ -32,7 +34,7 @@ import org.apache.hadoop.util.StringUtils;
  * This is a base INode class containing common fields for file and 
  * This is a base INode class containing common fields for file and 
  * directory inodes.
  * directory inodes.
  */
  */
-abstract class INode implements Comparable<byte[]>, FSInodeInfo {
+public abstract class INode implements Comparable<byte[]>, FSInodeInfo {
   /*
   /*
    *  The inode name is in java UTF8 encoding; 
    *  The inode name is in java UTF8 encoding; 
    *  The name in HdfsFileStatus should keep the same encoding as this.
    *  The name in HdfsFileStatus should keep the same encoding as this.
@@ -324,7 +326,7 @@ abstract class INode implements Comparable<byte[]>, FSInodeInfo {
   /**
   /**
    * Is this inode being constructed?
    * Is this inode being constructed?
    */
    */
-  boolean isUnderConstruction() {
+  public boolean isUnderConstruction() {
     return false;
     return false;
   }
   }
 
 

+ 14 - 3
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -409,20 +409,31 @@ class INodeDirectory extends INode {
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   long[] computeContentSummary(long[] summary) {
   long[] computeContentSummary(long[] summary) {
+    // Walk through the children of this node, using a new summary array
+    // for the (sub)tree rooted at this node
+    assert 4 == summary.length;
+    long[] subtreeSummary = new long[]{0,0,0,0};
     if (children != null) {
     if (children != null) {
       for (INode child : children) {
       for (INode child : children) {
-        child.computeContentSummary(summary);
+        child.computeContentSummary(subtreeSummary);
       }
       }
     }
     }
     if (this instanceof INodeDirectoryWithQuota) {
     if (this instanceof INodeDirectoryWithQuota) {
       // Warn if the cached and computed diskspace values differ
       // Warn if the cached and computed diskspace values differ
       INodeDirectoryWithQuota node = (INodeDirectoryWithQuota)this;
       INodeDirectoryWithQuota node = (INodeDirectoryWithQuota)this;
       long space = node.diskspaceConsumed();
       long space = node.diskspaceConsumed();
-      if (-1 != node.getDsQuota() && space != summary[3]) {
+      assert -1 == node.getDsQuota() || space == subtreeSummary[3];
+      if (-1 != node.getDsQuota() && space != subtreeSummary[3]) {
         NameNode.LOG.warn("Inconsistent diskspace for directory "
         NameNode.LOG.warn("Inconsistent diskspace for directory "
-            +getLocalName()+". Cached: "+space+" Computed: "+summary[3]);
+            +getLocalName()+". Cached: "+space+" Computed: "+subtreeSummary[3]);
       }
       }
     }
     }
+
+    // update the passed summary array with the values for this node's subtree
+    for (int i = 0; i < summary.length; i++) {
+      summary[i] += subtreeSummary[i];
+    }
+
     summary[2]++;
     summary[2]++;
     return summary;
     return summary;
   }
   }

+ 9 - 5
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -24,8 +24,11 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 
 
-class INodeFile extends INode {
+/** I-node for closed file. */
+public class INodeFile extends INode {
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
 
 
   //Number of bits for Block size
   //Number of bits for Block size
@@ -106,7 +109,7 @@ class INodeFile extends INode {
    * Get file blocks 
    * Get file blocks 
    * @return file blocks
    * @return file blocks
    */
    */
-  BlockInfo[] getBlocks() {
+  public BlockInfo[] getBlocks() {
     return this.blocks;
     return this.blocks;
   }
   }
 
 
@@ -149,7 +152,7 @@ class INodeFile extends INode {
   /**
   /**
    * Set file block
    * Set file block
    */
    */
-  void setBlock(int idx, BlockInfo blk) {
+  public void setBlock(int idx, BlockInfo blk) {
     this.blocks[idx] = blk;
     this.blocks[idx] = blk;
   }
   }
 
 
@@ -237,7 +240,7 @@ class INodeFile extends INode {
    * Get the last block of the file.
    * Get the last block of the file.
    * Make sure it has the right type.
    * Make sure it has the right type.
    */
    */
-  <T extends BlockInfo> T getLastBlock() throws IOException {
+  public <T extends BlockInfo> T getLastBlock() throws IOException {
     if (blocks == null || blocks.length == 0)
     if (blocks == null || blocks.length == 0)
       return null;
       return null;
     T returnBlock = null;
     T returnBlock = null;
@@ -252,7 +255,8 @@ class INodeFile extends INode {
     return returnBlock;
     return returnBlock;
   }
   }
 
 
-  int numBlocks() {
+  /** @return the number of blocks */ 
+  public int numBlocks() {
     return blocks == null ? 0 : blocks.length;
     return blocks == null ? 0 : blocks.length;
   }
   }
 }
 }

+ 10 - 5
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java

@@ -21,10 +21,15 @@ import java.io.IOException;
 
 
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 
 
-
-class INodeFileUnderConstruction extends INodeFile {
+/**
+ * I-node for file being written.
+ */
+public class INodeFileUnderConstruction extends INodeFile {
   private  String clientName;         // lease holder
   private  String clientName;         // lease holder
   private final String clientMachine;
   private final String clientMachine;
   private final DatanodeDescriptor clientNode; // if client is a cluster node too.
   private final DatanodeDescriptor clientNode; // if client is a cluster node too.
@@ -43,7 +48,7 @@ class INodeFileUnderConstruction extends INodeFile {
     this.clientNode = clientNode;
     this.clientNode = clientNode;
   }
   }
 
 
-  public INodeFileUnderConstruction(byte[] name,
+  INodeFileUnderConstruction(byte[] name,
                              short blockReplication,
                              short blockReplication,
                              long modificationTime,
                              long modificationTime,
                              long preferredBlockSize,
                              long preferredBlockSize,
@@ -80,7 +85,7 @@ class INodeFileUnderConstruction extends INodeFile {
    * Is this inode being constructed?
    * Is this inode being constructed?
    */
    */
   @Override
   @Override
-  boolean isUnderConstruction() {
+  public boolean isUnderConstruction() {
     return true;
     return true;
   }
   }
 
 
@@ -122,7 +127,7 @@ class INodeFileUnderConstruction extends INodeFile {
    * Convert the last block of the file to an under-construction block.
    * Convert the last block of the file to an under-construction block.
    * Set its locations.
    * Set its locations.
    */
    */
-  BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
+  public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
                                           DatanodeDescriptor[] targets)
                                           DatanodeDescriptor[] targets)
   throws IOException {
   throws IOException {
     if (blocks == null || blocks.length == 0) {
     if (blocks == null || blocks.length == 0) {

+ 1 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

@@ -163,7 +163,7 @@ public class LeaseManager {
   /**
   /**
    * Finds the pathname for the specified pendingFile
    * Finds the pathname for the specified pendingFile
    */
    */
-  synchronized String findPath(INodeFileUnderConstruction pendingFile)
+  public synchronized String findPath(INodeFileUnderConstruction pendingFile)
       throws IOException {
       throws IOException {
     Lease lease = getLease(pendingFile.getClientName());
     Lease lease = getLease(pendingFile.getClientName());
     if (lease != null) {
     if (lease != null) {

+ 4 - 3
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java

@@ -464,6 +464,7 @@ public class NNStorage extends Storage implements Closeable {
         // Close any edits stream associated with this dir and remove directory
         // Close any edits stream associated with this dir and remove directory
         LOG.warn("writeTransactionIdToStorage failed on " + sd,
         LOG.warn("writeTransactionIdToStorage failed on " + sd,
             e);
             e);
+        reportErrorsOnDirectory(sd);
       }
       }
     }
     }
   }
   }
@@ -828,17 +829,17 @@ public class NNStorage extends Storage implements Closeable {
    * @throws IOException
    * @throws IOException
    */
    */
   void reportErrorsOnDirectory(StorageDirectory sd) {
   void reportErrorsOnDirectory(StorageDirectory sd) {
-    LOG.warn("Error reported on storage directory " + sd);
+    LOG.error("Error reported on storage directory " + sd);
 
 
     String lsd = listStorageDirectories();
     String lsd = listStorageDirectories();
     LOG.debug("current list of storage dirs:" + lsd);
     LOG.debug("current list of storage dirs:" + lsd);
 
 
-    LOG.info("About to remove corresponding storage: "
+    LOG.warn("About to remove corresponding storage: "
              + sd.getRoot().getAbsolutePath());
              + sd.getRoot().getAbsolutePath());
     try {
     try {
       sd.unlock();
       sd.unlock();
     } catch (Exception e) {
     } catch (Exception e) {
-      LOG.info("Unable to unlock bad storage directory: "
+      LOG.warn("Unable to unlock bad storage directory: "
                +  sd.getRoot().getPath(), e);
                +  sd.getRoot().getPath(), e);
     }
     }
 
 

+ 30 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -248,7 +248,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
   /** Return the {@link FSNamesystem} object.
   /** Return the {@link FSNamesystem} object.
    * @return {@link FSNamesystem} object.
    * @return {@link FSNamesystem} object.
    */
    */
-  FSNamesystem getNamesystem() {
+  public FSNamesystem getNamesystem() {
     return namesystem;
     return namesystem;
   }
   }
 
 
@@ -439,6 +439,13 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     // The rpc-server port can be ephemeral... ensure we have the correct info
     // The rpc-server port can be ephemeral... ensure we have the correct info
     this.rpcAddress = this.server.getListenerAddress(); 
     this.rpcAddress = this.server.getListenerAddress(); 
     setRpcServerAddress(conf);
     setRpcServerAddress(conf);
+    
+    try {
+      validateConfigurationSettings(conf);
+    } catch (IOException e) {
+      LOG.fatal(e.toString());
+      throw e;
+    }
 
 
     activate(conf);
     activate(conf);
     LOG.info(getRole() + " up at: " + rpcAddress);
     LOG.info(getRole() + " up at: " + rpcAddress);
@@ -446,6 +453,28 @@ public class NameNode implements NamenodeProtocols, FSConstants {
       LOG.info(getRole() + " service server is up at: " + serviceRPCAddress); 
       LOG.info(getRole() + " service server is up at: " + serviceRPCAddress); 
     }
     }
   }
   }
+  
+  /**
+   * Verifies that the final Configuration Settings look ok for the NameNode to
+   * properly start up
+   * Things to check for include:
+   * - HTTP Server Port does not equal the RPC Server Port
+   * @param conf
+   * @throws IOException
+   */
+  protected void validateConfigurationSettings(final Configuration conf) 
+      throws IOException {
+    // check to make sure the web port and rpc port do not match 
+    if(getHttpServerAddress(conf).getPort() 
+        == getRpcServerAddress(conf).getPort()) {
+      String errMsg = "dfs.namenode.rpc-address " +
+          "("+ getRpcServerAddress(conf) + ") and " +
+          "dfs.namenode.http-address ("+ getHttpServerAddress(conf) + ") " +
+          "configuration keys are bound to the same port, unable to start " +
+          "NameNode. Port: " + getRpcServerAddress(conf).getPort();
+      throw new IOException(errMsg);
+    } 
+  }
 
 
   /**
   /**
    * Activate name-node servers and threads.
    * Activate name-node servers and threads.

+ 3 - 3
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -36,20 +36,20 @@ 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.BlockReader;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 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.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.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.NodeBase;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 
 
 /**
 /**

+ 4 - 4
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java

@@ -36,24 +36,24 @@ 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.DFSUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 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;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
 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.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 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.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 org.znerd.xmlenc.*;
+import org.znerd.xmlenc.XMLOutputter;
 
 
 class NamenodeJspHelper {
 class NamenodeJspHelper {
   static String getSafeModeText(FSNamesystem fsn) {
   static String getSafeModeText(FSNamesystem fsn) {

+ 1 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -253,7 +253,7 @@ public class SecondaryNameNode implements Runnable {
             infoServer.addSslListener(secInfoSocAddr, conf, false, true);
             infoServer.addSslListener(secInfoSocAddr, conf, false, true);
           }
           }
           
           
-          infoServer.setAttribute("secondary.name.node", this);
+          infoServer.setAttribute("secondary.name.node", SecondaryNameNode.this);
           infoServer.setAttribute("name.system.image", checkpointImage);
           infoServer.setAttribute("name.system.image", checkpointImage);
           infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);
           infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);
           infoServer.addInternalServlet("getimage", "/getimage",
           infoServer.addInternalServlet("getimage", "/getimage",

+ 54 - 76
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.DFSInputStream;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeJspHelper;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeJspHelper;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.mortbay.jetty.InclusiveByteRange;
 import org.mortbay.jetty.InclusiveByteRange;
 
 
@@ -46,7 +47,7 @@ public class StreamFile extends DfsServlet {
 
 
   public static final String CONTENT_LENGTH = "Content-Length";
   public static final String CONTENT_LENGTH = "Content-Length";
 
 
-  /** getting a client for connecting to dfs */
+  /* Return a DFS client to use to make the given HTTP request */
   protected DFSClient getDFSClient(HttpServletRequest request)
   protected DFSClient getDFSClient(HttpServletRequest request)
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
     final Configuration conf =
     final Configuration conf =
@@ -57,6 +58,7 @@ public class StreamFile extends DfsServlet {
     return DatanodeJspHelper.getDFSClient(request, datanode, conf, ugi);
     return DatanodeJspHelper.getDFSClient(request, datanode, conf, ugi);
   }
   }
   
   
+  @SuppressWarnings("unchecked")
   public void doGet(HttpServletRequest request, HttpServletResponse response)
   public void doGet(HttpServletRequest request, HttpServletResponse response)
     throws ServletException, IOException {
     throws ServletException, IOException {
     final String path = request.getPathInfo() != null ? 
     final String path = request.getPathInfo() != null ? 
@@ -69,9 +71,10 @@ public class StreamFile extends DfsServlet {
       return;
       return;
     }
     }
     
     
-    Enumeration<?> reqRanges = request.getHeaders("Range");
-    if (reqRanges != null && !reqRanges.hasMoreElements())
+    Enumeration<String> reqRanges = request.getHeaders("Range");
+    if (reqRanges != null && !reqRanges.hasMoreElements()) {
       reqRanges = null;
       reqRanges = null;
+    }
 
 
     DFSClient dfs;
     DFSClient dfs;
     try {
     try {
@@ -81,107 +84,82 @@ public class StreamFile extends DfsServlet {
       return;
       return;
     }
     }
     
     
-    final DFSInputStream in = dfs.open(filename);
-    final long fileLen = in.getFileLength();
-    OutputStream os = response.getOutputStream();
+    DFSInputStream in = null;
+    OutputStream out = null;
 
 
     try {
     try {
+      in = dfs.open(filename);
+      out = response.getOutputStream();
+      final long fileLen = in.getFileLength();
       if (reqRanges != null) {
       if (reqRanges != null) {
-        List<?> ranges = InclusiveByteRange.satisfiableRanges(reqRanges,
-                                                           fileLen);
-        StreamFile.sendPartialData(in, os, response, fileLen, ranges);
+        List<InclusiveByteRange> ranges = 
+          InclusiveByteRange.satisfiableRanges(reqRanges, fileLen);
+        StreamFile.sendPartialData(in, out, response, fileLen, ranges);
       } else {
       } else {
         // No ranges, so send entire file
         // No ranges, so send entire file
         response.setHeader("Content-Disposition", "attachment; filename=\"" + 
         response.setHeader("Content-Disposition", "attachment; filename=\"" + 
                            filename + "\"");
                            filename + "\"");
         response.setContentType("application/octet-stream");
         response.setContentType("application/octet-stream");
         response.setHeader(CONTENT_LENGTH, "" + fileLen);
         response.setHeader(CONTENT_LENGTH, "" + fileLen);
-        StreamFile.writeTo(in, os, 0L, fileLen);
+        StreamFile.copyFromOffset(in, out, 0L, fileLen);
       }
       }
-    } catch(IOException e) {
+      in.close();
+      in = null;
+      out.close();
+      out = null;
+      dfs.close();
+      dfs = null;
+    } catch (IOException ioe) {
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("response.isCommitted()=" + response.isCommitted(), e);
+        LOG.debug("response.isCommitted()=" + response.isCommitted(), ioe);
       }
       }
-      throw e;
+      throw ioe;
     } finally {
     } finally {
-      try {
-        in.close();
-        os.close();
-      } finally {
-        dfs.close();
-      }
-    }      
+      IOUtils.cleanup(LOG, in);
+      IOUtils.cleanup(LOG, out);
+      IOUtils.cleanup(LOG, dfs);
+    }
   }
   }
   
   
+  /**
+   * Send a partial content response with the given range. If there are
+   * no satisfiable ranges, or if multiple ranges are requested, which
+   * is unsupported, respond with range not satisfiable.
+   *
+   * @param in stream to read from
+   * @param out stream to write to
+   * @param response http response to use
+   * @param contentLength for the response header
+   * @param ranges to write to respond with
+   * @throws IOException on error sending the response
+   */
   static void sendPartialData(FSInputStream in,
   static void sendPartialData(FSInputStream in,
-                              OutputStream os,
+                              OutputStream out,
                               HttpServletResponse response,
                               HttpServletResponse response,
                               long contentLength,
                               long contentLength,
-                              List<?> ranges)
-  throws IOException {
-
+                              List<InclusiveByteRange> ranges)
+      throws IOException {
     if (ranges == null || ranges.size() != 1) {
     if (ranges == null || ranges.size() != 1) {
-      //  if there are no satisfiable ranges, or if multiple ranges are
-      // requested (we don't support multiple range requests), send 416 response
       response.setContentLength(0);
       response.setContentLength(0);
-      int status = HttpServletResponse.SC_REQUESTED_RANGE_NOT_SATISFIABLE;
-      response.setStatus(status);
-      response.setHeader("Content-Range", 
+      response.setStatus(HttpServletResponse.SC_REQUESTED_RANGE_NOT_SATISFIABLE);
+      response.setHeader("Content-Range",
                 InclusiveByteRange.to416HeaderRangeString(contentLength));
                 InclusiveByteRange.to416HeaderRangeString(contentLength));
     } else {
     } else {
-      //  if there is only a single valid range (must be satisfiable 
-      //  since were here now), send that range with a 206 response
-      InclusiveByteRange singleSatisfiableRange =
-        (InclusiveByteRange)ranges.get(0);
+      InclusiveByteRange singleSatisfiableRange = ranges.get(0);
       long singleLength = singleSatisfiableRange.getSize(contentLength);
       long singleLength = singleSatisfiableRange.getSize(contentLength);
       response.setStatus(HttpServletResponse.SC_PARTIAL_CONTENT);
       response.setStatus(HttpServletResponse.SC_PARTIAL_CONTENT);
       response.setHeader("Content-Range", 
       response.setHeader("Content-Range", 
         singleSatisfiableRange.toHeaderRangeString(contentLength));
         singleSatisfiableRange.toHeaderRangeString(contentLength));
-      System.out.println("first: "+singleSatisfiableRange.getFirst(contentLength));
-      System.out.println("singleLength: "+singleLength);
-      
-      StreamFile.writeTo(in,
-                         os,
-                         singleSatisfiableRange.getFirst(contentLength),
-                         singleLength);
+      copyFromOffset(in, out,
+                     singleSatisfiableRange.getFirst(contentLength),
+                     singleLength);
     }
     }
   }
   }
-  
-  static void writeTo(FSInputStream in,
-                      OutputStream os,
-                      long start,
-                      long count) 
-  throws IOException {
-    byte buf[] = new byte[4096];
-    long bytesRemaining = count;
-    int bytesRead;
-    int bytesToRead;
-
-    in.seek(start);
-
-    while (true) {
-      // number of bytes to read this iteration
-      bytesToRead = (int)(bytesRemaining<buf.length ? 
-                                                      bytesRemaining:
-                                                      buf.length);
-      
-      // number of bytes actually read this iteration
-      bytesRead = in.read(buf, 0, bytesToRead);
-
-      // if we can't read anymore, break
-      if (bytesRead == -1) {
-        break;
-      } 
-      
-      os.write(buf, 0, bytesRead);
-
-      bytesRemaining -= bytesRead;
-
-      // if we don't need to read anymore, break
-      if (bytesRemaining <= 0) {
-        break;
-      }
 
 
-    } 
+  /* Copy count bytes at the given offset from one stream to another */
+  static void copyFromOffset(FSInputStream in, OutputStream out, long offset,
+      long count) throws IOException {
+    in.seek(offset);
+    IOUtils.copyBytes(in, out, count, false);
   }
   }
 }
 }

+ 8 - 3
hdfs/src/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java

@@ -17,15 +17,20 @@
  */
  */
 package org.apache.hadoop.hdfs.server.protocol;
 package org.apache.hadoop.hdfs.server.protocol;
 
 
-import java.io.*;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.util.List;
 import java.util.List;
 
 
 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.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.server.namenode.DatanodeDescriptor.BlockTargetPair;
-import org.apache.hadoop.io.*;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
 
 
 
 
 /****************************************************
 /****************************************************

+ 1 - 1
hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj

@@ -46,7 +46,7 @@ public aspect DataTransferProtocolAspects {
   */
   */
 
 
   pointcut receiverOp(DataXceiver dataxceiver):
   pointcut receiverOp(DataXceiver dataxceiver):
-    call(Op Receiver.readOp(DataInputStream)) && target(dataxceiver);
+    call(Op Receiver.readOp()) && target(dataxceiver);
 
 
   after(DataXceiver dataxceiver) returning(Op op): receiverOp(dataxceiver) {
   after(DataXceiver dataxceiver) returning(Op op): receiverOp(dataxceiver) {
     LOG.info("FI: receiverOp " + op + ", datanode="
     LOG.info("FI: receiverOp " + op + ", datanode="

+ 1 - 1
hdfs/src/test/hdfs/org/apache/hadoop/cli/testHDFSConf.xml

@@ -15440,7 +15440,7 @@
       <comparators>
       <comparators>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <type>RegexpComparator</type>
-          <expected-output>put: The DiskSpace quota of /dir1 is exceeded: quota=1024 diskspace consumed=[0-9.]+[kmg]*</expected-output>
+          <expected-output>put: The DiskSpace quota of /dir1 is exceeded: quota=1.0k diskspace consumed=[0-9.]+[kmg]*</expected-output>
         </comparator>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>

+ 3 - 3
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -61,10 +61,10 @@ 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;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
 import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 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;
@@ -683,8 +683,8 @@ public class DFSTestUtil {
     final DataInputStream in = new DataInputStream(NetUtils.getInputStream(s));
     final DataInputStream in = new DataInputStream(NetUtils.getInputStream(s));
 
 
     // send the request
     // send the request
-    Sender.opTransferBlock(out, b, dfsClient.clientName,
-        new DatanodeInfo[]{datanodes[1]}, new Token<BlockTokenIdentifier>());
+    new Sender(out).transferBlock(b, new Token<BlockTokenIdentifier>(),
+        dfsClient.clientName, new DatanodeInfo[]{datanodes[1]});
     out.flush();
     out.flush();
 
 
     return BlockOpResponseProto.parseDelimitedFrom(in);
     return BlockOpResponseProto.parseDelimitedFrom(in);

+ 27 - 12
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -320,7 +320,8 @@ public class MiniDFSCluster {
    * Servers will be started on free ports.
    * Servers will be started on free ports.
    * <p>
    * <p>
    * The caller must manage the creation of NameNode and DataNode directories
    * The caller must manage the creation of NameNode and DataNode directories
-   * and have already set dfs.namenode.name.dir and dfs.datanode.data.dir in the given conf.
+   * and have already set {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
+   * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} in the given conf.
    * 
    * 
    * @param conf the base configuration to use in starting the servers.  This
    * @param conf the base configuration to use in starting the servers.  This
    *          will be modified as necessary.
    *          will be modified as necessary.
@@ -381,7 +382,8 @@ public class MiniDFSCluster {
   
   
   /**
   /**
    * NOTE: if possible, the other constructors that don't have nameNode port 
    * NOTE: if possible, the other constructors that don't have nameNode port 
-   * parameter should be used as they will ensure that the servers use free ports.
+   * parameter should be used as they will ensure that the servers use free 
+   * ports.
    * <p>
    * <p>
    * Modify the config and start up the servers.  
    * Modify the config and start up the servers.  
    * 
    * 
@@ -390,9 +392,12 @@ public class MiniDFSCluster {
    * @param conf the base configuration to use in starting the servers.  This
    * @param conf the base configuration to use in starting the servers.  This
    *          will be modified as necessary.
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param numDataNodes Number of DataNodes to start; may be zero
-   * @param format if true, format the NameNode and DataNodes before starting up
+   * @param format if true, format the NameNode and DataNodes before starting 
+   *          up
    * @param manageDfsDirs if true, the data directories for servers will be
    * @param manageDfsDirs if true, the data directories for servers will be
-   *          created and dfs.namenode.name.dir and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
+   *          {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in 
+   *          the conf
    * @param operation the operation with which to start the servers.  If null
    * @param operation the operation with which to start the servers.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -422,7 +427,9 @@ public class MiniDFSCluster {
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param format if true, format the NameNode and DataNodes before starting up
    * @param format if true, format the NameNode and DataNodes before starting up
    * @param manageDfsDirs if true, the data directories for servers will be
    * @param manageDfsDirs if true, the data directories for servers will be
-   *          created and dfs.namenode.name.dir and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
+   *          {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in 
+   *          the conf
    * @param operation the operation with which to start the servers.  If null
    * @param operation the operation with which to start the servers.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -454,9 +461,12 @@ public class MiniDFSCluster {
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param format if true, format the NameNode and DataNodes before starting up
    * @param format if true, format the NameNode and DataNodes before starting up
    * @param manageNameDfsDirs if true, the data directories for servers will be
    * @param manageNameDfsDirs if true, the data directories for servers will be
-   *          created and dfs.namenode.name.dir and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
+   *          {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in 
+   *          the conf
    * @param manageDataDfsDirs if true, the data directories for datanodes will
    * @param manageDataDfsDirs if true, the data directories for datanodes will
-   *          be created and dfs.datanode.data.dir set to same in the conf
+   *          be created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} 
+   *          set to same in the conf
    * @param operation the operation with which to start the servers.  If null
    * @param operation the operation with which to start the servers.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -723,7 +733,8 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set 
+   *          in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -754,7 +765,8 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be 
+   *          set in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -787,7 +799,8 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be 
+   *          set in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -913,7 +926,8 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be 
+   *          set in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -943,7 +957,8 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will 
+   *          be set in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
    * @param racks array of strings indicating the rack that each DataNode is on

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

@@ -25,7 +25,7 @@ import junit.framework.TestCase;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 
 
 /**
 /**
  * This class tests DatanodeDescriptor.getBlocksScheduled() at the
  * This class tests DatanodeDescriptor.getBlocksScheduled() at the

+ 9 - 15
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestByteRangeInputStream.java

@@ -23,12 +23,13 @@ import java.io.InputStream;
 import java.net.HttpURLConnection;
 import java.net.HttpURLConnection;
 import java.net.MalformedURLException;
 import java.net.MalformedURLException;
 import java.net.URL;
 import java.net.URL;
-import junit.framework.TestCase;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+
 import org.apache.hadoop.hdfs.ByteRangeInputStream;
 import org.apache.hadoop.hdfs.ByteRangeInputStream;
 import org.apache.hadoop.hdfs.ByteRangeInputStream.URLOpener;
 import org.apache.hadoop.hdfs.ByteRangeInputStream.URLOpener;
 
 
+import org.junit.Test;
+import static org.junit.Assert.*;
+
 class MockHttpURLConnection extends HttpURLConnection {
 class MockHttpURLConnection extends HttpURLConnection {
   MockURL m;
   MockURL m;
   
   
@@ -101,13 +102,9 @@ class MockURL extends URLOpener {
   }
   }
 }
 }
 
 
-
-
-public class TestByteRangeInputStream extends TestCase {
-  
-  private static final Log LOG = 
-                           LogFactory.getLog(TestByteRangeInputStream.class);
+public class TestByteRangeInputStream {
   
   
+  @Test
   public void testByteRange() throws IOException, InterruptedException {
   public void testByteRange() throws IOException, InterruptedException {
     MockURL o = new MockURL("http://test/");
     MockURL o = new MockURL("http://test/");
     MockURL r =  new MockURL((URL)null);
     MockURL r =  new MockURL((URL)null);
@@ -149,7 +146,7 @@ public class TestByteRangeInputStream extends TestCase {
     is.seek(101);
     is.seek(101);
     is.read();
     is.read();
 
 
-    assertNull("Seek to 101 should not result in another request", null);
+    assertNull("Seek to 101 should not result in another request", r.getMsg());
 
 
     r.setMsg(null);
     r.setMsg(null);
     is.seek(2500);
     is.seek(2500);
@@ -168,7 +165,7 @@ public class TestByteRangeInputStream extends TestCase {
            + "but 206 is expected");
            + "but 206 is expected");
     } catch (IOException e) {
     } catch (IOException e) {
       assertEquals("Should fail because incorrect response code was sent",
       assertEquals("Should fail because incorrect response code was sent",
-                   "206 expected, but received 200", e.getMessage());
+                   "HTTP_PARTIAL expected, received 200", e.getMessage());
     }
     }
 
 
     r.responseCode = 206;
     r.responseCode = 206;
@@ -180,10 +177,7 @@ public class TestByteRangeInputStream extends TestCase {
            + "but 200 is expected");
            + "but 200 is expected");
     } catch (IOException e) {
     } catch (IOException e) {
       assertEquals("Should fail because incorrect response code was sent",
       assertEquals("Should fail because incorrect response code was sent",
-                   "200 expected, but received 206", e.getMessage());
+                   "HTTP_OK expected, received 206", e.getMessage());
     }
     }
-
-
-
   }
   }
 }
 }

+ 36 - 17
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSPermission.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 import java.util.Random;
 import java.util.Random;
@@ -31,7 +30,6 @@ import junit.framework.TestCase;
 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.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
@@ -68,7 +66,6 @@ public class TestDFSPermission extends TestCase {
   final static private int NUM_TEST_PERMISSIONS = 
   final static private int NUM_TEST_PERMISSIONS = 
     conf.getInt("test.dfs.permission.num", 10) * (MAX_PERMISSION + 1) / 100;
     conf.getInt("test.dfs.permission.num", 10) * (MAX_PERMISSION + 1) / 100;
 
 
-
   final private static String PATH_NAME = "xx";
   final private static String PATH_NAME = "xx";
   final private static Path FILE_DIR_PATH = new Path("/", PATH_NAME);
   final private static Path FILE_DIR_PATH = new Path("/", PATH_NAME);
   final private static Path NON_EXISTENT_PATH = new Path("/parent", PATH_NAME);
   final private static Path NON_EXISTENT_PATH = new Path("/parent", PATH_NAME);
@@ -115,44 +112,66 @@ public class TestDFSPermission extends TestCase {
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     try {
     try {
       cluster.waitActive();
       cluster.waitActive();
-      fs = FileSystem.get(conf);
-      LOG.info("ROOT=" + fs.getFileStatus(new Path("/")));
       testPermissionSetting(OpType.CREATE); // test file creation
       testPermissionSetting(OpType.CREATE); // test file creation
       testPermissionSetting(OpType.MKDIRS); // test directory creation
       testPermissionSetting(OpType.MKDIRS); // test directory creation
     } finally {
     } finally {
-      fs.close();
       cluster.shutdown();
       cluster.shutdown();
     }
     }
   }
   }
 
 
+  private void initFileSystem(short umask) throws Exception {
+    // set umask in configuration, converting to padded octal
+    conf.set(FsPermission.UMASK_LABEL, String.format("%1$03o", umask));
+    fs = FileSystem.get(conf);
+  }
+
+  private void closeFileSystem() throws Exception {
+    fs.close();
+  }
+  
   /* check permission setting works correctly for file or directory */
   /* check permission setting works correctly for file or directory */
   private void testPermissionSetting(OpType op) throws Exception {
   private void testPermissionSetting(OpType op) throws Exception {
+    short uMask = DEFAULT_UMASK;
     // case 1: use default permission but all possible umasks
     // case 1: use default permission but all possible umasks
     PermissionGenerator generator = new PermissionGenerator(r);
     PermissionGenerator generator = new PermissionGenerator(r);
+    FsPermission permission = new FsPermission(DEFAULT_PERMISSION);
     for (short i = 0; i < NUM_TEST_PERMISSIONS; i++) {
     for (short i = 0; i < NUM_TEST_PERMISSIONS; i++) {
-      createAndCheckPermission(op, FILE_DIR_PATH, generator.next(),
-          new FsPermission(DEFAULT_PERMISSION), true);
+      uMask = generator.next();
+      initFileSystem(uMask);
+      createAndCheckPermission(op, FILE_DIR_PATH, uMask, permission, true);
+      closeFileSystem();
     }
     }
-
     // case 2: use permission 0643 and the default umask
     // case 2: use permission 0643 and the default umask
-    createAndCheckPermission(op, FILE_DIR_PATH, DEFAULT_UMASK,
-        new FsPermission((short) 0643), true);
+    uMask = DEFAULT_UMASK;
+    initFileSystem(uMask);
+    createAndCheckPermission(op, FILE_DIR_PATH, uMask, new FsPermission(
+        (short) 0643), true);
+    closeFileSystem();
 
 
     // case 3: use permission 0643 and umask 0222
     // case 3: use permission 0643 and umask 0222
-    createAndCheckPermission(op, FILE_DIR_PATH, (short) 0222, 
-        new FsPermission((short) 0643), false);
+    uMask = (short) 0222;
+    initFileSystem(uMask);
+    createAndCheckPermission(op, FILE_DIR_PATH, uMask, new FsPermission(
+        (short) 0643), false);
+    closeFileSystem();
 
 
     // case 4: set permission
     // case 4: set permission
-    fs.setPermission(FILE_DIR_PATH, new FsPermission((short) 0111));
+    uMask = (short) 0111;
+    initFileSystem(uMask);
+    fs.setPermission(FILE_DIR_PATH, new FsPermission(uMask));
     short expectedPermission = (short) ((op == OpType.CREATE) ? 0 : 0111);
     short expectedPermission = (short) ((op == OpType.CREATE) ? 0 : 0111);
     checkPermission(FILE_DIR_PATH, expectedPermission, true);
     checkPermission(FILE_DIR_PATH, expectedPermission, true);
+    closeFileSystem();
 
 
     // case 5: test non-existent parent directory
     // case 5: test non-existent parent directory
-    assertFalse(fs.exists(NON_EXISTENT_PATH));
-    createAndCheckPermission(op, NON_EXISTENT_PATH, DEFAULT_UMASK,
-        new FsPermission(DEFAULT_PERMISSION), false);
+    uMask = DEFAULT_UMASK;
+    initFileSystem(uMask);
+    assertFalse("File shouldn't exists", fs.exists(NON_EXISTENT_PATH));
+    createAndCheckPermission(op, NON_EXISTENT_PATH, uMask, new FsPermission(
+        DEFAULT_PERMISSION), false);
     Path parent = NON_EXISTENT_PATH.getParent();
     Path parent = NON_EXISTENT_PATH.getParent();
     checkPermission(parent, getPermission(parent.getParent()), true);
     checkPermission(parent, getPermission(parent.getParent()), true);
+    closeFileSystem();
   }
   }
 
 
   /* get the permission of a file/directory */
   /* get the permission of a file/directory */

+ 6 - 3
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java

@@ -117,7 +117,8 @@ public class TestDFSStorageStateRecovery extends TestCase {
   
   
   /**
   /**
    * Sets up the storage directories for namenode as defined by
    * Sets up the storage directories for namenode as defined by
-   * dfs.namenode.name.dir. For each element in dfs.namenode.name.dir, the subdirectories 
+   * {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY}. For each element 
+   * in {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY}, the subdirectories 
    * represented by the first four elements of the <code>state</code> array
    * represented by the first four elements of the <code>state</code> array
    * will be created and populated.
    * will be created and populated.
    * 
    * 
@@ -145,7 +146,8 @@ public class TestDFSStorageStateRecovery extends TestCase {
   
   
   /**
   /**
    * Sets up the storage directories for a datanode under
    * Sets up the storage directories for a datanode under
-   * dfs.datanode.data.dir. For each element in dfs.datanode.data.dir, the subdirectories 
+   * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY}. For each element in 
+   * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY}, the subdirectories 
    * represented by the first four elements of the <code>state</code> array 
    * represented by the first four elements of the <code>state</code> array 
    * will be created and populated. 
    * will be created and populated. 
    * See {@link UpgradeUtilities#createDataNodeStorageDirs()}
    * See {@link UpgradeUtilities#createDataNodeStorageDirs()}
@@ -172,7 +174,8 @@ public class TestDFSStorageStateRecovery extends TestCase {
   
   
   /**
   /**
    * Sets up the storage directories for a block pool under
    * Sets up the storage directories for a block pool under
-   * dfs.datanode.data.dir. For each element in dfs.datanode.data.dir, the subdirectories 
+   * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY}. For each element 
+   * in {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY}, the subdirectories 
    * represented by the first four elements of the <code>state</code> array 
    * represented by the first four elements of the <code>state</code> array 
    * will be created and populated. 
    * will be created and populated. 
    * See {@link UpgradeUtilities#createBlockPoolStorageDirs()}
    * See {@link UpgradeUtilities#createBlockPoolStorageDirs()}

+ 71 - 11
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgrade.java

@@ -22,14 +22,12 @@ import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_N
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
-
-import junit.framework.TestCase;
+import java.util.regex.Pattern;
 
 
 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.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
@@ -37,6 +35,9 @@ import org.apache.hadoop.hdfs.server.namenode.TestParallelImageWrite;
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
 
 
+import org.apache.hadoop.util.StringUtils;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Joiner;
@@ -51,8 +52,7 @@ import static org.junit.Assert.*;
 public class TestDFSUpgrade {
 public class TestDFSUpgrade {
  
  
   private static final int EXPECTED_TXID = 17;
   private static final int EXPECTED_TXID = 17;
-  private static final Log LOG = LogFactory.getLog(
-                                                   "org.apache.hadoop.hdfs.TestDFSUpgrade");
+  private static final Log LOG = LogFactory.getLog(TestDFSUpgrade.class.getName());
   private Configuration conf;
   private Configuration conf;
   private int testCounter = 0;
   private int testCounter = 0;
   private MiniDFSCluster cluster = null;
   private MiniDFSCluster cluster = null;
@@ -125,11 +125,27 @@ public class TestDFSUpgrade {
       
       
     }
     }
   }
   }
+
   /**
   /**
    * Attempts to start a NameNode with the given operation.  Starting
    * Attempts to start a NameNode with the given operation.  Starting
    * the NameNode should throw an exception.
    * the NameNode should throw an exception.
    */
    */
   void startNameNodeShouldFail(StartupOption operation) {
   void startNameNodeShouldFail(StartupOption operation) {
+    startNameNodeShouldFail(operation, null, null);
+  }
+
+  /**
+   * Attempts to start a NameNode with the given operation.  Starting
+   * the NameNode should throw an exception.
+   * @param operation - NameNode startup operation
+   * @param exceptionClass - if non-null, will check that the caught exception
+   *     is assignment-compatible with exceptionClass
+   * @param messagePattern - if non-null, will check that a substring of the 
+   *     message from the caught exception matches this pattern, via the
+   *     {@link Matcher#find()} method.
+   */
+  void startNameNodeShouldFail(StartupOption operation,
+      Class<? extends Exception> exceptionClass, Pattern messagePattern) {
     try {
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
                                                 .startupOption(operation)
                                                 .startupOption(operation)
@@ -137,9 +153,23 @@ public class TestDFSUpgrade {
                                                 .manageDataDfsDirs(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageNameDfsDirs(false)
                                                 .manageNameDfsDirs(false)
                                                 .build(); // should fail
                                                 .build(); // should fail
-      throw new AssertionError("NameNode should have failed to start");
-    } catch (Exception expected) {
-      // expected
+      fail("NameNode should have failed to start");
+      
+    } catch (Exception e) {
+      // expect exception
+      if (exceptionClass != null) {
+        assertTrue("Caught exception is not of expected class "
+            + exceptionClass.getSimpleName() + ": "
+            + StringUtils.stringifyException(e), 
+            exceptionClass.isInstance(e));
+      }
+      if (messagePattern != null) {
+        assertTrue("Caught exception message string does not match expected pattern \""
+            + messagePattern.pattern() + "\" : "
+            + StringUtils.stringifyException(e), 
+            messagePattern.matcher(e.getMessage()).find());
+      }
+      LOG.info("Successfully detected expected NameNode startup failure.");
     }
     }
   }
   }
   
   
@@ -169,6 +199,11 @@ public class TestDFSUpgrade {
                                            .build();
                                            .build();
   }
   }
   
   
+  @BeforeClass
+  public static void initialize() throws Exception {
+    UpgradeUtilities.initialize();
+  }
+  
   /**
   /**
    * This test attempts to upgrade the NameNode and DataNode under
    * This test attempts to upgrade the NameNode and DataNode under
    * a number of valid and invalid conditions.
    * a number of valid and invalid conditions.
@@ -176,8 +211,6 @@ public class TestDFSUpgrade {
   @Test
   @Test
   public void testUpgrade() throws Exception {
   public void testUpgrade() throws Exception {
     File[] baseDirs;
     File[] baseDirs;
-    UpgradeUtilities.initialize();
-    
     StorageInfo storageInfo = null;
     StorageInfo storageInfo = null;
     for (int numDirs = 1; numDirs <= 2; numDirs++) {
     for (int numDirs = 1; numDirs <= 2; numDirs++) {
       conf = new HdfsConfiguration();
       conf = new HdfsConfiguration();
@@ -320,6 +353,30 @@ public class TestDFSUpgrade {
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
     }
     }
   }
   }
+  
+  /*
+   * Stand-alone test to detect failure of one SD during parallel upgrade.
+   * At this time, can only be done with manual hack of {@link FSImage.doUpgrade()}
+   */
+  @Ignore
+  public void testUpgrade4() throws Exception {
+    int numDirs = 4;
+    conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);      
+    conf = UpgradeUtilities.initializeStorageStateConf(numDirs, conf);
+    String[] nameNodeDirs = conf.getStrings(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY);
+
+    log("NameNode upgrade with one bad storage dir", numDirs);
+    UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
+    try {
+      // assert("storage dir has been prepared for failure before reaching this point");
+      startNameNodeShouldFail(StartupOption.UPGRADE, IOException.class,
+          Pattern.compile("failed in 1 storage"));
+    } finally {
+      // assert("storage dir shall be returned to normal state before exiting");
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+    }
+  }
  
  
   private void deleteStorageFilesWithPrefix(String[] nameNodeDirs, String prefix)
   private void deleteStorageFilesWithPrefix(String[] nameNodeDirs, String prefix)
   throws Exception {
   throws Exception {
@@ -342,6 +399,7 @@ public class TestDFSUpgrade {
     fail("Expected IOException is not thrown");
     fail("Expected IOException is not thrown");
   }
   }
   
   
+  @Ignore
   public void test203LayoutVersion() {
   public void test203LayoutVersion() {
     for (int lv : Storage.LAYOUT_VERSIONS_203) {
     for (int lv : Storage.LAYOUT_VERSIONS_203) {
       assertTrue(Storage.is203LayoutVersion(lv));
       assertTrue(Storage.is203LayoutVersion(lv));
@@ -349,7 +407,9 @@ public class TestDFSUpgrade {
   }
   }
   
   
   public static void main(String[] args) throws Exception {
   public static void main(String[] args) throws Exception {
-    new TestDFSUpgrade().testUpgrade();
+    TestDFSUpgrade t = new TestDFSUpgrade();
+    TestDFSUpgrade.initialize();
+    t.testUpgrade();
   }
   }
 }
 }
 
 

+ 30 - 28
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java

@@ -72,7 +72,8 @@ public class TestDataTransferProtocol extends TestCase {
   DatanodeID datanode;
   DatanodeID datanode;
   InetSocketAddress dnAddr;
   InetSocketAddress dnAddr;
   ByteArrayOutputStream sendBuf = new ByteArrayOutputStream(128);
   ByteArrayOutputStream sendBuf = new ByteArrayOutputStream(128);
-  DataOutputStream sendOut = new DataOutputStream(sendBuf);
+  final DataOutputStream sendOut = new DataOutputStream(sendBuf);
+  final Sender sender = new Sender(sendOut);
   ByteArrayOutputStream recvBuf = new ByteArrayOutputStream(128);
   ByteArrayOutputStream recvBuf = new ByteArrayOutputStream(128);
   DataOutputStream recvOut = new DataOutputStream(recvBuf);
   DataOutputStream recvOut = new DataOutputStream(recvBuf);
 
 
@@ -185,9 +186,9 @@ public class TestDataTransferProtocol extends TestCase {
       String description, Boolean eofExcepted) throws IOException {
       String description, Boolean eofExcepted) throws IOException {
     sendBuf.reset();
     sendBuf.reset();
     recvBuf.reset();
     recvBuf.reset();
-    Sender.opWriteBlock(sendOut, block, 0,
-        stage, newGS, block.getNumBytes(), block.getNumBytes(), "cl", null,
-        new DatanodeInfo[1], BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.writeBlock(block, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        new DatanodeInfo[1], null, stage,
+        0, block.getNumBytes(), block.getNumBytes(), newGS);
     if (eofExcepted) {
     if (eofExcepted) {
       sendResponse(Status.ERROR, null, recvOut);
       sendResponse(Status.ERROR, null, recvOut);
       sendRecvData(description, true);
       sendRecvData(description, true);
@@ -372,10 +373,11 @@ public class TestDataTransferProtocol extends TestCase {
     
     
     /* Test OP_WRITE_BLOCK */
     /* Test OP_WRITE_BLOCK */
     sendBuf.reset();
     sendBuf.reset();
-    Sender.opWriteBlock(sendOut, 
-        new ExtendedBlock(poolId, newBlockId), 0,
-        BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
-        new DatanodeInfo[1], BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.writeBlock(new ExtendedBlock(poolId, newBlockId),
+        BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        new DatanodeInfo[1], null,
+        BlockConstructionStage.PIPELINE_SETUP_CREATE,
+        0, 0L, 0L, 0L);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     
     
     // bad bytes per checksum
     // bad bytes per checksum
@@ -386,10 +388,10 @@ public class TestDataTransferProtocol extends TestCase {
 
 
     sendBuf.reset();
     sendBuf.reset();
     recvBuf.reset();
     recvBuf.reset();
-    Sender.opWriteBlock(sendOut,
-        new ExtendedBlock(poolId, ++newBlockId), 0,
-        BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
-        new DatanodeInfo[1], BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.writeBlock(new ExtendedBlock(poolId, ++newBlockId),
+        BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        new DatanodeInfo[1], null,
+        BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0L, 0L, 0L);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     sendOut.writeInt(512);
     sendOut.writeInt(512);
 
 
@@ -409,10 +411,10 @@ public class TestDataTransferProtocol extends TestCase {
     // test for writing a valid zero size block
     // test for writing a valid zero size block
     sendBuf.reset();
     sendBuf.reset();
     recvBuf.reset();
     recvBuf.reset();
-    Sender.opWriteBlock(sendOut, 
-        new ExtendedBlock(poolId, ++newBlockId), 0,
-        BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
-        new DatanodeInfo[1], BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.writeBlock(new ExtendedBlock(poolId, ++newBlockId),
+        BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        new DatanodeInfo[1], null,
+        BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0L, 0L, 0L);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     sendOut.writeInt(512);         // checksum size
     sendOut.writeInt(512);         // checksum size
 
 
@@ -439,22 +441,22 @@ public class TestDataTransferProtocol extends TestCase {
     sendBuf.reset();
     sendBuf.reset();
     recvBuf.reset();
     recvBuf.reset();
     blk.setBlockId(blkid-1);
     blk.setBlockId(blkid-1);
-    Sender.opReadBlock(sendOut, blk, 0L, fileLen, "cl",
-          BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        0L, fileLen);
     sendRecvData("Wrong block ID " + newBlockId + " for read", false); 
     sendRecvData("Wrong block ID " + newBlockId + " for read", false); 
 
 
     // negative block start offset -1L
     // negative block start offset -1L
     sendBuf.reset();
     sendBuf.reset();
     blk.setBlockId(blkid);
     blk.setBlockId(blkid);
-    Sender.opReadBlock(sendOut, blk, -1L, fileLen, "cl",
-          BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        -1L, fileLen);
     sendRecvData("Negative start-offset for read for block " + 
     sendRecvData("Negative start-offset for read for block " + 
                  firstBlock.getBlockId(), false);
                  firstBlock.getBlockId(), false);
 
 
     // bad block start offset
     // bad block start offset
     sendBuf.reset();
     sendBuf.reset();
-    Sender.opReadBlock(sendOut, blk, fileLen, fileLen, "cl",
-          BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        fileLen, fileLen);
     sendRecvData("Wrong start-offset for reading block " +
     sendRecvData("Wrong start-offset for reading block " +
                  firstBlock.getBlockId(), false);
                  firstBlock.getBlockId(), false);
     
     
@@ -462,8 +464,8 @@ public class TestDataTransferProtocol extends TestCase {
     recvBuf.reset();
     recvBuf.reset();
     sendResponse(Status.SUCCESS, null, recvOut);
     sendResponse(Status.SUCCESS, null, recvOut);
     sendBuf.reset();
     sendBuf.reset();
-    Sender.opReadBlock(sendOut, blk, 0L, 
-        -1 - random.nextInt(oneMil), "cl", BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        0L, -1L-random.nextInt(oneMil));
     sendRecvData("Negative length for reading block " +
     sendRecvData("Negative length for reading block " +
                  firstBlock.getBlockId(), false);
                  firstBlock.getBlockId(), false);
     
     
@@ -471,15 +473,15 @@ public class TestDataTransferProtocol extends TestCase {
     recvBuf.reset();
     recvBuf.reset();
     sendResponse(Status.ERROR, null, recvOut);
     sendResponse(Status.ERROR, null, recvOut);
     sendBuf.reset();
     sendBuf.reset();
-    Sender.opReadBlock(sendOut, blk, 0L, 
-        fileLen + 1, "cl", BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        0L, fileLen+1);
     sendRecvData("Wrong length for reading block " +
     sendRecvData("Wrong length for reading block " +
                  firstBlock.getBlockId(), false);
                  firstBlock.getBlockId(), false);
     
     
     //At the end of all this, read the file to make sure that succeeds finally.
     //At the end of all this, read the file to make sure that succeeds finally.
     sendBuf.reset();
     sendBuf.reset();
-    Sender.opReadBlock(sendOut, blk, 0L, 
-        fileLen, "cl", BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        0L, fileLen);
     readFile(fileSys, file, fileLen);
     readFile(fileSys, file, fileLen);
     } finally {
     } finally {
       cluster.shutdown();
       cluster.shutdown();

+ 68 - 1
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestQuota.java

@@ -54,6 +54,23 @@ public class TestQuota {
     }
     }
   }
   }
   
   
+  /**
+   * Tests to make sure we're getting human readable Quota exception messages
+   * Test for @link{ NSQuotaExceededException, DSQuotaExceededException}
+   * @throws Exception
+   */
+  @Test
+  public void testDSQuotaExceededExceptionIsHumanReadable() throws Exception {
+    Integer bytes = 1024;
+    try {
+      throw new DSQuotaExceededException(bytes, bytes);
+    } catch(DSQuotaExceededException e) {
+      
+      assertEquals("The DiskSpace quota is exceeded: quota=1.0k " +
+          "diskspace consumed=1.0k", e.getMessage());
+    }
+  }
+  
   /** Test quota related commands: 
   /** Test quota related commands: 
    *    setQuota, clrQuota, setSpaceQuota, clrSpaceQuota, and count 
    *    setQuota, clrQuota, setSpaceQuota, clrSpaceQuota, and count 
    */
    */
@@ -695,7 +712,57 @@ public class TestQuota {
       // verify increase in space
       // verify increase in space
       c = dfs.getContentSummary(dstPath);
       c = dfs.getContentSummary(dstPath);
       assertEquals(c.getSpaceConsumed(), 5 * fileSpace + file2Len);
       assertEquals(c.getSpaceConsumed(), 5 * fileSpace + file2Len);
-      
+
+      // Test HDFS-2053 :
+
+      // Create directory /hdfs-2053
+      final Path quotaDir2053 = new Path("/hdfs-2053");
+      assertTrue(dfs.mkdirs(quotaDir2053));
+
+      // Create subdirectories /hdfs-2053/{A,B,C}
+      final Path quotaDir2053_A = new Path(quotaDir2053, "A");
+      assertTrue(dfs.mkdirs(quotaDir2053_A));
+      final Path quotaDir2053_B = new Path(quotaDir2053, "B");
+      assertTrue(dfs.mkdirs(quotaDir2053_B));
+      final Path quotaDir2053_C = new Path(quotaDir2053, "C");
+      assertTrue(dfs.mkdirs(quotaDir2053_C));
+
+      // Factors to vary the sizes of test files created in each subdir.
+      // The actual factors are not really important but they allow us to create
+      // identifiable file sizes per subdir, which helps during debugging.
+      int sizeFactorA = 1;
+      int sizeFactorB = 2;
+      int sizeFactorC = 4;
+
+      // Set space quota for subdirectory C
+      dfs.setQuota(quotaDir2053_C, FSConstants.QUOTA_DONT_SET,
+          (sizeFactorC + 1) * fileSpace);
+      c = dfs.getContentSummary(quotaDir2053_C);
+      assertEquals(c.getSpaceQuota(), (sizeFactorC + 1) * fileSpace);
+
+      // Create a file under subdirectory A
+      DFSTestUtil.createFile(dfs, new Path(quotaDir2053_A, "fileA"),
+          sizeFactorA * fileLen, replication, 0);
+      c = dfs.getContentSummary(quotaDir2053_A);
+      assertEquals(c.getSpaceConsumed(), sizeFactorA * fileSpace);
+
+      // Create a file under subdirectory B
+      DFSTestUtil.createFile(dfs, new Path(quotaDir2053_B, "fileB"),
+          sizeFactorB * fileLen, replication, 0);
+      c = dfs.getContentSummary(quotaDir2053_B);
+      assertEquals(c.getSpaceConsumed(), sizeFactorB * fileSpace);
+
+      // Create a file under subdirectory C (which has a space quota)
+      DFSTestUtil.createFile(dfs, new Path(quotaDir2053_C, "fileC"),
+          sizeFactorC * fileLen, replication, 0);
+      c = dfs.getContentSummary(quotaDir2053_C);
+      assertEquals(c.getSpaceConsumed(), sizeFactorC * fileSpace);
+
+      // Check space consumed for /hdfs-2053
+      c = dfs.getContentSummary(quotaDir2053);
+      assertEquals(c.getSpaceConsumed(),
+          (sizeFactorA + sizeFactorB + sizeFactorC) * fileSpace);
+
     } finally {
     } finally {
       cluster.shutdown();
       cluster.shutdown();
     }
     }

+ 9 - 5
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/UpgradeUtilities.java

@@ -183,8 +183,9 @@ public class UpgradeUtilities {
   }
   }
   
   
   /**
   /**
-   * Initialize dfs.namenode.name.dir and dfs.datanode.data.dir with the specified number of
-   * directory entries. Also initialize dfs.blockreport.intervalMsec.
+   * Initialize {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
+   * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} with the specified 
+   * number of directory entries. Also initialize dfs.blockreport.intervalMsec.
    */
    */
   public static Configuration initializeStorageStateConf(int numDirs,
   public static Configuration initializeStorageStateConf(int numDirs,
                                                          Configuration conf) {
                                                          Configuration conf) {
@@ -308,7 +309,8 @@ public class UpgradeUtilities {
   }
   }
   
   
   /**
   /**
-   * Simulate the <code>dfs.namenode.name.dir</code> of a populated DFS filesystem.
+   * Simulate the {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} of a populated 
+   * DFS filesystem.
    * This method populates for each parent directory, <code>parent/dirName</code>
    * This method populates for each parent directory, <code>parent/dirName</code>
    * with the content of namenode storage directory that comes from a singleton
    * with the content of namenode storage directory that comes from a singleton
    * namenode master (that contains edits, fsimage, version and time files). 
    * namenode master (that contains edits, fsimage, version and time files). 
@@ -335,7 +337,8 @@ public class UpgradeUtilities {
   }  
   }  
   
   
   /**
   /**
-   * Simulate the <code>dfs.datanode.data.dir</code> of a populated DFS filesystem.
+   * Simulate the {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} of a 
+   * populated DFS filesystem.
    * This method populates for each parent directory, <code>parent/dirName</code>
    * This method populates for each parent directory, <code>parent/dirName</code>
    * with the content of datanode storage directory that comes from a singleton
    * with the content of datanode storage directory that comes from a singleton
    * datanode master (that contains version and block files). If the destination
    * datanode master (that contains version and block files). If the destination
@@ -362,7 +365,8 @@ public class UpgradeUtilities {
   }
   }
   
   
   /**
   /**
-   * Simulate the <code>dfs.datanode.data.dir</code> of a populated DFS filesystem.
+   * Simulate the {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} of a 
+   * populated DFS filesystem.
    * This method populates for each parent directory, <code>parent/dirName</code>
    * This method populates for each parent directory, <code>parent/dirName</code>
    * with the content of block pool storage directory that comes from a singleton
    * with the content of block pool storage directory that comes from a singleton
    * datanode master (that contains version and block files). If the destination
    * datanode master (that contains version and block files). If the destination

+ 8 - 6
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCorruptReplicaInfo.java → hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java

@@ -15,17 +15,20 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.util.*;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
 import junit.framework.TestCase;
 import junit.framework.TestCase;
+
 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.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.CorruptReplicasMap;
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 
 
 
 
 /**
 /**
@@ -79,7 +82,6 @@ public class TestCorruptReplicaInfo extends TestCase {
       
       
       DatanodeDescriptor dn1 = new DatanodeDescriptor();
       DatanodeDescriptor dn1 = new DatanodeDescriptor();
       DatanodeDescriptor dn2 = new DatanodeDescriptor();
       DatanodeDescriptor dn2 = new DatanodeDescriptor();
-      DatanodeDescriptor dn3 = new DatanodeDescriptor();
       
       
       crm.addToCorruptReplicasMap(getBlock(0), dn1);
       crm.addToCorruptReplicasMap(getBlock(0), dn1);
       assertEquals("Number of corrupt blocks not returning correctly",
       assertEquals("Number of corrupt blocks not returning correctly",

+ 1 - 1
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDatanodeDescriptor.java → hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 
 

+ 2 - 1
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java → hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;

+ 1 - 1
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestPendingReplication.java → hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 import java.lang.System;
 import java.lang.System;

+ 9 - 9
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java → hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java

@@ -15,8 +15,7 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -24,18 +23,19 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 
 
+import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.net.Node;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
 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.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-
-import junit.framework.TestCase;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.net.Node;
 
 
 public class TestReplicationPolicy extends TestCase {
 public class TestReplicationPolicy extends TestCase {
   private static final int BLOCK_SIZE = 1024;
   private static final int BLOCK_SIZE = 1024;

+ 4 - 3
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestUnderReplicatedBlocks.java → hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlocks.java

@@ -15,7 +15,9 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import junit.framework.TestCase;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
@@ -25,8 +27,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 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.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-
-import junit.framework.TestCase;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 
 
 public class TestUnderReplicatedBlocks extends TestCase {
 public class TestUnderReplicatedBlocks extends TestCase {
   public void testSetrepIncWithUnderReplicatedBlocks() throws Exception {
   public void testSetrepIncWithUnderReplicatedBlocks() throws Exception {

+ 2 - 2
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java

@@ -258,8 +258,8 @@ public class TestBlockReplacement extends TestCase {
     sock.setKeepAlive(true);
     sock.setKeepAlive(true);
     // sendRequest
     // sendRequest
     DataOutputStream out = new DataOutputStream(sock.getOutputStream());
     DataOutputStream out = new DataOutputStream(sock.getOutputStream());
-    Sender.opReplaceBlock(out, block, source
-        .getStorageID(), sourceProxy, BlockTokenSecretManager.DUMMY_TOKEN);
+    new Sender(out).replaceBlock(block, BlockTokenSecretManager.DUMMY_TOKEN,
+        source.getStorageID(), sourceProxy);
     out.flush();
     out.flush();
     // receiveResponse
     // receiveResponse
     DataInputStream reply = new DataInputStream(sock.getInputStream());
     DataInputStream reply = new DataInputStream(sock.getInputStream());

+ 12 - 11
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java

@@ -17,30 +17,31 @@
  */
  */
 package org.apache.hadoop.hdfs.server.datanode;
 package org.apache.hadoop.hdfs.server.datanode;
 
 
+import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeTrue;
+
 import java.io.File;
 import java.io.File;
 import java.util.ArrayList;
 import java.util.ArrayList;
 
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+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.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import static org.apache.hadoop.test.MetricsAsserts.*;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
-
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
-import static org.junit.Assert.*;
-import static org.junit.Assume.assumeTrue;
 
 
 /**
 /**
  * Test reporting of DN volume failure counts and metrics.
  * Test reporting of DN volume failure counts and metrics.

+ 11 - 18
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java

@@ -133,17 +133,17 @@ public class TestDataNodeVolumeFailureToleration {
   }
   }
 
 
   /** 
   /** 
-   * Restart the cluster with a new volume tolerated value.
-   * @param volTolerated
-   * @param manageCluster
+   * Restart the datanodes with a new volume tolerated value.
+   * @param volTolerated number of dfs data dir failures to tolerate
+   * @param manageDfsDirs whether the mini cluster should manage data dirs
    * @throws IOException
    * @throws IOException
    */
    */
-  private void restartCluster(int volTolerated, boolean manageCluster)
+  private void restartDatanodes(int volTolerated, boolean manageDfsDirs)
       throws IOException {
       throws IOException {
     //Make sure no datanode is running
     //Make sure no datanode is running
     cluster.shutdownDataNodes();
     cluster.shutdownDataNodes();
     conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, volTolerated);
     conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, volTolerated);
-    cluster.startDataNodes(conf, 1, manageCluster, null, null);
+    cluster.startDataNodes(conf, 1, manageDfsDirs, null, null);
     cluster.waitActive();
     cluster.waitActive();
   }
   }
 
 
@@ -174,19 +174,14 @@ public class TestDataNodeVolumeFailureToleration {
 
 
   /**
   /**
    * Tests for a given volumes to be tolerated and volumes failed.
    * Tests for a given volumes to be tolerated and volumes failed.
-   * 
-   * @param volumesTolerated
-   * @param volumesFailed
-   * @param expectedBPServiceState
-   * @param clusterManaged
-   * @throws IOException
-   * @throws InterruptedException
    */
    */
   private void testVolumeConfig(int volumesTolerated, int volumesFailed,
   private void testVolumeConfig(int volumesTolerated, int volumesFailed,
-      boolean expectedBPServiceState, boolean clusterManaged)
+      boolean expectedBPServiceState, boolean manageDfsDirs)
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
     assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
     assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
     final int dnIndex = 0;
     final int dnIndex = 0;
+    // Fail the current directory since invalid storage directory perms
+    // get fixed up automatically on datanode startup.
     File[] dirs = {
     File[] dirs = {
         new File(MiniDFSCluster.getStorageDir(dnIndex, 0), "current"),
         new File(MiniDFSCluster.getStorageDir(dnIndex, 0), "current"),
         new File(MiniDFSCluster.getStorageDir(dnIndex, 1), "current") };
         new File(MiniDFSCluster.getStorageDir(dnIndex, 1), "current") };
@@ -195,11 +190,10 @@ public class TestDataNodeVolumeFailureToleration {
       for (int i = 0; i < volumesFailed; i++) {
       for (int i = 0; i < volumesFailed; i++) {
         prepareDirToFail(dirs[i]);
         prepareDirToFail(dirs[i]);
       }
       }
-      restartCluster(volumesTolerated, clusterManaged);
+      restartDatanodes(volumesTolerated, manageDfsDirs);
       assertEquals(expectedBPServiceState, cluster.getDataNodes().get(0)
       assertEquals(expectedBPServiceState, cluster.getDataNodes().get(0)
           .isBPServiceAlive(cluster.getNamesystem().getBlockPoolId()));
           .isBPServiceAlive(cluster.getNamesystem().getBlockPoolId()));
     } finally {
     } finally {
-      // restore its old permission
       for (File dir : dirs) {
       for (File dir : dirs) {
         FileUtil.chmod(dir.toString(), "755");
         FileUtil.chmod(dir.toString(), "755");
       }
       }
@@ -215,8 +209,7 @@ public class TestDataNodeVolumeFailureToleration {
   private void prepareDirToFail(File dir) throws IOException,
   private void prepareDirToFail(File dir) throws IOException,
       InterruptedException {
       InterruptedException {
     dir.mkdirs();
     dir.mkdirs();
-    assertTrue("Couldn't chmod local vol", FileUtil
-        .chmod(dir.toString(), "000") == 0);
+    assertEquals("Couldn't chmod local vol", 0,
+        FileUtil.chmod(dir.toString(), "000"));
   }
   }
-
 }
 }

+ 4 - 4
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java

@@ -140,10 +140,10 @@ public class TestDiskError {
     // write the header.
     // write the header.
     DataOutputStream out = new DataOutputStream(s.getOutputStream());
     DataOutputStream out = new DataOutputStream(s.getOutputStream());
 
 
-    Sender.opWriteBlock(out, block.getBlock(), 1,
-        BlockConstructionStage.PIPELINE_SETUP_CREATE,
-        0L, 0L, 0L, "", null, new DatanodeInfo[0],
-        BlockTokenSecretManager.DUMMY_TOKEN);
+    new Sender(out).writeBlock(block.getBlock(),
+        BlockTokenSecretManager.DUMMY_TOKEN, "",
+        new DatanodeInfo[0], null,
+        BlockConstructionStage.PIPELINE_SETUP_CREATE, 1, 0L, 0L, 0L);
 
 
     // write check header
     // write check header
     out.writeByte( 1 );
     out.writeByte( 1 );

+ 1 - 0
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration;
 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.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 
 

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

@@ -19,10 +19,11 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
-import org.apache.hadoop.ipc.Server;
 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;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.ipc.Server;
 
 
 /**
 /**
  * This is a utility class to expose NameNode functionality for unit tests.
  * This is a utility class to expose NameNode functionality for unit tests.

+ 2 - 0
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java

@@ -34,6 +34,8 @@ import org.apache.hadoop.hdfs.TestFileCreation;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;

+ 66 - 0
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -23,6 +23,7 @@ import java.net.InetSocketAddress;
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.List;
 import java.util.List;
 import java.util.Random;
 import java.util.Random;
@@ -173,6 +174,71 @@ public class TestCheckpoint extends TestCase {
     resurrectNameDir(first); // put back namedir
     resurrectNameDir(first); // put back namedir
   }
   }
 
 
+  /**
+   * Tests EditLogFileOutputStream doesn't throw NullPointerException on being
+   * closed twice.
+   * See https://issues.apache.org/jira/browse/HDFS-2011
+   */
+  public void testEditLogFileOutputStreamCloses()
+    throws IOException,NullPointerException {
+    System.out.println("Testing EditLogFileOutputStream doesn't throw " +
+                       "NullPointerException on being closed twice");
+    File editLogStreamFile = null;
+    try {
+      editLogStreamFile = new File(System.getProperty("test.build.data","/tmp"),
+                                   "editLogStream.dat");
+      EditLogFileOutputStream editLogStream =
+                             new EditLogFileOutputStream(editLogStreamFile, 0);
+      editLogStream.close();
+      //Closing an twice should not throw a NullPointerException
+      editLogStream.close();
+    } finally {
+      if (editLogStreamFile != null)
+        // Cleanup the editLogStream.dat file we created
+          editLogStreamFile.delete();
+    }
+    System.out.println("Successfully tested EditLogFileOutputStream doesn't " +
+           "throw NullPointerException on being closed twice");
+  }
+
+  /**
+   * Checks that an IOException in NNStorage.writeTransactionIdFile is handled
+   * correctly (by removing the storage directory)
+   * See https://issues.apache.org/jira/browse/HDFS-2011
+   */
+  public void testWriteTransactionIdHandlesIOE() throws Exception {
+    System.out.println("Check IOException handled correctly by writeTransactionIdFile");
+    ArrayList<URI> fsImageDirs = new ArrayList<URI>();
+    ArrayList<URI> editsDirs = new ArrayList<URI>();
+    File filePath =
+      new File(System.getProperty("test.build.data","/tmp"), "storageDirToCheck");
+    assertTrue("Couldn't create directory storageDirToCheck",
+               filePath.exists() || filePath.mkdirs());
+    fsImageDirs.add(filePath.toURI());
+    editsDirs.add(filePath.toURI());
+    NNStorage nnStorage = new NNStorage(new HdfsConfiguration(),
+      fsImageDirs, editsDirs);
+    try {
+      assertTrue("List of storage directories didn't have storageDirToCheck.",
+                 nnStorage.getEditsDirectories().iterator().next().
+                 toString().indexOf("storageDirToCheck") != -1);
+      assertTrue("List of removed storage directories wasn't empty",
+                 nnStorage.getRemovedStorageDirs().isEmpty());
+    } finally {
+      // Delete storage directory to cause IOException in writeTransactionIdFile 
+      assertTrue("Couldn't remove directory " + filePath.getAbsolutePath(),
+                 filePath.delete());
+    }
+    // Just call writeTransactionIdFile using any random number
+    nnStorage.writeTransactionIdFileToStorage(1);
+    List<StorageDirectory> listRsd = nnStorage.getRemovedStorageDirs();
+    assertTrue("Removed directory wasn't what was expected",
+               listRsd.size() > 0 && listRsd.get(listRsd.size() - 1).getRoot().
+               toString().indexOf("storageDirToCheck") != -1);
+    System.out.println("Successfully checked IOException is handled correctly "
+                       + "by writeTransactionIdFile");
+  }
+
   /*
   /*
    * Simulate namenode crashing after rolling edit log.
    * Simulate namenode crashing after rolling edit log.
    */
    */

+ 3 - 2
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestComputeInvalidateWork.java

@@ -17,14 +17,15 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 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.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 
 
-import junit.framework.TestCase;
-
 /**
 /**
  * Test if FSNamesystem handles heartbeat right
  * Test if FSNamesystem handles heartbeat right
  */
  */

+ 1 - 1
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java

@@ -29,9 +29,9 @@ 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.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;

+ 7 - 9
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java

@@ -17,19 +17,15 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.Random;
 import java.util.Random;
 
 
-import org.junit.BeforeClass;
-import org.junit.AfterClass;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
@@ -40,8 +36,10 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
 
 /**
 /**
  * This class tests the decommissioning of nodes.
  * This class tests the decommissioning of nodes.

+ 3 - 2
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHost2NodesMap.java

@@ -18,10 +18,11 @@
 
 
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+
 public class TestHost2NodesMap extends TestCase {
 public class TestHost2NodesMap extends TestCase {
   static private Host2NodesMap map = new Host2NodesMap();
   static private Host2NodesMap map = new Host2NodesMap();
   private final static DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] {
   private final static DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] {

+ 6 - 6
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java

@@ -21,17 +21,17 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.File;
 import java.io.File;
 import java.util.ArrayList;
 import java.util.ArrayList;
 
 
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.DF;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+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;
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import junit.framework.TestCase;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 
 
 
 
 
 

+ 4 - 3
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java

@@ -21,6 +21,8 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.TimeoutException;
 
 
+import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -30,9 +32,8 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
-
-import junit.framework.TestCase;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas;
 
 
 /**
 /**
  * Test if live nodes count per node is correct 
  * Test if live nodes count per node is correct 

+ 5 - 4
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java

@@ -20,20 +20,21 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 
 
+import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 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.TestDatanodeBlockScanner;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.TestDatanodeBlockScanner;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-
-import junit.framework.TestCase;
 
 
 public class TestOverReplicatedBlocks extends TestCase {
 public class TestOverReplicatedBlocks extends TestCase {
   /** Test processOverReplicatedBlock can handle corrupt replicas fine.
   /** Test processOverReplicatedBlock can handle corrupt replicas fine.

+ 59 - 0
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestSecondaryWebUi.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 java.io.IOException;
+import java.net.URL;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.Test;
+
+public class TestSecondaryWebUi {
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testSecondaryWebUi() throws IOException {
+    Configuration conf = new Configuration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
+        "0.0.0.0:0");
+    MiniDFSCluster cluster = null;
+    SecondaryNameNode snn = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+          .build();
+      cluster.waitActive();
+      
+      snn = new SecondaryNameNode(conf);
+      String pageContents = DFSTestUtil.urlGet(new URL("http://localhost:" +
+          SecondaryNameNode.getHttpAddress(conf).getPort() + "/status.jsp"));
+      assertTrue(pageContents.contains("Last Checkpoint Time"));
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      if (snn != null) {
+        snn.shutdown();
+      }
+    }
+  }
+}

+ 91 - 1
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java

@@ -25,7 +25,9 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.io.RandomAccessFile;
+import java.net.InetAddress;
 import java.net.URI;
 import java.net.URI;
+import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 import java.util.Properties;
 import java.util.Properties;
@@ -44,10 +46,13 @@ import org.apache.hadoop.fs.RawLocalFileSystem;
 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.hdfs.DFSClient;
 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.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -55,8 +60,8 @@ import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.MD5Hash;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.junit.Assert;
 
 
 /**
 /**
  * Startup and checkpoint tests
  * Startup and checkpoint tests
@@ -478,4 +483,89 @@ public class TestStartup extends TestCase {
       }
       }
     }
     }
   }
   }
+  
+  /**
+   * This test tests hosts include list contains host names.  After namenode
+   * restarts, the still alive datanodes should not have any trouble in getting
+   * registrant again.
+   */
+  public void testNNRestart() throws IOException, InterruptedException {
+    MiniDFSCluster cluster = null;
+    FileSystem localFileSys;
+    Path hostsFile;
+    Path excludeFile;
+    Configuration conf = new HdfsConfiguration();
+    int HEARTBEAT_INTERVAL = 1; // heartbeat interval in seconds
+    // Set up the hosts/exclude files.
+    localFileSys = FileSystem.getLocal(conf);
+    Path workingDir = localFileSys.getWorkingDirectory();
+    Path dir = new Path(workingDir, "build/test/data/work-dir/restartnn");
+    hostsFile = new Path(dir, "hosts");
+    excludeFile = new Path(dir, "exclude");
+
+    // Setup conf
+    conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
+    writeConfigFile(localFileSys, excludeFile, null);    
+    conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
+    // write into hosts file
+    ArrayList<String>list = new ArrayList<String>();
+    byte b[] = {127, 0, 0, 1};
+    InetAddress inetAddress = InetAddress.getByAddress(b);
+    list.add(inetAddress.getHostName());
+    writeConfigFile(localFileSys, hostsFile, list);
+    int numNameNodes = 1;
+    int numDatanodes = 1;
+    
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numNameNodes(numNameNodes)
+      .numDataNodes(numDatanodes).setupHostsFile(true).build();
+      cluster.waitActive();
+  
+      cluster.restartNameNode();
+      NameNode nn = cluster.getNameNode();
+      assertNotNull(nn);
+      Assert.assertTrue(cluster.isDataNodeUp());
+      
+      DatanodeInfo[] info = nn.getDatanodeReport(DatanodeReportType.LIVE);
+      for (int i = 0 ; i < 5 && info.length != numDatanodes; i++) {
+        Thread.sleep(HEARTBEAT_INTERVAL * 1000);
+        info = nn.getDatanodeReport(DatanodeReportType.LIVE);
+      }
+      assertEquals("Number of live nodes should be "+numDatanodes, numDatanodes, 
+          info.length);
+      
+    } catch (IOException e) {
+      fail(StringUtils.stringifyException(e));
+      throw e;
+    } finally {
+      cleanupFile(localFileSys, excludeFile.getParent());
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+  
+  private void writeConfigFile(FileSystem localFileSys, Path name,
+      ArrayList<String> nodes) throws IOException {
+    // delete if it already exists
+    if (localFileSys.exists(name)) {
+      localFileSys.delete(name, true);
+    }
+
+    if (nodes != null) {
+      FSDataOutputStream stm = localFileSys.create(name);
+      for (Iterator<String> it = nodes.iterator(); it.hasNext();) {
+        String node = it.next();
+        stm.writeBytes(node);
+        stm.writeBytes("\n");
+      }
+      stm.close();
+    }
+  }
+  
+  private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
+    assertTrue(fileSys.exists(name));
+    fileSys.delete(name, true);
+    assertTrue(!fileSys.exists(name));
+  }
 }
 }

+ 86 - 11
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java

@@ -18,6 +18,10 @@
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
 
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.EOFException;
 import java.io.EOFException;
 import java.io.File;
 import java.io.File;
@@ -28,22 +32,23 @@ import java.security.MessageDigest;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.Random;
 import java.util.Random;
+import java.util.Set;
 
 
 import static org.mockito.Matchers.anyByte;
 import static org.mockito.Matchers.anyByte;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.spy;
 
 
-import junit.framework.TestCase;
-
 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.cli.CLITestCmdDFS;
 import org.apache.hadoop.cli.CLITestCmdDFS;
-import org.apache.hadoop.cli.util.*;
+import org.apache.hadoop.cli.util.CLICommandDFSAdmin;
+import org.apache.hadoop.cli.util.CommandExecutor;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+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.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -55,12 +60,16 @@ import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEdits
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
 
+import com.google.common.collect.ImmutableSet;
 /**
 /**
  * Startup and checkpoint tests
  * Startup and checkpoint tests
  * 
  * 
  */
  */
-public class TestStorageRestore extends TestCase {
+public class TestStorageRestore {
   public static final String NAME_NODE_HOST = "localhost:";
   public static final String NAME_NODE_HOST = "localhost:";
   public static final String NAME_NODE_HTTP_HOST = "0.0.0.0:";
   public static final String NAME_NODE_HTTP_HOST = "0.0.0.0:";
   private static final Log LOG =
   private static final Log LOG =
@@ -72,8 +81,8 @@ public class TestStorageRestore extends TestCase {
   static final int fileSize = 8192;
   static final int fileSize = 8192;
   private File path1, path2, path3;
   private File path1, path2, path3;
   private MiniDFSCluster cluster;  
   private MiniDFSCluster cluster;  
- 
-  protected void setUp() throws Exception {
+  @Before
+  public void setUpNameDirs() throws Exception {
     config = new HdfsConfiguration();
     config = new HdfsConfiguration();
     hdfsDir = new File(MiniDFSCluster.getBaseDirectory()).getCanonicalFile();
     hdfsDir = new File(MiniDFSCluster.getBaseDirectory()).getCanonicalFile();
     if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
     if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
@@ -110,12 +119,13 @@ public class TestStorageRestore extends TestCase {
   /**
   /**
    * invalidate storage by removing the second and third storage directories
    * invalidate storage by removing the second and third storage directories
    */
    */
-  public void invalidateStorage(FSImage fi) throws IOException {
+  public void invalidateStorage(FSImage fi, Set<File> filesToInvalidate) throws IOException {
     ArrayList<StorageDirectory> al = new ArrayList<StorageDirectory>(2);
     ArrayList<StorageDirectory> al = new ArrayList<StorageDirectory>(2);
     Iterator<StorageDirectory> it = fi.getStorage().dirIterator();
     Iterator<StorageDirectory> it = fi.getStorage().dirIterator();
     while(it.hasNext()) {
     while(it.hasNext()) {
       StorageDirectory sd = it.next();
       StorageDirectory sd = it.next();
-      if(sd.getRoot().equals(path2) || sd.getRoot().equals(path3)) {
+      if(filesToInvalidate.contains(sd.getRoot())) {
+        LOG.info("causing IO error on " + sd.getRoot());
         al.add(sd);
         al.add(sd);
       }
       }
     }
     }
@@ -140,7 +150,7 @@ public class TestStorageRestore extends TestCase {
    * test
    * test
    */
    */
   public void printStorages(FSImage fs) {
   public void printStorages(FSImage fs) {
-    LOG.info("current storages and corresoponding sizes:");
+    LOG.info("current storages and corresponding sizes:");
     for(Iterator<StorageDirectory> it = fs.getStorage().dirIterator(); it.hasNext(); ) {
     for(Iterator<StorageDirectory> it = fs.getStorage().dirIterator(); it.hasNext(); ) {
       StorageDirectory sd = it.next();
       StorageDirectory sd = it.next();
 
 
@@ -164,6 +174,7 @@ public class TestStorageRestore extends TestCase {
    * 8. verify that all the image and edits files are the same.
    * 8. verify that all the image and edits files are the same.
    */
    */
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
+  @Test
   public void testStorageRestore() throws Exception {
   public void testStorageRestore() throws Exception {
     int numDatanodes = 0;
     int numDatanodes = 0;
     cluster = new MiniDFSCluster.Builder(config).numDataNodes(numDatanodes)
     cluster = new MiniDFSCluster.Builder(config).numDataNodes(numDatanodes)
@@ -181,7 +192,7 @@ public class TestStorageRestore extends TestCase {
     
     
     System.out.println("****testStorageRestore: dir 'test' created, invalidating storage...");
     System.out.println("****testStorageRestore: dir 'test' created, invalidating storage...");
   
   
-    invalidateStorage(cluster.getNameNode().getFSImage());
+    invalidateStorage(cluster.getNameNode().getFSImage(), ImmutableSet.of(path2, path3));
     printStorages(cluster.getNameNode().getFSImage());
     printStorages(cluster.getNameNode().getFSImage());
     System.out.println("****testStorageRestore: storage invalidated");
     System.out.println("****testStorageRestore: storage invalidated");
 
 
@@ -265,6 +276,7 @@ public class TestStorageRestore extends TestCase {
    * Test dfsadmin -restoreFailedStorage command
    * Test dfsadmin -restoreFailedStorage command
    * @throws Exception
    * @throws Exception
    */
    */
+  @Test
   public void testDfsAdminCmd() throws Exception {
   public void testDfsAdminCmd() throws Exception {
     cluster = new MiniDFSCluster.Builder(config).
     cluster = new MiniDFSCluster.Builder(config).
                                  numDataNodes(2).
                                  numDataNodes(2).
@@ -296,7 +308,7 @@ public class TestStorageRestore extends TestCase {
       restore = fsi.getStorage().getRestoreFailedStorage();
       restore = fsi.getStorage().getRestoreFailedStorage();
       assertTrue("After set false call restore is " + restore, restore);
       assertTrue("After set false call restore is " + restore, restore);
       
       
-   // run one more time - no change in value
+      // run one more time - no change in value
       cmd = "-fs NAMENODE -restoreFailedStorage check";
       cmd = "-fs NAMENODE -restoreFailedStorage check";
       CommandExecutor.Result cmdResult = executor.executeCommand(cmd);
       CommandExecutor.Result cmdResult = executor.executeCommand(cmd);
       restore = fsi.getStorage().getRestoreFailedStorage();
       restore = fsi.getStorage().getRestoreFailedStorage();
@@ -310,4 +322,67 @@ public class TestStorageRestore extends TestCase {
       cluster.shutdown();
       cluster.shutdown();
     }
     }
   }
   }
+
+  /**
+   * Test to simulate interleaved checkpointing by 2 2NNs after a storage
+   * directory has been taken offline. The first will cause the directory to
+   * come back online, but it won't have any valid contents. The second 2NN will
+   * then try to perform a checkpoint. The NN should not serve up the image or
+   * edits from the restored (empty) dir.
+   */
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testMultipleSecondaryCheckpoint() throws IOException {
+    
+    SecondaryNameNode secondary = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(config).numDataNodes(1)
+          .manageNameDfsDirs(false).build();
+      cluster.waitActive();
+      
+      secondary = new SecondaryNameNode(config);
+  
+      FSImage fsImage = cluster.getNameNode().getFSImage();
+      printStorages(fsImage);
+      
+      FileSystem fs = cluster.getFileSystem();
+      Path testPath = new Path("/", "test");
+      assertTrue(fs.mkdirs(testPath));
+      
+      printStorages(fsImage);
+  
+      // Take name1 offline
+      invalidateStorage(fsImage, ImmutableSet.of(path1));
+      
+      // Simulate a 2NN beginning a checkpoint, but not finishing. This will
+      // cause name1 to be restored.
+      cluster.getNameNode().rollEditLog();
+      
+      printStorages(fsImage);
+      
+      // Now another 2NN comes along to do a full checkpoint.
+      secondary.doCheckpoint();
+      
+      printStorages(fsImage);
+      
+      // The created file should still exist in the in-memory FS state after the
+      // checkpoint.
+      assertTrue("path exists before restart", fs.exists(testPath));
+      
+      secondary.shutdown();
+      
+      // Restart the NN so it reloads the edits from on-disk.
+      cluster.restartNameNode();
+  
+      // The created file should still exist after the restart.
+      assertTrue("path should still exist after restart", fs.exists(testPath));
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      if (secondary != null) {
+        secondary.shutdown();
+      }
+    }
+  }
 }
 }

+ 149 - 8
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStreamFile.java

@@ -20,17 +20,30 @@ package org.apache.hadoop.hdfs.server.namenode;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertArrayEquals;
 
 
 import java.io.ByteArrayOutputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Enumeration;
 import java.util.Enumeration;
 import java.util.List;
 import java.util.List;
 import java.util.Vector;
 import java.util.Vector;
 
 
+import javax.servlet.ServletContext;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.servlet.http.HttpServletResponse;
 
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.mockito.Mockito;
 import org.mortbay.jetty.InclusiveByteRange;
 import org.mortbay.jetty.InclusiveByteRange;
 
 
 /*
 /*
@@ -186,9 +199,30 @@ class MockHttpServletResponse implements HttpServletResponse {
 }
 }
 
 
 
 
+public class TestStreamFile {
+  private HdfsConfiguration CONF = new HdfsConfiguration();
+  private DFSClient clientMock = Mockito.mock(DFSClient.class);
+  private HttpServletRequest mockHttpServletRequest = 
+    Mockito.mock(HttpServletRequest.class);
+  private HttpServletResponse mockHttpServletResponse = 
+    Mockito.mock(HttpServletResponse.class);
+  private final ServletContext mockServletContext = 
+    Mockito.mock(ServletContext.class);
 
 
-public class TestStreamFile extends TestCase {
+  StreamFile sfile = new StreamFile() {
+    private static final long serialVersionUID = -5513776238875189473L;
   
   
+    public ServletContext getServletContext() {
+      return mockServletContext;
+    }
+  
+    @Override
+    protected DFSClient getDFSClient(HttpServletRequest request)
+      throws IOException, InterruptedException {
+      return clientMock;
+    }
+  };
+     
   // return an array matching the output of mockfsinputstream
   // return an array matching the output of mockfsinputstream
   private static byte[] getOutputArray(int start, int count) {
   private static byte[] getOutputArray(int start, int count) {
     byte[] a = new byte[count];
     byte[] a = new byte[count];
@@ -200,6 +234,7 @@ public class TestStreamFile extends TestCase {
     return a;
     return a;
   }
   }
   
   
+  @Test
   public void testWriteTo() throws IOException, InterruptedException {
   public void testWriteTo() throws IOException, InterruptedException {
 
 
     FSInputStream fsin = new MockFSInputStream();
     FSInputStream fsin = new MockFSInputStream();
@@ -219,7 +254,7 @@ public class TestStreamFile extends TestCase {
     assertTrue("Pairs array must be even", pairs.length % 2 == 0);
     assertTrue("Pairs array must be even", pairs.length % 2 == 0);
     
     
     for (int i = 0; i < pairs.length; i+=2) {
     for (int i = 0; i < pairs.length; i+=2) {
-      StreamFile.writeTo(fsin, os, pairs[i], pairs[i+1]);
+      StreamFile.copyFromOffset(fsin, os, pairs[i], pairs[i+1]);
       assertArrayEquals("Reading " + pairs[i+1]
       assertArrayEquals("Reading " + pairs[i+1]
                         + " bytes from offset " + pairs[i],
                         + " bytes from offset " + pairs[i],
                         getOutputArray(pairs[i], pairs[i+1]),
                         getOutputArray(pairs[i], pairs[i+1]),
@@ -228,20 +263,22 @@ public class TestStreamFile extends TestCase {
     }
     }
     
     
   }
   }
-  
-  private List<?> strToRanges(String s, int contentLength) {
+
+  @SuppressWarnings("unchecked")
+  private List<InclusiveByteRange> strToRanges(String s, int contentLength) {
     List<String> l = Arrays.asList(new String[]{"bytes="+s});
     List<String> l = Arrays.asList(new String[]{"bytes="+s});
     Enumeration<?> e = (new Vector<String>(l)).elements();
     Enumeration<?> e = (new Vector<String>(l)).elements();
     return InclusiveByteRange.satisfiableRanges(e, contentLength);
     return InclusiveByteRange.satisfiableRanges(e, contentLength);
   }
   }
   
   
+  @Test
   public void testSendPartialData() throws IOException, InterruptedException {
   public void testSendPartialData() throws IOException, InterruptedException {
     FSInputStream in = new MockFSInputStream();
     FSInputStream in = new MockFSInputStream();
     ByteArrayOutputStream os = new ByteArrayOutputStream();
     ByteArrayOutputStream os = new ByteArrayOutputStream();
 
 
     // test if multiple ranges, then 416
     // test if multiple ranges, then 416
     { 
     { 
-      List<?> ranges = strToRanges("0-,10-300", 500);
+      List<InclusiveByteRange> ranges = strToRanges("0-,10-300", 500);
       MockHttpServletResponse response = new MockHttpServletResponse();
       MockHttpServletResponse response = new MockHttpServletResponse();
       StreamFile.sendPartialData(in, os, response, 500, ranges);
       StreamFile.sendPartialData(in, os, response, 500, ranges);
       assertEquals("Multiple ranges should result in a 416 error",
       assertEquals("Multiple ranges should result in a 416 error",
@@ -259,7 +296,7 @@ public class TestStreamFile extends TestCase {
 
 
     // test if invalid single range (out of bounds), then 416
     // test if invalid single range (out of bounds), then 416
     { 
     { 
-      List<?> ranges = strToRanges("600-800", 500);
+      List<InclusiveByteRange> ranges = strToRanges("600-800", 500);
       MockHttpServletResponse response = new MockHttpServletResponse();
       MockHttpServletResponse response = new MockHttpServletResponse();
       StreamFile.sendPartialData(in, os, response, 500, ranges);
       StreamFile.sendPartialData(in, os, response, 500, ranges);
       assertEquals("Single (but invalid) range should result in a 416",
       assertEquals("Single (but invalid) range should result in a 416",
@@ -269,7 +306,7 @@ public class TestStreamFile extends TestCase {
       
       
     // test if one (valid) range, then 206
     // test if one (valid) range, then 206
     { 
     { 
-      List<?> ranges = strToRanges("100-300", 500);
+      List<InclusiveByteRange> ranges = strToRanges("100-300", 500);
       MockHttpServletResponse response = new MockHttpServletResponse();
       MockHttpServletResponse response = new MockHttpServletResponse();
       StreamFile.sendPartialData(in, os, response, 500, ranges);
       StreamFile.sendPartialData(in, os, response, 500, ranges);
       assertEquals("Single (valid) range should result in a 206",
       assertEquals("Single (valid) range should result in a 206",
@@ -280,4 +317,108 @@ public class TestStreamFile extends TestCase {
     }
     }
     
     
   }
   }
+  
+  
+    // Test for positive scenario
+  @Test
+  public void testDoGetShouldWriteTheFileContentIntoServletOutputStream()
+      throws Exception {
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(1)
+        .build();
+    try {
+      Path testFile = createFile();
+      setUpForDoGetTest(cluster, testFile);
+      ServletOutputStreamExtn outStream = new ServletOutputStreamExtn();
+      Mockito.doReturn(outStream).when(mockHttpServletResponse)
+          .getOutputStream();
+      StreamFile sfile = new StreamFile() {
+
+        private static final long serialVersionUID = 7715590481809562722L;
+
+        public ServletContext getServletContext() {
+          return mockServletContext;
+        }
+      };
+      sfile.doGet(mockHttpServletRequest, mockHttpServletResponse);
+      assertEquals("Not writing the file data into ServletOutputStream",
+          outStream.getResult(), "test");
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  // Test for cleaning the streams in exception cases also
+  @Test
+  public void testDoGetShouldCloseTheDFSInputStreamIfResponseGetOutPutStreamThrowsAnyException()
+      throws Exception {
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(1)
+        .build();
+    try {
+      Path testFile = createFile();
+
+      setUpForDoGetTest(cluster, testFile);
+
+      Mockito.doThrow(new IOException()).when(mockHttpServletResponse)
+          .getOutputStream();
+      DFSInputStream fsMock = Mockito.mock(DFSInputStream.class);
+
+      Mockito.doReturn(fsMock).when(clientMock).open(testFile.toString());
+
+      Mockito.doReturn(Long.valueOf(4)).when(fsMock).getFileLength();
+
+      try {
+        sfile.doGet(mockHttpServletRequest, mockHttpServletResponse);
+        fail("Not throwing the IOException");
+      } catch (IOException e) {
+        Mockito.verify(clientMock, Mockito.atLeastOnce()).close();
+      }
+
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  private void setUpForDoGetTest(MiniDFSCluster cluster, Path testFile)
+      throws IOException {
+
+    Mockito.doReturn(CONF).when(mockServletContext).getAttribute(
+        JspHelper.CURRENT_CONF);
+    Mockito.doReturn(NameNode.getHostPortString(NameNode.getAddress(CONF)))
+        .when(mockHttpServletRequest).getParameter("nnaddr");
+    Mockito.doReturn(testFile.toString()).when(mockHttpServletRequest)
+        .getPathInfo();
+  }
+
+  static Path writeFile(FileSystem fs, Path f) throws IOException {
+    DataOutputStream out = fs.create(f);
+    try {
+      out.writeBytes("test");
+    } finally {
+      out.close();
+    }
+    assertTrue(fs.exists(f));
+    return f;
+  }
+
+  private Path createFile() throws IOException {
+    FileSystem fs = FileSystem.get(CONF);
+    Path testFile = new Path("/test/mkdirs/doGet");
+    writeFile(fs, testFile);
+    return testFile;
+  }
+
+  public static class ServletOutputStreamExtn extends ServletOutputStream {
+    private StringBuffer buffer = new StringBuffer(3);
+
+    public String getResult() {
+      return buffer.toString();
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+      buffer.append((char) b);
+    }
+  }
 }
 }

+ 74 - 0
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.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.server.namenode;
+
+import static org.junit.Assert.*;
+import org.junit.Test;
+import java.io.IOException;
+import junit.framework.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+
+/**
+ * This class tests the validation of the configuration object when passed 
+ * to the NameNode
+ */
+public class TestValidateConfigurationSettings {
+
+  /**
+   * Tests setting the rpc port to the same as the web port to test that 
+   * an exception
+   * is thrown when trying to re-use the same port
+   */
+  @Test
+  public void testThatMatchingRPCandHttpPortsThrowException() 
+      throws IOException {
+
+    Configuration conf = new HdfsConfiguration();
+    // set both of these to port 9000, should fail
+    FileSystem.setDefaultUri(conf, "hdfs://localhost:9000"); 
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:9000");
+    DFSTestUtil.formatNameNode(conf);
+    try {
+      NameNode nameNode = new NameNode(conf);
+      Assert.fail("Should have throw the exception since the ports match");
+    } catch (IOException e) {
+      // verify we're getting the right IOException
+      assertTrue(e.toString().contains("dfs.namenode.rpc-address (")); 
+      System.out.println("Got expected exception: " + e.toString());
+    }
+  }
+
+  /**
+   * Tests setting the rpc port to a different as the web port that an 
+   * exception is NOT thrown 
+   */
+  @Test
+  public void testThatDifferentRPCandHttpPortsAreOK() 
+      throws IOException {
+
+    Configuration conf = new HdfsConfiguration();
+    FileSystem.setDefaultUri(conf, "hdfs://localhost:8000");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:9000");
+    DFSTestUtil.formatNameNode(conf);
+    NameNode nameNode = new NameNode(conf); // should be OK!
+  }
+}

+ 9 - 8
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java

@@ -17,34 +17,35 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode.metrics;
 package org.apache.hadoop.hdfs.server.namenode.metrics;
 
 
+import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
+import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.Random;
 import java.util.Random;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
+import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.Path;
+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.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.namenode.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.test.MetricsAsserts;
 import org.apache.hadoop.test.MetricsAsserts;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
 
 
-import org.apache.commons.logging.LogFactory;
-
-import static org.apache.hadoop.test.MetricsAsserts.*;
-
 /**
 /**
  * Test for metrics published by the Namenode
  * Test for metrics published by the Namenode
  */
  */

+ 1 - 1
hdfs/src/test/hdfs/org/apache/hadoop/net/TestNetworkTopology.java

@@ -24,8 +24,8 @@ import java.util.Map;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 
 
 public class TestNetworkTopology extends TestCase {
 public class TestNetworkTopology extends TestCase {
   private final static NetworkTopology cluster = new NetworkTopology();
   private final static NetworkTopology cluster = new NetworkTopology();

+ 13 - 7
hdfs/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java

@@ -19,6 +19,16 @@
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
 import static junit.framework.Assert.assertTrue;
 import static junit.framework.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+import java.io.File;
+import java.io.IOException;
+
 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;
@@ -34,20 +44,16 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.After;
-
-import static org.junit.Assert.*;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.*;
-
-import java.io.File;
-import java.io.IOException;
 
 
 public class TestNNLeaseRecovery {
 public class TestNNLeaseRecovery {
   private static final Log LOG = LogFactory.getLog(TestNNLeaseRecovery.class);
   private static final Log LOG = LogFactory.getLog(TestNNLeaseRecovery.class);

+ 0 - 10
hdfs/src/webapps/hdfs/block_info_xml.jsp

@@ -59,19 +59,9 @@
 %>
 %>
 <%@ page
 <%@ page
   contentType="application/xml"
   contentType="application/xml"
-  import="java.io.IOException"
-  import="java.util.Iterator"
-  import="org.apache.hadoop.conf.Configuration"
-  import="org.apache.hadoop.hdfs.protocol.Block"
-  import="org.apache.hadoop.hdfs.server.namenode.INode"
-  import="org.apache.hadoop.hdfs.server.namenode.BlocksMap"
-  import="org.apache.hadoop.hdfs.server.namenode.BlockInfo"
-  import="org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor"
   import="org.apache.hadoop.hdfs.server.namenode.NamenodeJspHelper.XMLBlockInfo"
   import="org.apache.hadoop.hdfs.server.namenode.NamenodeJspHelper.XMLBlockInfo"
   import="org.apache.hadoop.hdfs.server.common.JspHelper"
   import="org.apache.hadoop.hdfs.server.common.JspHelper"
-  import="org.apache.hadoop.util.ServletUtil"
   import="org.znerd.xmlenc.*"
   import="org.znerd.xmlenc.*"
-  
 %>
 %>
 <%!
 <%!
   //for java.io.Serializable
   //for java.io.Serializable