Jelajahi Sumber

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 tahun lalu
induk
melakukan
45da9c0dea
95 mengubah file dengan 2075 tambahan dan 1042 penghapusan
  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)
 
     HDFS-2055. Add hflush support to libhdfs. (Travis Crawford via eli)
+   
+    HDFS-2083. Query JMX statistics over http via JMXJsonServlet. (tanping)
 
   IMPROVEMENTS
 
@@ -519,6 +521,25 @@ Trunk (unreleased changes)
     HDFS-1568. Improve the log messages in DataXceiver.  (Joey Echeverria via
     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
 
     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-2118. Couple dfs data dir improvements. (eli)
+
   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-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.
     (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
 
   INCOMPATIBLE CHANGES
@@ -819,6 +865,8 @@ Release 0.22.0 - Unreleased
     HDFS-528. Add ability for safemode to wait for a minimum number of 
     live datanodes (Todd Lipcon via eli)
 
+    HDFS-1753. Resource Leak in StreamFile. (Uma Maheswara Rao G via eli)
+
   IMPROVEMENTS
 
     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)
                                      throws IOException {
     // 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

+ 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.hdfs.server.namenode.StreamFile;
 
-
 /**
  * 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 
@@ -60,7 +59,9 @@ class ByteRangeInputStream extends FSInputStream {
     }  
   }
   
-  
+  enum StreamStatus {
+    NORMAL, SEEK
+  }
   protected InputStream in;
   protected URLOpener originalURL;
   protected URLOpener resolvedURL;
@@ -68,9 +69,7 @@ class ByteRangeInputStream extends FSInputStream {
   protected long currentPos = 0;
   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) {
     this(new URLOpener(url), new URLOpener(null));
@@ -82,18 +81,19 @@ class ByteRangeInputStream extends FSInputStream {
   }
   
   private InputStream getInputStream() throws IOException {
-    if (status != STATUS_NORMAL) {
+    if (status != StreamStatus.NORMAL) {
       
       if (in != null) {
         in.close();
         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 {
         connection.setRequestMethod("GET");
         if (startPos != 0) {
@@ -101,36 +101,35 @@ class ByteRangeInputStream extends FSInputStream {
         }
         connection.connect();
         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()) {
           HftpFileSystem.LOG.debug("filelength = " + filelength);
         }
         in = connection.getInputStream();
-      } catch(IOException ioe) {
+      } catch (IOException 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...
-        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)
-        throw new IOException("200 expected, but received "
-                              + connection.getResponseCode());
+        throw new IOException("HTTP_OK expected, received " + respCode);
       }
-      
+
       resolvedURL.setURL(connection.getURL());
-      status = STATUS_NORMAL;
+      status = StreamStatus.NORMAL;
     }
     
     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) {
       currentPos += n;
     } else if (currentPos < filelength) {
@@ -154,7 +153,7 @@ class ByteRangeInputStream extends FSInputStream {
     if (pos != currentPos) {
       startPos = 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
    */
   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 {
     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();
   final String clientName;
   Configuration conf;
-  long defaultBlockSize;
-  private short defaultReplication;
   SocketFactory socketFactory;
-  int socketTimeout;
-  final int writePacketSize;
   final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
   final FileSystem.Statistics stats;
   final int hdfsTimeout;    // timeout value for a DFS operation.
   final LeaseRenewer leaserenewer;
-
   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
@@ -257,16 +327,11 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   DFSClient(InetSocketAddress nameNodeAddr, ClientProtocol rpcNamenode,
       Configuration conf, FileSystem.Statistics stats)
     throws IOException {
+    // Copy only the required DFSClient configuration
+    this.dfsClientConf = new Conf(conf);
     this.conf = conf;
     this.stats = stats;
-    this.socketTimeout = 
-      conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 
-                  HdfsConstants.READ_TIMEOUT);
     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);
 
     // 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":
         nameNodeAddr.getHostName() + ":" + nameNodeAddr.getPort();
     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) {
       this.rpcNamenode = createRPCNamenode(nameNodeAddr, conf, ugi);
       this.namenode = createNamenode(this.rpcNamenode);
@@ -306,8 +360,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * to retrieve block locations when reading.
    */
   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.
    */
   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) {
-    return socketTimeout > 0 ?
+    return dfsClientConf.socketTimeout > 0 ?
         (HdfsConstants.READ_TIMEOUT_EXTENSION * numNodes +
-        socketTimeout) : 0;
+            dfsClientConf.socketTimeout) : 0;
   }
   
   int getHdfsTimeout() {
@@ -430,7 +479,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * @return the default block size in bytes
    */
   public long getDefaultBlockSize() {
-    return defaultBlockSize;
+    return dfsClientConf.defaultBlockSize;
   }
     
   /**
@@ -528,7 +577,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   }
   
   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) 
       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) 
       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, 
                              boolean overwrite,
                              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,
       long blockSize, Progressable progress) throws IOException {
     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) {
       permission = FsPermission.getDefault();
     }
-    FsPermission masked = permission.applyUMask(FsPermission.getUMask(conf));
+    FsPermission masked = permission.applyUMask(dfsClientConf.uMask);
     if(LOG.isDebugEnabled()) {
       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);
     return result;
   }
@@ -808,7 +858,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       throws IOException {
     try {
       FsPermission dirPerm = 
-          FsPermission.getDefault().applyUMask(FsPermission.getUMask(conf)); 
+          FsPermission.getDefault().applyUMask(dfsClientConf.uMask); 
       namenode.createSymlink(target, link, dirPerm, createParent);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
@@ -851,8 +901,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
                                      UnresolvedPathException.class);
     }
     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 {
     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);
           }
           // get block MD5
-          Sender.opBlockChecksum(out, block, lb.getBlockToken());
+          new Sender(out).blockChecksum(block, lb.getBlockToken());
 
           final BlockOpResponseProto reply =
             BlockOpResponseProto.parseFrom(HdfsProtoUtil.vintPrefixed(in));
@@ -1377,7 +1426,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     if (permission == null) {
       permission = FsPermission.getDefault();
     }
-    FsPermission masked = permission.applyUMask(FsPermission.getUMask(conf));
+    FsPermission masked = permission.applyUMask(dfsClientConf.uMask);
     if(LOG.isDebugEnabled()) {
       LOG.debug(src + ": masked=" + masked);
     }
@@ -1404,7 +1453,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     checkOpen();
     if (absPermission == null) {
       absPermission = 
-        FsPermission.getDefault().applyUMask(FsPermission.getUMask(conf));
+        FsPermission.getDefault().applyUMask(dfsClientConf.uMask);
     } 
 
     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
    */
   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 
    * parallel accesses to DFSInputStream (through ptreads) properly */
@@ -106,13 +106,9 @@ public class DFSInputStream extends FSInputStream {
     this.buffersize = buffersize;
     this.src = src;
     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();
   }
 
@@ -163,7 +159,7 @@ public class DFSInputStream extends FSInputStream {
       
       try {
         cdp = DFSClient.createClientDatanodeProtocolProxy(
-        datanode, dfsClient.conf, dfsClient.socketTimeout, locatedblock);
+        datanode, dfsClient.conf, dfsClient.getConf().socketTimeout, locatedblock);
         
         final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
         
@@ -771,8 +767,8 @@ public class DFSInputStream extends FSInputStream {
         // disaster.
         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 {

+ 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 {
   private final DFSClient dfsClient;
-  private Configuration conf;
   private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB
   private Socket s;
   // 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 
         // smaller size packet.
         //
-        computePacketChunkSize(Math.min(dfsClient.writePacketSize, freeInLastBlock), 
+        computePacketChunkSize(Math.min(dfsClient.getConf().writePacketSize, freeInLastBlock), 
             bytesPerChecksum);
       }
 
@@ -426,8 +425,8 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
                 && dataQueue.size() == 0 && 
                 (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 = (stage == BlockConstructionStage.DATA_STREAMING)?
                  timeout : 1000;
@@ -847,8 +846,8 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
             DataNode.SMALL_BUFFER_SIZE));
 
         //send the TRANSFER_BLOCK request
-        Sender.opTransferBlock(out, block,
-            dfsClient.clientName, targets, blockToken);
+        new Sender(out).transferBlock(block, blockToken, dfsClient.clientName,
+            targets);
 
         //ack
         in = new DataInputStream(NetUtils.getInputStream(sock));
@@ -953,8 +952,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
     private DatanodeInfo[] nextBlockOutputStream(String client) throws IOException {
       LocatedBlock lb = 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;
       do {
         hasError = false;
@@ -1021,10 +1019,9 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
         blockReplyStream = new DataInputStream(NetUtils.getInputStream(s));
 
         // 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);
         out.flush();
 
@@ -1079,9 +1076,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
     private LocatedBlock locateFollowingBlock(long start,
         DatanodeInfo[] excludedNodes) 
         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;
       while (true) {
         long localstart = System.currentTimeMillis();
@@ -1201,7 +1196,6 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
       int bytesPerChecksum, short replication) throws IOException {
     super(new PureJavaCrc32(), bytesPerChecksum, 4);
     this.dfsClient = dfsClient;
-    this.conf = dfsClient.conf;
     this.src = src;
     this.blockSize = blockSize;
     this.blockReplication = replication;
@@ -1232,7 +1226,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
       throws IOException {
     this(dfsClient, src, blockSize, progress, bytesPerChecksum, replication);
 
-    computePacketChunkSize(dfsClient.writePacketSize, bytesPerChecksum);
+    computePacketChunkSize(dfsClient.getConf().writePacketSize, bytesPerChecksum);
 
     try {
       dfsClient.namenode.create(
@@ -1269,7 +1263,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
       bytesCurBlock = lastBlock.getBlockSize();
       streamer = new DataStreamer(lastBlock, stat, bytesPerChecksum);
     } else {
-      computePacketChunkSize(dfsClient.writePacketSize, bytesPerChecksum);
+      computePacketChunkSize(dfsClient.getConf().writePacketSize, bytesPerChecksum);
       streamer = new DataStreamer();
     }
     streamer.start();
@@ -1385,7 +1379,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
       }
 
       if (!appendChunk) {
-        int psize = Math.min((int)(blockSize-bytesCurBlock), dfsClient.writePacketSize);
+        int psize = Math.min((int)(blockSize-bytesCurBlock), dfsClient.getConf().writePacketSize);
         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.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
  * to a filesystem over HTTP.
  * @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 {
     try {
       final URL url = new URI("http", null, nnAddr.getHostName(),
@@ -305,7 +313,7 @@ public class HftpFileSystem extends FileSystem {
     try {
       connection.setRequestMethod("GET");
       connection.connect();
-    } catch(IOException ioe) {
+    } catch (IOException ioe) {
       throwIOExceptionFromConnection(connection, ioe);
     }
     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();
     if (msg == null) {
       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 {
       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;
 
+import java.io.IOException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 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.
@@ -35,8 +41,101 @@ public interface DataTransferProtocol {
    * 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.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.OpCopyBlockProto;
 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.OpTransferBlockProto;
 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 */
 @InterfaceAudience.Private
 @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. */
-  protected final Op readOp(DataInputStream in) throws IOException {
+  protected final Op readOp() throws IOException {
     final short version = in.readShort();
     if (version != DataTransferProtocol.DATA_TRANSFER_VERSION) {
       throw new IOException( "Version Mismatch (Expected: " +
@@ -54,11 +57,10 @@ public abstract class Receiver {
   }
 
   /** 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) {
     case READ_BLOCK:
-      opReadBlock(in);
+      opReadBlock();
       break;
     case WRITE_BLOCK:
       opWriteBlock(in);
@@ -81,121 +83,60 @@ public abstract class Receiver {
   }
 
   /** Receive OP_READ_BLOCK */
-  private void opReadBlock(DataInputStream in) throws IOException {
+  private void opReadBlock() throws IOException {
     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 */
   private void opWriteBlock(DataInputStream in) throws IOException {
     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(),
-        fromProto(proto.getSource()),
         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} */
   private void opTransferBlock(DataInputStream in) throws IOException {
     final OpTransferBlockProto proto =
       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(),
-        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 */
   private void opReplaceBlock(DataInputStream in) throws IOException {
     OpReplaceBlockProto proto = OpReplaceBlockProto.parseFrom(vintPrefixed(in));
-
-    opReplaceBlock(in,
-        fromProto(proto.getHeader().getBlock()),
+    replaceBlock(fromProto(proto.getHeader().getBlock()),
+        fromProto(proto.getHeader().getToken()),
         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 */
   private void opCopyBlock(DataInputStream in) throws IOException {
     OpCopyBlockProto proto = OpCopyBlockProto.parseFrom(vintPrefixed(in));
-    
-    opCopyBlock(in,
-        fromProto(proto.getHeader().getBlock()),
+    copyBlock(fromProto(proto.getHeader().getBlock()),
         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 */
   private void opBlockChecksum(DataInputStream in) throws IOException {
     OpBlockChecksumProto proto = OpBlockChecksumProto.parseFrom(vintPrefixed(in));
     
-    opBlockChecksum(in,
-        fromProto(proto.getHeader().getBlock()),
+    blockChecksum(fromProto(proto.getHeader().getBlock()),
         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 */
 @InterfaceAudience.Private
 @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. */
   private static void op(final DataOutput out, final Op op
       ) throws IOException {
@@ -59,79 +66,85 @@ public class Sender {
     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()
       .setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName, blockToken))
       .setOffset(blockOffset)
-      .setLen(blockLen)
+      .setLen(length)
       .build();
 
     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()
       .setHeader(header)
-      .addAllTargets(
-          toProtos(targets, 1))
+      .addAllTargets(toProtos(targets, 1))
       .setStage(toProto(stage))
       .setPipelineSize(pipelineSize)
       .setMinBytesRcvd(minBytesRcvd)
       .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 {@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()
       .setHeader(DataTransferProtoUtil.buildClientHeader(
-          blk, client, blockToken))
+          blk, clientName, blockToken))
       .addAllTargets(toProtos(targets, 0))
       .build();
 
     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()
       .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
       .setDelHint(delHint)
-      .setSource(toProto(src))
+      .setSource(toProto(source))
       .build();
     
     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()
       .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
       .build();
@@ -139,10 +152,9 @@ public class Sender {
     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()
       .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
       .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.Status;
 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.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.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.io.IOUtils;
@@ -348,8 +348,8 @@ public class Balancer {
     private void sendRequest(DataOutputStream out) throws IOException {
       final ExtendedBlock eb = new ExtendedBlock(nnc.blockpoolID, block.getBlock());
       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 */ 

+ 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
  * 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.server.common.HdfsConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.util.LightWeightGSet;
 
 /**
  * Internal class for block metadata.
  */
-class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
+public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   private INodeFile inode;
 
   /** For implementing {@link LightWeightGSet.LinkedElement} interface */
@@ -44,12 +45,12 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * Construct an entry for blocksmap
    * @param replication the block's replication factor
    */
-  protected BlockInfo(int replication) {
+  public BlockInfo(int replication) {
     this.triplets = new Object[3*replication];
     this.inode = null;
   }
   
-  protected BlockInfo(Block blk, int replication) {
+  public BlockInfo(Block blk, int replication) {
     super(blk);
     this.triplets = new Object[3*replication];
     this.inode = null;
@@ -65,11 +66,11 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     this.inode = from.inode;
   }
 
-  INodeFile getINode() {
+  public INodeFile getINode() {
     return inode;
   }
 
-  void setINode(INodeFile inode) {
+  public void setINode(INodeFile inode) {
     this.inode = inode;
   }
 
@@ -162,7 +163,7 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   /**
    * 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
       return false;
     // find the last null node
@@ -176,7 +177,7 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   /**
    * Remove data-node from the block.
    */
-  boolean removeNode(DatanodeDescriptor node) {
+  public boolean removeNode(DatanodeDescriptor node) {
     int dnIndex = findDatanode(node);
     if(dnIndex < 0) // the node is not found
       return false;
@@ -218,7 +219,7 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * If the head is null then form a new 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);
     assert dnIndex >= 0 : "Data node is not found: current";
     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
    * empty after deletion.
    */
-  BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
+  public BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
     if(head == null)
       return null;
     int dnIndex = this.findDatanode(dn);
@@ -284,7 +285,7 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * to {@link BlockInfoUnderConstruction}.
    * @return {@link BlockUCState#COMPLETE}
    */
-  BlockUCState getBlockUCState() {
+  public BlockUCState getBlockUCState() {
     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}
    */
-  boolean isComplete() {
+  public boolean isComplete() {
     return getBlockUCState().equals(BlockUCState.COMPLETE);
   }
 
@@ -302,7 +303,7 @@ class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * 
    * @return BlockInfoUnderConstruction -  an under construction block.
    */
-  BlockInfoUnderConstruction convertToBlockUnderConstruction(
+  public BlockInfoUnderConstruction convertToBlockUnderConstruction(
       BlockUCState s, DatanodeDescriptor[] targets) {
     if(isComplete()) {
       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
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -24,12 +24,13 @@ import java.util.List;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 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>
  * 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} */
   private BlockUCState blockUCState;
 
@@ -128,11 +129,14 @@ class BlockInfoUnderConstruction extends BlockInfo {
    * Create block and set its state to
    * {@link BlockUCState#UNDER_CONSTRUCTION}.
    */
-  BlockInfoUnderConstruction(Block blk, int replication) {
+  public BlockInfoUnderConstruction(Block blk, int replication) {
     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,
                              DatanodeDescriptor[] targets) {
     super(blk, replication);
@@ -160,7 +164,8 @@ class BlockInfoUnderConstruction extends BlockInfo {
     return new BlockInfo(this);
   }
 
-  void setExpectedLocations(DatanodeDescriptor[] targets) {
+  /** Set expected locations */
+  public void setExpectedLocations(DatanodeDescriptor[] targets) {
     int numLocations = targets == null ? 0 : targets.length;
     this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
     for(int i = 0; i < numLocations; i++)
@@ -172,7 +177,7 @@ class BlockInfoUnderConstruction extends BlockInfo {
    * Create array of expected replica locations
    * (as has been assigned by chooseTargets()).
    */
-  DatanodeDescriptor[] getExpectedLocations() {
+  public DatanodeDescriptor[] getExpectedLocations() {
     int numLocations = replicas == null ? 0 : replicas.size();
     DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocations];
     for(int i = 0; i < numLocations; i++)
@@ -180,7 +185,8 @@ class BlockInfoUnderConstruction extends BlockInfo {
     return locations;
   }
 
-  int getNumExpectedLocations() {
+  /** Get the number of expected locations */
+  public int getNumExpectedLocations() {
     return replicas == null ? 0 : replicas.size();
   }
 
@@ -189,7 +195,7 @@ class BlockInfoUnderConstruction extends BlockInfo {
    * @see BlockUCState
    */
   @Override // BlockInfo
-  BlockUCState getBlockUCState() {
+  public BlockUCState getBlockUCState() {
     return blockUCState;
   }
 
@@ -197,7 +203,8 @@ class BlockInfoUnderConstruction extends BlockInfo {
     blockUCState = s;
   }
 
-  long getBlockRecoveryId() {
+  /** Get block recovery ID */
+  public long getBlockRecoveryId() {
     return blockRecoveryId;
   }
 
@@ -220,7 +227,7 @@ class BlockInfoUnderConstruction extends BlockInfo {
    * Find the first alive data-node starting from the previous primary and
    * make it primary.
    */
-  void initializeBlockRecovery(long recoveryId) {
+  public void initializeBlockRecovery(long recoveryId) {
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;
     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
  * 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.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.DatanodeInfo;
 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.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.
@@ -57,18 +61,43 @@ public class BlockManager {
 
   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
@@ -90,24 +119,24 @@ public class BlockManager {
   // eventually remove these extras.
   // Mapping: StorageID -> TreeSet<Block>
   //
-  Map<String, Collection<Block>> excessReplicateMap =
+  public final Map<String, Collection<Block>> excessReplicateMap =
     new TreeMap<String, Collection<Block>>();
 
   //
   // Store set of Blocks that need to be replicated 1 or more times.
   // We also store pending replication-orders.
   //
-  UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
+  public UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
   private PendingReplicationBlocks pendingReplications;
 
   //  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
-  int maxReplicationStreams;
+  public int maxReplicationStreams;
   // Minimum copies needed or else write is disallowed
-  int minReplication;
+  public int minReplication;
   // Default number of replicas
-  int defaultReplication;
+  public int defaultReplication;
   // How many entries are returned by getCorruptInodes()
   int maxCorruptFilesReturned;
   
@@ -121,9 +150,9 @@ public class BlockManager {
   Random r = new Random();
 
   // 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);
   }
   
@@ -178,16 +207,16 @@ public class BlockManager {
     FSNamesystem.LOG.info("shouldCheckForEnoughRacks = " + shouldCheckForEnoughRacks);
   }
 
-  void activate() {
+  public void activate() {
     pendingReplications.start();
   }
 
-  void close() {
+  public void close() {
     if (pendingReplications != null) pendingReplications.stop();
     blocksMap.close();
   }
 
-  void metaSave(PrintWriter out) {
+  public void metaSave(PrintWriter out) {
     //
     // Dump contents of neededReplication
     //
@@ -249,7 +278,7 @@ public class BlockManager {
    * @param block
    * @return true if the block has minimum replicas
    */
-  boolean checkMinReplication(Block block) {
+  public boolean checkMinReplication(Block block) {
     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
    * of replicas reported from data-nodes.
    */
-  void commitOrCompleteLastBlock(INodeFileUnderConstruction fileINode, 
+  public void commitOrCompleteLastBlock(INodeFileUnderConstruction fileINode, 
       Block commitBlock) throws IOException {
     
     if(commitBlock == null)
@@ -362,7 +391,7 @@ public class BlockManager {
    * @param fileINode file
    * @return the last block locations if the block is partial or null otherwise
    */
-  LocatedBlock convertLastBlockToUnderConstruction(
+  public LocatedBlock convertLastBlockToUnderConstruction(
       INodeFileUnderConstruction fileINode) throws IOException {
     BlockInfo oldBlock = fileINode.getLastBlock();
     if(oldBlock == null ||
@@ -393,7 +422,7 @@ public class BlockManager {
   /**
    * Get all valid locations of the block
    */
-  ArrayList<String> getValidLocations(Block block) {
+  public ArrayList<String> getValidLocations(Block block) {
     ArrayList<String> machineSet =
       new ArrayList<String>(blocksMap.numNodes(block));
     for(Iterator<DatanodeDescriptor> it =
@@ -407,7 +436,7 @@ public class BlockManager {
     return machineSet;
   }
 
-  List<LocatedBlock> getBlockLocations(BlockInfo[] blocks, long offset,
+  public List<LocatedBlock> getBlockLocations(BlockInfo[] blocks, long offset,
       long length, int nrBlocksToReturn) throws IOException {
     int curBlk = 0;
     long curPos = 0, blkSize = 0;
@@ -436,11 +465,15 @@ public class BlockManager {
     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 {
-    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 DatanodeDescriptor[] locations = uc.getExpectedLocations();
       return namesystem.createLocatedBlock(uc, locations, pos, false);
@@ -476,7 +509,7 @@ public class BlockManager {
    * Check whether the replication parameter is within the range
    * determined by system configuration.
    */
-   void verifyReplication(String src,
+   public void verifyReplication(String src,
                           short replication,
                           String clientName) throws IOException {
 
@@ -544,7 +577,7 @@ public class BlockManager {
    * @param b block
    * @param dn datanode
    */
-  void addToInvalidates(Block b, DatanodeInfo dn) {
+  public void addToInvalidates(Block b, DatanodeInfo dn) {
     addToInvalidates(b, dn, true);
   }
 
@@ -585,7 +618,7 @@ public class BlockManager {
     }
   }
 
-  void findAndMarkBlockAsCorrupt(Block blk,
+  public void findAndMarkBlockAsCorrupt(Block blk,
                                  DatanodeInfo dn) throws IOException {
     BlockInfo storedBlock = getStoredBlock(blk);
     if (storedBlock == null) {
@@ -668,14 +701,14 @@ public class BlockManager {
     }
   }
 
-  void updateState() {
+  public void updateState() {
     pendingReplicationBlocksCount = pendingReplications.size();
     underReplicatedBlocksCount = neededReplications.size();
     corruptReplicaBlocksCount = corruptReplicas.size();
   }
 
   /** Return number of under-replicated but not missing blocks */
-  int getUnderReplicatedNotMissingBlocks() {
+  public int getUnderReplicatedNotMissingBlocks() {
     return neededReplications.getUnderReplicatedBlockCount();
   }
   
@@ -684,7 +717,7 @@ public class BlockManager {
    * @param nodesToProcess number of datanodes to schedule deletion work
    * @return total number of block for deletion
    */
-  int computeInvalidateWork(int nodesToProcess) {
+  public int computeInvalidateWork(int nodesToProcess) {
     int numOfNodes = recentInvalidateSets.size();
     nodesToProcess = Math.min(numOfNodes, nodesToProcess);
 
@@ -724,7 +757,7 @@ public class BlockManager {
    *
    * @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
     List<List<Block>> blocksToReplicate =
       chooseUnderReplicatedBlocks(blocksToProcess);
@@ -1031,7 +1064,7 @@ public class BlockManager {
    * If there were any replication requests that timed out, reap them
    * and put them back into the neededReplication queue
    */
-  void processPendingReplications() {
+  public void processPendingReplications() {
     Block[] timedOutItems = pendingReplications.getTimedOutBlocks();
     if (timedOutItems != null) {
       namesystem.writeLock();
@@ -1464,7 +1497,7 @@ public class BlockManager {
     short fileReplication = fileINode.getReplication();
     if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
       neededReplications.remove(storedBlock, numCurrentReplica,
-          num.decommissionedReplicas, fileReplication);
+          num.decommissionedReplicas(), fileReplication);
     } else {
       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,
    * over or under replicated. Place it into the respective queue.
    */
-  void processMisReplicatedBlocks() {
+  public void processMisReplicatedBlocks() {
     long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0;
     namesystem.writeLock();
     try {
@@ -1570,7 +1603,7 @@ public class BlockManager {
    * If there are any extras, call chooseExcessReplicates() to
    * mark them in the excessReplicateMap.
    */
-  void processOverReplicatedBlock(Block block, short replication,
+  public void processOverReplicatedBlock(Block block, short replication,
       DatanodeDescriptor addedNode, DatanodeDescriptor delNodeHint) {
     assert namesystem.hasWriteLock();
     if (addedNode == delNodeHint) {
@@ -1597,7 +1630,7 @@ public class BlockManager {
         addedNode, delNodeHint, replicator);
   }
 
-  void addToExcessReplicate(DatanodeInfo dn, Block block) {
+  public void addToExcessReplicate(DatanodeInfo dn, Block block) {
     assert namesystem.hasWriteLock();
     Collection<Block> excessBlocks = excessReplicateMap.get(dn.getStorageID());
     if (excessBlocks == null) {
@@ -1618,7 +1651,7 @@ public class BlockManager {
    * Modify (block-->datanode) map. Possibly generate replication tasks, if the
    * removed block is still valid.
    */
-  void removeStoredBlock(Block block, DatanodeDescriptor node) {
+  public void removeStoredBlock(Block block, DatanodeDescriptor node) {
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
           + block + " from " + node.getName());
@@ -1673,7 +1706,7 @@ public class BlockManager {
   /**
    * 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 {
     // decrement number of blocks scheduled to this datanode.
     node.decBlocksScheduled();
@@ -1726,7 +1759,7 @@ public class BlockManager {
   /**
    * Return the number of nodes that are live and decommissioned.
    */
-  NumberReplicas countNodes(Block b) {
+  public NumberReplicas countNodes(Block b) {
     int count = 0;
     int live = 0;
     int corrupt = 0;
@@ -1805,7 +1838,7 @@ public class BlockManager {
    * Return true if there are any blocks on this node that have not
    * yet reached their replication factor. Otherwise returns false.
    */
-  boolean isReplicationInProgress(DatanodeDescriptor srcNode) {
+  public boolean isReplicationInProgress(DatanodeDescriptor srcNode) {
     boolean status = false;
     int underReplicatedBlocks = 0;
     int decommissionOnlyReplicas = 0;
@@ -1855,11 +1888,11 @@ public class BlockManager {
     return status;
   }
 
-  int getActiveBlockCount() {
+  public int getActiveBlockCount() {
     return blocksMap.size() - (int)pendingDeletionBlocksCount;
   }
 
-  DatanodeDescriptor[] getNodes(BlockInfo block) {
+  public DatanodeDescriptor[] getNodes(BlockInfo block) {
     DatanodeDescriptor[] nodes =
       new DatanodeDescriptor[block.numNodes()];
     Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
@@ -1869,22 +1902,22 @@ public class BlockManager {
     return nodes;
   }
 
-  int getTotalBlocks() {
+  public int getTotalBlocks() {
     return blocksMap.size();
   }
 
-  void removeBlock(Block block) {
+  public void removeBlock(Block block) {
     addToInvalidates(block);
     corruptReplicas.removeFromCorruptReplicasMap(block);
     blocksMap.removeBlock(block);
   }
 
-  BlockInfo getStoredBlock(Block block) {
+  public BlockInfo getStoredBlock(Block block) {
     return blocksMap.getStoredBlock(block);
   }
 
   /* updates a block in under replication queue */
-  void updateNeededReplications(Block block, int curReplicasDelta,
+  public void updateNeededReplications(Block block, int curReplicasDelta,
       int expectedReplicasDelta) {
     namesystem.writeLock();
     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.
     NumberReplicas number = countNodes(block);
     if (isNeededReplication(block, numExpectedReplicas, number.liveReplicas())) { 
       neededReplications.add(block,
                              number.liveReplicas(),
-                             number.decommissionedReplicas,
+                             number.decommissionedReplicas(),
                              numExpectedReplicas);
     }
   }
@@ -1926,11 +1959,8 @@ public class BlockManager {
     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);
     if (blocks != null) {
       pendingDeletionBlocksCount -= blocks.size();
@@ -1998,7 +2028,7 @@ public class BlockManager {
   //Returns the number of racks over which a given block is replicated
   //decommissioning/decommissioned nodes are not counted. corrupt replicas 
   //are also ignored
-  int getNumberOfRacks(Block b) {
+  public int getNumberOfRacks(Block b) {
     HashSet<String> rackSet = new HashSet<String>(0);
     Collection<DatanodeDescriptor> corruptNodes = 
                                   corruptReplicas.getNodes(b);
@@ -2056,32 +2086,32 @@ public class BlockManager {
     }
   }
   
-  long getMissingBlocksCount() {
+  public long getMissingBlocksCount() {
     // not locking
     return this.neededReplications.getCorruptBlockSize();
   }
 
-  BlockInfo addINode(BlockInfo block, INodeFile iNode) {
+  public BlockInfo addINode(BlockInfo block, INodeFile iNode) {
     return blocksMap.addINode(block, iNode);
   }
 
-  INodeFile getINode(Block b) {
+  public INodeFile getINode(Block b) {
     return blocksMap.getINode(b);
   }
 
-  void removeFromCorruptReplicasMap(Block block) {
+  public void removeFromCorruptReplicasMap(Block block) {
     corruptReplicas.removeFromCorruptReplicasMap(block);
   }
 
-  int numCorruptReplicas(Block block) {
+  public int numCorruptReplicas(Block block) {
     return corruptReplicas.numCorruptReplicas(block);
   }
 
-  void removeBlockFromMap(Block block) {
+  public void removeBlockFromMap(Block block) {
     blocksMap.removeBlock(block);
   }
 
-  int getCapacity() {
+  public int getCapacity() {
     namesystem.readLock();
     try {
       return blocksMap.getCapacity();
@@ -2104,7 +2134,7 @@ public class BlockManager {
    * @return Up to numExpectedBlocks blocks from startingBlockId if it exists
    *
    */
-  long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
+  public long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
                                    Long startingBlockId) {
     return corruptReplicas.getCorruptReplicaBlockIds(numExpectedBlocks,
                                                      startingBlockId);
@@ -2113,7 +2143,7 @@ public class BlockManager {
   /**
    * Return an iterator over the set of blocks for which there are no replicas.
    */
-  UnderReplicatedBlocks.BlockIterator getCorruptReplicaBlockIterator() {
+  public BlockIterator getCorruptReplicaBlockIterator() {
     return neededReplications
         .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
  * 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.Collection;
@@ -26,6 +26,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.Node;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -94,7 +96,7 @@ public abstract class BlockPlacementPolicy {
    * @return array of DatanodeDescriptor instances chosen as target
    * and sorted as a pipeline.
    */
-  abstract DatanodeDescriptor[] chooseTarget(String srcPath,
+  public abstract DatanodeDescriptor[] chooseTarget(String srcPath,
                                              int numOfReplicas,
                                              DatanodeDescriptor writer,
                                              List<DatanodeDescriptor> chosenNodes,
@@ -222,11 +224,11 @@ public abstract class BlockPlacementPolicy {
    * @param numOfReplicas number of replicas wanted.
    * @param writer the writer's machine, null if not in the cluster.
    * @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
    * and sorted as a pipeline.
    */
-  DatanodeDescriptor[] chooseTarget(String srcPath,
+  public DatanodeDescriptor[] chooseTarget(String srcPath,
                                     int numOfReplicas,
                                     DatanodeDescriptor writer,
                                     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
  * 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.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.FSConstants;
 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.Node;
 import org.apache.hadoop.net.NodeBase;
@@ -89,7 +92,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
 
   /** {@inheritDoc} */
   @Override
-  DatanodeDescriptor[] chooseTarget(String srcPath,
+  public DatanodeDescriptor[] chooseTarget(String srcPath,
                                     int numOfReplicas,
                                     DatanodeDescriptor writer,
                                     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
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.util.Iterator;
 
 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.LightWeightGSet;
 
@@ -28,7 +29,7 @@ import org.apache.hadoop.hdfs.util.LightWeightGSet;
  * block's metadata currently includes INode it belongs to and
  * the datanodes that store the block.
  */
-class BlocksMap {
+public class BlocksMap {
   private static class NodeIterator implements Iterator<DatanodeDescriptor> {
     private BlockInfo blockInfo;
     private int nextIdx = 0;
@@ -100,7 +101,7 @@ class BlocksMap {
   /**
    * 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);
     if (info != b) {
       info = b;
@@ -136,7 +137,7 @@ class BlocksMap {
    * Searches for the block in the BlocksMap and 
    * 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));
   }
 
@@ -185,7 +186,7 @@ class BlocksMap {
   /**
    * Check if the block exists in map
    */
-  boolean contains(Block block) {
+  public boolean contains(Block 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
  * 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.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.Server;
 
 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
  * 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.IOException;
@@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.io.WritableUtils;
@@ -44,7 +45,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   
   // Stores status of decommissioning.
   // 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 */
   @InterfaceAudience.Private
@@ -96,8 +97,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private int numBlocks = 0;
   // isAlive == heartbeats.contains(this)
   // 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 */
   private BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
@@ -204,7 +205,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * Add datanode to the block.
    * 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))
       return false;
     // 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 datanode from the block.
    */
-  boolean removeBlock(BlockInfo b) {
+  public boolean removeBlock(BlockInfo b) {
     blockList = b.listRemove(blockList, this);
     if ( b.removeNode(this) ) {
       numBlocks--;
@@ -242,7 +243,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * @param newBlock - a replacement block
    * @return the new block
    */
-  BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
+  public BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
     boolean done = removeBlock(oldBlock);
     assert done : "Old block should belong to the data-node when replacing";
     done = addBlock(newBlock);
@@ -250,7 +251,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return newBlock;
   }
 
-  void resetBlocks() {
+  public void resetBlocks() {
     this.capacity = 0;
     this.remaining = 0;
     this.blockPoolUsed = 0;
@@ -268,7 +269,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * 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) {
     this.capacity = capacity;
     this.dfsUsed = dfsUsed;
@@ -283,7 +284,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * 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 DatanodeDescriptor node;
       
@@ -307,7 +308,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
-  Iterator<BlockInfo> getBlockIterator() {
+  public Iterator<BlockInfo> getBlockIterator() {
     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);
   }
 
-  BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
+  public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
     List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
       return null;
@@ -375,7 +376,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Remove the specified number of blocks to be invalidated
    */
-  Block[] getInvalidateBlocks(int maxblocks) {
+  public Block[] getInvalidateBlocks(int maxblocks) {
     return getBlockArray(invalidateBlocks, maxblocks); 
   }
 
@@ -418,7 +419,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /** Serialization for FSEditLog */
-  void readFieldsFromFSEditLog(DataInput in) throws IOException {
+  public void readFieldsFromFSEditLog(DataInput in) throws IOException {
     this.name = DeprecatedUTF8.readString(in);
     this.storageID = DeprecatedUTF8.readString(in);
     this.infoPort = in.readShort() & 0x0000ffff;
@@ -445,7 +446,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Increments counter for number of blocks scheduled. 
    */
-  void incBlocksScheduled() {
+  public void incBlocksScheduled() {
     currApproxBlocksScheduled++;
   }
   
@@ -485,12 +486,13 @@ public class DatanodeDescriptor extends DatanodeInfo {
     // by DatanodeID
     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,
         int onlyRep, int underConstruction) {
@@ -501,32 +503,34 @@ public class DatanodeDescriptor extends DatanodeInfo {
       decommissionOnlyReplicas = onlyRep;
       underReplicatedInOpenFiles = underConstruction;
     }
-    
-    synchronized int getUnderReplicatedBlocks() {
+
+    /** @return the number of under-replicated blocks */
+    public synchronized int getUnderReplicatedBlocks() {
       if (isDecommissionInProgress() == false) {
         return 0;
       }
       return underReplicatedBlocks;
     }
-    synchronized int getDecommissionOnlyReplicas() {
+    /** @return the number of decommission-only replicas */
+    public synchronized int getDecommissionOnlyReplicas() {
       if (isDecommissionInProgress() == false) {
         return 0;
       }
       return decommissionOnlyReplicas;
     }
-
-    synchronized int getUnderReplicatedInOpenFiles() {
+    /** @return the number of under-replicated blocks in open files */
+    public synchronized int getUnderReplicatedInOpenFiles() {
       if (isDecommissionInProgress() == false) {
         return 0;
       }
       return underReplicatedInOpenFiles;
     }
-
-    synchronized void setStartTime(long time) {
+    /** Set start time */
+    public synchronized void setStartTime(long time) {
       startTime = time;
     }
-    
-    synchronized long getStartTime() {
+    /** @return start time */
+    public synchronized long getStartTime() {
       if (isDecommissionInProgress() == false) {
         return 0;
       }
@@ -538,11 +542,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * Set the flag to indicate if this datanode is disallowed from communicating
    * with the namenode.
    */
-  void setDisallowed(boolean flag) {
+  public void setDisallowed(boolean flag) {
     disallowed = flag;
   }
-  
-  boolean isDisallowed() {
+  /** Is the datanode disallowed from communicating with the namenode? */
+  public boolean isDisallowed() {
     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
  * 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.server.namenode.FSNamesystem;
+
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 import org.apache.hadoop.util.*;
 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
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.util.*;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 
 /* Class for keeping track of under replication blocks
  * Blocks have replication priority, with priority 0 indicating 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 public final int QUEUE_WITH_CORRUPT_BLOCKS = 4;
   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 */
-  synchronized int size() {
+  public synchronized int size() {
     int size = 0;
     for (int i=0; i<LEVEL; i++) {
       size += priorityQueues.get(i).size();
@@ -70,7 +71,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
   
   /* 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) {
       if(set.contains(block)) { return true; }
     }
@@ -218,7 +219,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
     return new BlockIterator();
   }
   
-  class BlockIterator implements Iterator<Block> {
+  public class BlockIterator implements Iterator<Block> {
     private int level;
     private boolean isIteratorForLevel = false;
     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.URL;
 import java.net.URLEncoder;
-import java.util.Arrays;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -45,14 +45,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.BlockReader;
 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.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.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.http.HtmlQuoting;
 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) {
       ioe = e;
     }
+    finally {
+      IOUtils.closeStream(checksumOut);
+    }
     // close block file
     try {
       if (out != null) {
@@ -244,6 +247,9 @@ class BlockReceiver implements Closeable, FSConstants {
     } catch (IOException e) {
       ioe = e;
     }
+    finally{
+      IOUtils.closeStream(out);
+    }
     // disk check
     if(ioe != null) {
       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));
         }
 
-        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
         blockSender.sendBlock(out, baseStream, null);
@@ -2186,20 +2186,21 @@ public class DataNode extends Configured
         continue;
       }
       // drop any (illegal) authority in the URI for backwards compatibility
-      File data = new File(dirURI.getPath());
+      File dir = new File(dirURI.getPath());
       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 "
           + DFS_DATANODE_DATA_DIR_KEY + " are invalid: "
           + invalidDirs);
+    }
     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
   
   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.isLocal = s.getInetAddress().equals(s.getLocalAddress());
     this.datanode = datanode;
@@ -122,18 +125,14 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
   DataNode getDataNode() {return datanode;}
 
   /**
-   * Read/write data from/to the DataXceiveServer.
+   * Read/write data from/to the DataXceiverServer.
    */
   public void run() {
     updateCurrentThreadName("Waiting for operation");
 
-    DataInputStream in=null; 
     int opsProcessed = 0;
     Op op = null;
     try {
-      in = new DataInputStream(
-          new BufferedInputStream(NetUtils.getInputStream(s), 
-                                  SMALL_BUFFER_SIZE));
       int stdTimeout = s.getSoTimeout();
 
       // 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;
             s.setSoTimeout(socketKeepaliveTimeout);
           }
-          op = readOp(in);
+          op = readOp();
         } catch (InterruptedIOException ignored) {
           // Time out while we wait for client rpc
           break;
@@ -176,7 +175,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
         }
 
         opStartTime = now();
-        processOp(op, in);
+        processOp(op);
         ++opsProcessed;
       } while (!s.isClosed() && socketKeepaliveTimeout > 0);
     } catch (Throwable t) {
@@ -196,13 +195,12 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
     }
   }
 
-  /**
-   * Read a block from the disk.
-   */
   @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, 
         datanode.socketWriteTimeout);
     DataOutputStream out = new DataOutputStream(
@@ -225,7 +223,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
     updateCurrentThreadName("Sending block " + block);
     try {
       try {
-        blockSender = new BlockSender(block, startOffset, length,
+        blockSender = new BlockSender(block, blockOffset, length,
             true, true, false, datanode, clientTraceFmt);
       } catch(IOException e) {
         LOG.info("opReadBlock " + block + " received exception " + e);
@@ -284,16 +282,17 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
     datanode.metrics.incrReadsFromClient(isLocal);
   }
 
-  /**
-   * Write a block to disk.
-   */
   @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);
     final boolean isDatanode = clientname.length() == 0;
     final boolean isClient = !isDatanode;
@@ -308,7 +307,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("opWriteBlock: stage=" + stage + ", clientname=" + clientname 
-      		+ "\n  block  =" + block + ", newGs=" + newGs
+      		+ "\n  block  =" + block + ", newGs=" + latestGenerationStamp
       		+ ", bytesRcvd=[" + minBytesRcvd + ", " + maxBytesRcvd + "]"
           + "\n  targets=" + Arrays.asList(targets)
           + "; pipelineSize=" + pipelineSize + ", srcDataNode=" + srcDataNode
@@ -351,10 +350,10 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
         blockReceiver = new BlockReceiver(block, in, 
             s.getRemoteSocketAddress().toString(),
             s.getLocalSocketAddress().toString(),
-            stage, newGs, minBytesRcvd, maxBytesRcvd,
+            stage, latestGenerationStamp, minBytesRcvd, maxBytesRcvd,
             clientname, srcDataNode, datanode);
       } 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));
           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
             blockReceiver.writeChecksumHeader(mirrorOut);
@@ -464,7 +463,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
       // update its generation stamp
       if (isClient && 
           stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
-        block.setGenerationStamp(newGs);
+        block.setGenerationStamp(latestGenerationStamp);
         block.setNumBytes(minBytesRcvd);
       }
       
@@ -499,10 +498,10 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
   }
 
   @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,
         Op.TRANSFER_BLOCK, BlockTokenSecretManager.AccessMode.COPY);
 
@@ -511,19 +510,16 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
     final DataOutputStream out = new DataOutputStream(
         NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
     try {
-      datanode.transferReplicaForPipelineRecovery(blk, targets, client);
+      datanode.transferReplicaForPipelineRecovery(blk, targets, clientName);
       writeResponse(Status.SUCCESS, out);
     } finally {
       IOUtils.closeStream(out);
     }
   }
   
-  /**
-   * Get block checksum (MD5 of CRC32).
-   */
   @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(
         NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
     checkAccess(out, true, block, blockToken,
@@ -572,12 +568,9 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
     datanode.metrics.addBlockChecksumOp(elapsed());
   }
 
-  /**
-   * Read a block from the disk and then sends it to a destination.
-   */
   @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);
     // Read in the header
     if (datanode.isBlockTokenEnabled) {
@@ -647,15 +640,12 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
     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
-  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 */
     block.setNumBytes(dataXceiverServer.estimateBlockSize);
@@ -699,7 +689,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
                      new BufferedOutputStream(baseStream, SMALL_BUFFER_SIZE));
 
       /* send request to the proxy */
-      Sender.opCopyBlock(proxyOut, block, blockToken);
+      new Sender(proxyOut).copyBlock(block, blockToken);
 
       // receive the response from the proxy
       proxyReply = new DataInputStream(new BufferedInputStream(
@@ -727,7 +717,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
           dataXceiverServer.balanceThrottler, null);
                     
       // notify name node
-      datanode.notifyNamenodeReceivedBlock(block, sourceID);
+      datanode.notifyNamenodeReceivedBlock(block, delHint);
 
       LOG.info("Moved block " + block + 
           " 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.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.server.balancer.Balancer;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Daemon;
 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));
   }
 
-  /**
-   */
+  @Override
   public void run() {
     while (datanode.shouldRun) {
       try {
         Socket s = ss.accept();
         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) {
         // wake up to see if should continue to run
       } catch (IOException ie) {
-        LOG.warn(datanode.getMachineName() + ":DataXceiveServer: ", ie);
+        LOG.warn(datanode.getMachineName() + ":DataXceiverServer: ", ie);
       } catch (Throwable te) {
         LOG.error(datanode.getMachineName()
-            + ":DataXceiveServer: Exiting due to: ", te);
+            + ":DataXceiverServer: Exiting due to: ", te);
         datanode.shouldRun = false;
       }
     }
@@ -151,7 +157,7 @@ class DataXceiverServer implements Runnable, FSConstants {
       ss.close();
     } catch (IOException ie) {
       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 {
       this.ss.close();
     } catch (IOException ie) {
-      LOG.warn(datanode.getMachineName() + ":DataXceiveServer.kill(): " 
+      LOG.warn(datanode.getMachineName() + ":DataXceiverServer.kill(): "
                               + 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,
                   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;
 

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

@@ -17,8 +17,12 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.BufferedReader;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.net.InetSocketAddress;
+import java.net.URL;
+import java.net.URLConnection;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -27,13 +31,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
-import javax.management.JMX;
-import javax.management.MBeanServerConnection;
 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.LogFactory;
@@ -43,24 +41,27 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.util.StringUtils;
+import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.type.TypeReference;
 import org.znerd.xmlenc.XMLOutputter;
 
 /**
  * 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
 class ClusterJspHelper {
   private static final Log LOG = LogFactory.getLog(ClusterJspHelper.class);
   public static final String OVERALL_STATUS = "overall-status";
   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 
    * 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 cs = new ClusterStatus();
@@ -79,26 +80,24 @@ class ClusterJspHelper {
       NamenodeMXBeanHelper nnHelper = null;
       try {
         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
-          cs.clusterid = nnHelper.getClusterId();
+          cs.clusterid = nnHelper.getClusterId(mbeanProps);
         }
         cs.addNamenodeStatus(nn);
       } catch ( Exception e ) {
         // track exceptions encountered when connecting to namenodes
         cs.addException(isa.getHostName(), e);
         continue;
-      } finally {
-        if (nnHelper != null) {
-          nnHelper.cleanup();
-        }
-      }
+      } 
     }
     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() {
     String clusterid = "";
@@ -127,21 +126,18 @@ class ClusterJspHelper {
       NamenodeMXBeanHelper nnHelper = null;
       try {
         nnHelper = new NamenodeMXBeanHelper(isa, conf);
+        String mbeanProps= queryMbean(nnHelper.httpAddress, conf);
         if (clusterid.equals("")) {
-          clusterid = nnHelper.getClusterId();
+          clusterid = nnHelper.getClusterId(mbeanProps);
         }
-        nnHelper.getDecomNodeInfoForReport(statusMap);
+        nnHelper.getDecomNodeInfoForReport(statusMap, mbeanProps);
       } catch (Exception e) {
         // catch exceptions encountered while connecting to namenodes
         String nnHost = isa.getHostName();
         decommissionExceptions.put(nnHost, e);
         unreportedNamenode.add(nnHost);
         continue;
-      } finally {
-        if (nnHelper != null) {
-          nnHelper.cleanup();
-        }
-      }
+      } 
     }
     updateUnknownStatus(statusMap, unreportedNamenode);
     getDecommissionNodeClusterState(statusMap);
@@ -260,40 +256,20 @@ class ClusterJspHelper {
     }
     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 {
     private static final ObjectMapper mapper = new ObjectMapper();
-    private final InetSocketAddress rpcAddress;
     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)
         throws IOException, MalformedObjectNameException {
-      this.rpcAddress = addr;
       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 */
@@ -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
      */
     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 json JSON string returned from JMX call
+     * @param json JSON string
      */
     private static void getDeadNodeCount(String json, NamenodeStatus nn)
         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();
       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;
     }
     
     /**
-     * Connect to namenode to get decommission node information.
+     * Get the decommission node information.
      * @param statusMap data node status map
-     * @param connector JMXConnector
+     * @param props string
      */
     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
      *          is an inner map whose key is namenode, value is datanode status.
      *          reported by each 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
      * @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
      *          inner map (key:namenode, value:decommisionning state).
      * @param host datanode hostname
-     * @param decomnode
-     * @param json
+     * @param decomnode DecommissionNode
+     * @param json String
      * @throws IOException
      */
     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
      *          inner map (key:namenode, value:decommisionning state).
      * @param host datanode
-     * @param decomnode Decommissionnode
-     * @param json JSON string returned from JMX connection
+     * @param decomnode DecommissionNode
+     * @param json String
      */
     private static void getDecommissionNodeStatus(
         Map<String, Map<String, String>> dataNodeStatusMap, String host,
@@ -508,19 +482,6 @@ class ClusterJspHelper {
         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(); // 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.LogFactory;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 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
     // 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;
   }
   

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

@@ -17,40 +17,47 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.server.common.Util.now;
+
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 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.locks.Condition;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 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.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.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 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.FSConstants;
 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.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 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.StartupOption;
 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.
@@ -61,7 +68,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
  * and logged to disk.
  * 
  *************************************************/
-class FSDirectory implements Closeable {
+public class FSDirectory implements Closeable {
 
   INodeDirectoryWithQuota rootDir;
   FSImage fsImage;  
@@ -1337,7 +1344,7 @@ class FSDirectory implements Closeable {
    * @throws QuotaExceededException if the new count violates any quota limit
    * @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,
                                                 FileNotFoundException,
                                                 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;
 
+import static org.apache.hadoop.hdfs.server.common.Util.now;
+
 import java.io.BufferedInputStream;
 import java.io.DataInputStream;
 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.LayoutVersion;
 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.namenode.FSEditLogOp.LogHeader;
 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.FSEditLogOp.*;
 
 public class FSEditLogLoader {
   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.LayoutVersion;
 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.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.");
     }
     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();
   }
 

+ 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.LayoutVersion;
 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.InconsistentFSStateException;
 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.DataOutputStream;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 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.protocol.Block;
 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.io.Text;
 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.net.InetAddress;
 import java.net.URI;
+import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
 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.delegation.DelegationTokenIdentifier;
 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.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 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.UnderReplicatedBlocks.BlockIterator;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
@@ -235,7 +241,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   // Stores the correct file name hierarchy
   //
   public FSDirectory dir;
-  BlockManager blockManager;
+  public BlockManager blockManager;
   
   // Block pool ID used by this namenode
   String blockPoolId;
@@ -270,10 +276,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * Stores a set of DatanodeDescriptor objects.
    * This is a subset of {@link #datanodeMap}, containing nodes that are 
    * considered alive.
-   * The {@link HeartbeatMonitor} periodically checks for outdated entries,
+   * The HeartbeatMonitor periodically checks for out-dated entries,
    * 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); 
 
@@ -314,8 +320,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   private volatile SafeModeInfo safeMode;  // safe mode information
   private Host2NodesMap host2DataNodeMap = new Host2NodesMap();
     
-  // datanode networktoplogy
-  NetworkTopology clusterMap = new NetworkTopology();
+  /** datanode network toplogy */
+  public NetworkTopology clusterMap = new NetworkTopology();
   private DNSToSwitchMapping dnsToSwitchMapping;
 
   private HostsFileReader hostsReader; 
@@ -329,7 +335,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   private final GenerationStamp generationStamp = new GenerationStamp();
 
   // 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.
   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
-  void readLock() {
+  public void readLock() {
     this.fsLock.readLock().lock();
   }
 
-  void readUnlock() {
+  public void readUnlock() {
     this.fsLock.readLock().unlock();
   }
 
-  void writeLock() {
+  public void writeLock() {
     this.fsLock.writeLock().lock();
   }
 
-  void writeUnlock() {
+  public void writeUnlock() {
     this.fsLock.writeLock().unlock();
   }
 
-  boolean hasWriteLock() {
+  public boolean hasWriteLock() {
     return this.fsLock.isWriteLockedByCurrentThread();
   }
 
@@ -1014,7 +1020,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   }
 
   /** 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 {
     return new LocatedBlock(getExtendedBlock(b), locations, offset, corrupt);
   }
@@ -3013,7 +3019,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * @return an array of datanode commands 
    * @throws IOException
    */
-  DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
+  public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
       long capacity, long dfsUsed, long remaining, long blockPoolUsed,
       int xceiverCount, int xmitsInProgress, int failedVolumes) 
         throws IOException {
@@ -3521,7 +3527,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * If no such a node is available,
    * then pick a node with least free space
    */
-  void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess, 
+  public void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess, 
                               Block b, short replication,
                               DatanodeDescriptor addedNode,
                               DatanodeDescriptor delNodeHint,
@@ -3785,9 +3791,19 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
             nodes.add(dn);
           }
           //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;
   }
     
-  /**
-   * 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 
    * 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) {
     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) {
     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 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.
    * @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
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
@@ -4637,7 +4653,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   /**
    * 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
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
@@ -4648,7 +4664,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   /**
    * Check whether replication queues are populated.
    */
-  boolean isPopulatingReplQueues() {
+  public boolean isPopulatingReplQueues() {
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
@@ -4660,7 +4676,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * Increment number of blocks that reached minimal 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
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
@@ -4671,7 +4687,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   /**
    * 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
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null) // mostly true
@@ -4978,13 +4994,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   @Override // FSNamesystemMBean
   @Metric
   public long getPendingReplicationBlocks() {
-    return blockManager.pendingReplicationBlocksCount;
+    return blockManager.getPendingReplicationBlocksCount();
   }
 
   @Override // FSNamesystemMBean
   @Metric
   public long getUnderReplicatedBlocks() {
-    return blockManager.underReplicatedBlocksCount;
+    return blockManager.getUnderReplicatedBlocksCount();
   }
 
   /** 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 */
   @Metric({"CorruptBlocks", "Number of blocks with corrupt replicas"})
   public long getCorruptReplicaBlocks() {
-    return blockManager.corruptReplicaBlocksCount;
+    return blockManager.getCorruptReplicaBlocksCount();
   }
 
   @Override // FSNamesystemMBean
   @Metric
   public long getScheduledReplicationBlocks() {
-    return blockManager.scheduledReplicationBlocksCount;
+    return blockManager.getScheduledReplicationBlocksCount();
   }
 
   @Metric
   public long getPendingDeletionBlocks() {
-    return blockManager.pendingDeletionBlocksCount;
+    return blockManager.getPendingDeletionBlocksCount();
   }
 
   @Metric
   public long getExcessBlocks() {
-    return blockManager.excessBlocksCount;
+    return blockManager.getExcessBlocksCount();
   }
   
   @Metric
@@ -5380,7 +5396,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   }
 
   /** Get a datanode descriptor given corresponding storageID */
-  DatanodeDescriptor getDatanode(String nodeID) {
+  public DatanodeDescriptor getDatanode(String nodeID) {
     assert hasReadOrWriteLock();
     return datanodeMap.get(nodeID);
   }
@@ -5444,7 +5460,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
       if (startBlockAfter != null) {
         startBlockId = Block.filename2id(startBlockAfter);
       }
-      BlockIterator blkIterator = blockManager.getCorruptReplicaBlockIterator();
+      UnderReplicatedBlocks.BlockIterator blkIterator = blockManager.getCorruptReplicaBlockIterator();
       while (blkIterator.hasNext()) {
         Block blk = blkIterator.next();
         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);
 
           HdfsFileStatus info = nn.getFileInfo(path);
-          if ((info != null) && !info.isDir()) {
+          if (info != null && !info.isDir()) {
             try {
               response.sendRedirect(createUri(path, info, ugi, nn, request,
                   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;
 
-import java.util.*;
+import java.util.HashMap;
+import java.util.Random;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+
 class Host2NodesMap {
   private HashMap<String, DatanodeDescriptor[]> map
     = 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.List;
 
-import org.apache.hadoop.fs.Path;
 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.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 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 
  * 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 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?
    */
-  boolean isUnderConstruction() {
+  public boolean isUnderConstruction() {
     return false;
   }
 

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

@@ -409,20 +409,31 @@ class INodeDirectory extends INode {
 
   /** {@inheritDoc} */
   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) {
       for (INode child : children) {
-        child.computeContentSummary(summary);
+        child.computeContentSummary(subtreeSummary);
       }
     }
     if (this instanceof INodeDirectoryWithQuota) {
       // Warn if the cached and computed diskspace values differ
       INodeDirectoryWithQuota node = (INodeDirectoryWithQuota)this;
       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 "
-            +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]++;
     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.PermissionStatus;
 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);
 
   //Number of bits for Block size
@@ -106,7 +109,7 @@ class INodeFile extends INode {
    * Get file blocks 
    * @return file blocks
    */
-  BlockInfo[] getBlocks() {
+  public BlockInfo[] getBlocks() {
     return this.blocks;
   }
 
@@ -149,7 +152,7 @@ class INodeFile extends INode {
   /**
    * Set file block
    */
-  void setBlock(int idx, BlockInfo blk) {
+  public void setBlock(int idx, BlockInfo blk) {
     this.blocks[idx] = blk;
   }
 
@@ -237,7 +240,7 @@ class INodeFile extends INode {
    * Get the last block of the file.
    * 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)
       return null;
     T returnBlock = null;
@@ -252,7 +255,8 @@ class INodeFile extends INode {
     return returnBlock;
   }
 
-  int numBlocks() {
+  /** @return the number of blocks */ 
+  public int numBlocks() {
     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.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;
 
-
-class INodeFileUnderConstruction extends INodeFile {
+/**
+ * I-node for file being written.
+ */
+public class INodeFileUnderConstruction extends INodeFile {
   private  String clientName;         // lease holder
   private final String clientMachine;
   private final DatanodeDescriptor clientNode; // if client is a cluster node too.
@@ -43,7 +48,7 @@ class INodeFileUnderConstruction extends INodeFile {
     this.clientNode = clientNode;
   }
 
-  public INodeFileUnderConstruction(byte[] name,
+  INodeFileUnderConstruction(byte[] name,
                              short blockReplication,
                              long modificationTime,
                              long preferredBlockSize,
@@ -80,7 +85,7 @@ class INodeFileUnderConstruction extends INodeFile {
    * Is this inode being constructed?
    */
   @Override
-  boolean isUnderConstruction() {
+  public boolean isUnderConstruction() {
     return true;
   }
 
@@ -122,7 +127,7 @@ class INodeFileUnderConstruction extends INodeFile {
    * Convert the last block of the file to an under-construction block.
    * Set its locations.
    */
-  BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
+  public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
                                           DatanodeDescriptor[] targets)
   throws IOException {
     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
    */
-  synchronized String findPath(INodeFileUnderConstruction pendingFile)
+  public synchronized String findPath(INodeFileUnderConstruction pendingFile)
       throws IOException {
     Lease lease = getLease(pendingFile.getClientName());
     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
         LOG.warn("writeTransactionIdToStorage failed on " + sd,
             e);
+        reportErrorsOnDirectory(sd);
       }
     }
   }
@@ -828,17 +829,17 @@ public class NNStorage extends Storage implements Closeable {
    * @throws IOException
    */
   void reportErrorsOnDirectory(StorageDirectory sd) {
-    LOG.warn("Error reported on storage directory " + sd);
+    LOG.error("Error reported on storage directory " + sd);
 
     String lsd = listStorageDirectories();
     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());
     try {
       sd.unlock();
     } catch (Exception e) {
-      LOG.info("Unable to unlock bad storage directory: "
+      LOG.warn("Unable to unlock bad storage directory: "
                +  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 {@link FSNamesystem} object.
    */
-  FSNamesystem getNamesystem() {
+  public FSNamesystem getNamesystem() {
     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
     this.rpcAddress = this.server.getListenerAddress(); 
     setRpcServerAddress(conf);
+    
+    try {
+      validateConfigurationSettings(conf);
+    } catch (IOException e) {
+      LOG.fatal(e.toString());
+      throw e;
+    }
 
     activate(conf);
     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); 
     }
   }
+  
+  /**
+   * 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.

+ 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.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.BlockReader;
 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.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NodeBase;
-import org.apache.hadoop.security.AccessControlException;
 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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSUtil;
 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.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
 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.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.UpgradeStatusReport;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.ServletUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
-
-import org.znerd.xmlenc.*;
+import org.znerd.xmlenc.XMLOutputter;
 
 class NamenodeJspHelper {
   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.setAttribute("secondary.name.node", this);
+          infoServer.setAttribute("secondary.name.node", SecondaryNameNode.this);
           infoServer.setAttribute("name.system.image", checkpointImage);
           infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);
           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.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeJspHelper;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.mortbay.jetty.InclusiveByteRange;
 
@@ -46,7 +47,7 @@ public class StreamFile extends DfsServlet {
 
   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)
       throws IOException, InterruptedException {
     final Configuration conf =
@@ -57,6 +58,7 @@ public class StreamFile extends DfsServlet {
     return DatanodeJspHelper.getDFSClient(request, datanode, conf, ugi);
   }
   
+  @SuppressWarnings("unchecked")
   public void doGet(HttpServletRequest request, HttpServletResponse response)
     throws ServletException, IOException {
     final String path = request.getPathInfo() != null ? 
@@ -69,9 +71,10 @@ public class StreamFile extends DfsServlet {
       return;
     }
     
-    Enumeration<?> reqRanges = request.getHeaders("Range");
-    if (reqRanges != null && !reqRanges.hasMoreElements())
+    Enumeration<String> reqRanges = request.getHeaders("Range");
+    if (reqRanges != null && !reqRanges.hasMoreElements()) {
       reqRanges = null;
+    }
 
     DFSClient dfs;
     try {
@@ -81,107 +84,82 @@ public class StreamFile extends DfsServlet {
       return;
     }
     
-    final DFSInputStream in = dfs.open(filename);
-    final long fileLen = in.getFileLength();
-    OutputStream os = response.getOutputStream();
+    DFSInputStream in = null;
+    OutputStream out = null;
 
     try {
+      in = dfs.open(filename);
+      out = response.getOutputStream();
+      final long fileLen = in.getFileLength();
       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 {
         // No ranges, so send entire file
         response.setHeader("Content-Disposition", "attachment; filename=\"" + 
                            filename + "\"");
         response.setContentType("application/octet-stream");
         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()) {
-        LOG.debug("response.isCommitted()=" + response.isCommitted(), e);
+        LOG.debug("response.isCommitted()=" + response.isCommitted(), ioe);
       }
-      throw e;
+      throw ioe;
     } 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,
-                              OutputStream os,
+                              OutputStream out,
                               HttpServletResponse response,
                               long contentLength,
-                              List<?> ranges)
-  throws IOException {
-
+                              List<InclusiveByteRange> ranges)
+      throws IOException {
     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);
-      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));
     } 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);
       response.setStatus(HttpServletResponse.SC_PARTIAL_CONTENT);
       response.setHeader("Content-Range", 
         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;
 
-import java.io.*;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 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):
-    call(Op Receiver.readOp(DataInputStream)) && target(dataxceiver);
+    call(Op Receiver.readOp()) && target(dataxceiver);
 
   after(DataXceiver dataxceiver) returning(Op op): receiverOp(dataxceiver) {
     LOG.info("FI: receiverOp " + op + ", datanode="

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

@@ -15440,7 +15440,7 @@
       <comparators>
         <comparator>
           <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>
       </comparators>
     </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.proto.DataTransferProtos.BlockOpResponseProto;
 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.datanode.DataNode;
 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.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@@ -683,8 +683,8 @@ public class DFSTestUtil {
     final DataInputStream in = new DataInputStream(NetUtils.getInputStream(s));
 
     // 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();
 
     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.
    * <p>
    * 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
    *          will be modified as necessary.
@@ -381,7 +382,8 @@ public class MiniDFSCluster {
   
   /**
    * 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>
    * 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
    *          will be modified as necessary.
    * @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
-   *          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
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @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 format if true, format the NameNode and DataNodes before starting up
    * @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
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @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 format if true, format the NameNode and DataNodes before starting up
    * @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
-   *          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
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @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.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @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
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @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.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @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
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @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.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @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
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @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.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @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
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @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.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @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
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @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.FileSystem;
 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

+ 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.MalformedURLException;
 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.URLOpener;
 
+import org.junit.Test;
+import static org.junit.Assert.*;
+
 class MockHttpURLConnection extends HttpURLConnection {
   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 {
     MockURL o = new MockURL("http://test/");
     MockURL r =  new MockURL((URL)null);
@@ -149,7 +146,7 @@ public class TestByteRangeInputStream extends TestCase {
     is.seek(101);
     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);
     is.seek(2500);
@@ -168,7 +165,7 @@ public class TestByteRangeInputStream extends TestCase {
            + "but 206 is expected");
     } catch (IOException e) {
       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;
@@ -180,10 +177,7 @@ public class TestByteRangeInputStream extends TestCase {
            + "but 200 is expected");
     } catch (IOException e) {
       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;
 
 import java.io.IOException;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
@@ -31,7 +30,6 @@ 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.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -68,7 +66,6 @@ public class TestDFSPermission extends TestCase {
   final static private int NUM_TEST_PERMISSIONS = 
     conf.getInt("test.dfs.permission.num", 10) * (MAX_PERMISSION + 1) / 100;
 
-
   final private static String PATH_NAME = "xx";
   final private static Path FILE_DIR_PATH = new Path("/", 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();
     try {
       cluster.waitActive();
-      fs = FileSystem.get(conf);
-      LOG.info("ROOT=" + fs.getFileStatus(new Path("/")));
       testPermissionSetting(OpType.CREATE); // test file creation
       testPermissionSetting(OpType.MKDIRS); // test directory creation
     } finally {
-      fs.close();
       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 */
   private void testPermissionSetting(OpType op) throws Exception {
+    short uMask = DEFAULT_UMASK;
     // case 1: use default permission but all possible umasks
     PermissionGenerator generator = new PermissionGenerator(r);
+    FsPermission permission = new FsPermission(DEFAULT_PERMISSION);
     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
-    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
-    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
-    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);
     checkPermission(FILE_DIR_PATH, expectedPermission, true);
+    closeFileSystem();
 
     // 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();
     checkPermission(parent, getPermission(parent.getParent()), true);
+    closeFileSystem();
   }
 
   /* 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
-   * 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
    * will be created and populated.
    * 
@@ -145,7 +146,8 @@ public class TestDFSStorageStateRecovery extends TestCase {
   
   /**
    * 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 
    * will be created and populated. 
    * See {@link UpgradeUtilities#createDataNodeStorageDirs()}
@@ -172,7 +174,8 @@ public class TestDFSStorageStateRecovery extends TestCase {
   
   /**
    * 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 
    * will be created and populated. 
    * 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.IOException;
-
-import junit.framework.TestCase;
+import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 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.StorageInfo;
 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.getImageFileName;
 
+import org.apache.hadoop.util.StringUtils;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import com.google.common.base.Joiner;
@@ -51,8 +52,7 @@ import static org.junit.Assert.*;
 public class TestDFSUpgrade {
  
   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 int testCounter = 0;
   private MiniDFSCluster cluster = null;
@@ -125,11 +125,27 @@ public class TestDFSUpgrade {
       
     }
   }
+
   /**
    * Attempts to start a NameNode with the given operation.  Starting
    * the NameNode should throw an exception.
    */
   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 {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
                                                 .startupOption(operation)
@@ -137,9 +153,23 @@ public class TestDFSUpgrade {
                                                 .manageDataDfsDirs(false)
                                                 .manageNameDfsDirs(false)
                                                 .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();
   }
   
+  @BeforeClass
+  public static void initialize() throws Exception {
+    UpgradeUtilities.initialize();
+  }
+  
   /**
    * This test attempts to upgrade the NameNode and DataNode under
    * a number of valid and invalid conditions.
@@ -176,8 +211,6 @@ public class TestDFSUpgrade {
   @Test
   public void testUpgrade() throws Exception {
     File[] baseDirs;
-    UpgradeUtilities.initialize();
-    
     StorageInfo storageInfo = null;
     for (int numDirs = 1; numDirs <= 2; numDirs++) {
       conf = new HdfsConfiguration();
@@ -320,6 +353,30 @@ public class TestDFSUpgrade {
       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)
   throws Exception {
@@ -342,6 +399,7 @@ public class TestDFSUpgrade {
     fail("Expected IOException is not thrown");
   }
   
+  @Ignore
   public void test203LayoutVersion() {
     for (int lv : Storage.LAYOUT_VERSIONS_203) {
       assertTrue(Storage.is203LayoutVersion(lv));
@@ -349,7 +407,9 @@ public class TestDFSUpgrade {
   }
   
   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;
   InetSocketAddress dnAddr;
   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);
   DataOutputStream recvOut = new DataOutputStream(recvBuf);
 
@@ -185,9 +186,9 @@ public class TestDataTransferProtocol extends TestCase {
       String description, Boolean eofExcepted) throws IOException {
     sendBuf.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) {
       sendResponse(Status.ERROR, null, recvOut);
       sendRecvData(description, true);
@@ -372,10 +373,11 @@ public class TestDataTransferProtocol extends TestCase {
     
     /* Test OP_WRITE_BLOCK */
     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);
     
     // bad bytes per checksum
@@ -386,10 +388,10 @@ public class TestDataTransferProtocol extends TestCase {
 
     sendBuf.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.writeInt(512);
 
@@ -409,10 +411,10 @@ public class TestDataTransferProtocol extends TestCase {
     // test for writing a valid zero size block
     sendBuf.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.writeInt(512);         // checksum size
 
@@ -439,22 +441,22 @@ public class TestDataTransferProtocol extends TestCase {
     sendBuf.reset();
     recvBuf.reset();
     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); 
 
     // negative block start offset -1L
     sendBuf.reset();
     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 " + 
                  firstBlock.getBlockId(), false);
 
     // bad block start offset
     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 " +
                  firstBlock.getBlockId(), false);
     
@@ -462,8 +464,8 @@ public class TestDataTransferProtocol extends TestCase {
     recvBuf.reset();
     sendResponse(Status.SUCCESS, null, recvOut);
     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 " +
                  firstBlock.getBlockId(), false);
     
@@ -471,15 +473,15 @@ public class TestDataTransferProtocol extends TestCase {
     recvBuf.reset();
     sendResponse(Status.ERROR, null, recvOut);
     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 " +
                  firstBlock.getBlockId(), false);
     
     //At the end of all this, read the file to make sure that succeeds finally.
     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);
     } finally {
       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: 
    *    setQuota, clrQuota, setSpaceQuota, clrSpaceQuota, and count 
    */
@@ -695,7 +712,57 @@ public class TestQuota {
       // verify increase in space
       c = dfs.getContentSummary(dstPath);
       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 {
       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,
                                                          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>
    * with the content of namenode storage directory that comes from a singleton
    * 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>
    * with the content of datanode storage directory that comes from a singleton
    * 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>
    * with the content of block pool storage directory that comes from a singleton
    * 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
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 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 org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 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 dn2 = new DatanodeDescriptor();
-      DatanodeDescriptor dn3 = new DatanodeDescriptor();
       
       crm.addToCorruptReplicasMap(getBlock(0), dn1);
       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
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 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
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.io.IOException;
 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.server.common.GenerationStamp;
 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.DatanodeCommand;
 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
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import junit.framework.TestCase;
 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
  * limitations under the License.
  */
-
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -24,18 +23,19 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 
+import junit.framework.TestCase;
+
 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.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 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 {
   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
  * 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.fs.FileSystem;
@@ -25,8 +27,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 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 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);
     // sendRequest
     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();
     // receiveResponse
     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;
 
+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.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.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 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.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.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-import static org.junit.Assert.*;
-import static org.junit.Assume.assumeTrue;
 
 /**
  * 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
    */
-  private void restartCluster(int volTolerated, boolean manageCluster)
+  private void restartDatanodes(int volTolerated, boolean manageDfsDirs)
       throws IOException {
     //Make sure no datanode is running
     cluster.shutdownDataNodes();
     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();
   }
 
@@ -174,19 +174,14 @@ public class TestDataNodeVolumeFailureToleration {
 
   /**
    * 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,
-      boolean expectedBPServiceState, boolean clusterManaged)
+      boolean expectedBPServiceState, boolean manageDfsDirs)
       throws IOException, InterruptedException {
     assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
     final int dnIndex = 0;
+    // Fail the current directory since invalid storage directory perms
+    // get fixed up automatically on datanode startup.
     File[] dirs = {
         new File(MiniDFSCluster.getStorageDir(dnIndex, 0), "current"),
         new File(MiniDFSCluster.getStorageDir(dnIndex, 1), "current") };
@@ -195,11 +190,10 @@ public class TestDataNodeVolumeFailureToleration {
       for (int i = 0; i < volumesFailed; i++) {
         prepareDirToFail(dirs[i]);
       }
-      restartCluster(volumesTolerated, clusterManaged);
+      restartDatanodes(volumesTolerated, manageDfsDirs);
       assertEquals(expectedBPServiceState, cluster.getDataNodes().get(0)
           .isBPServiceAlive(cluster.getNamesystem().getBlockPoolId()));
     } finally {
-      // restore its old permission
       for (File dir : dirs) {
         FileUtil.chmod(dir.toString(), "755");
       }
@@ -215,8 +209,7 @@ public class TestDataNodeVolumeFailureToleration {
   private void prepareDirToFail(File dir) throws IOException,
       InterruptedException {
     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.
     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
     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.PermissionStatus;
 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.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 org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 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.

+ 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.LocatedBlock;
 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.junit.AfterClass;
 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.IOException;
 import java.net.URI;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Random;
@@ -173,6 +174,71 @@ public class TestCheckpoint extends TestCase {
     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.
    */

+ 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;
 
+import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 
-import junit.framework.TestCase;
-
 /**
  * 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.MiniDFSCluster;
 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.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 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;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Iterator;
 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.fs.FSDataOutputStream;
 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.protocol.DatanodeInfo;
 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.

+ 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;
 
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-
 import junit.framework.TestCase;
 
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+
 public class TestHost2NodesMap extends TestCase {
   static private Host2NodesMap map = new Host2NodesMap();
   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.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.fs.DF;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 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.concurrent.TimeoutException;
 
+import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 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.protocol.Block;
 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 

+ 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.IOException;
 
+import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.TestDatanodeBlockScanner;
 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.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-
-import junit.framework.TestCase;
 
 public class TestOverReplicatedBlocks extends TestCase {
   /** 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.IOException;
 import java.io.RandomAccessFile;
+import java.net.InetAddress;
 import java.net.URI;
+import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 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.PermissionStatus;
 
+import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 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.server.common.Storage.StorageDirectory;
 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.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.junit.Assert;
 
 /**
  * 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;
 
 
+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.EOFException;
 import java.io.File;
@@ -28,22 +32,23 @@ import java.security.MessageDigest;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Random;
+import java.util.Set;
 
 import static org.mockito.Matchers.anyByte;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
 
-import junit.framework.TestCase;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 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 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
  * 
  */
-public class TestStorageRestore extends TestCase {
+public class TestStorageRestore {
   public static final String NAME_NODE_HOST = "localhost:";
   public static final String NAME_NODE_HTTP_HOST = "0.0.0.0:";
   private static final Log LOG =
@@ -72,8 +81,8 @@ public class TestStorageRestore extends TestCase {
   static final int fileSize = 8192;
   private File path1, path2, path3;
   private MiniDFSCluster cluster;  
- 
-  protected void setUp() throws Exception {
+  @Before
+  public void setUpNameDirs() throws Exception {
     config = new HdfsConfiguration();
     hdfsDir = new File(MiniDFSCluster.getBaseDirectory()).getCanonicalFile();
     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
    */
-  public void invalidateStorage(FSImage fi) throws IOException {
+  public void invalidateStorage(FSImage fi, Set<File> filesToInvalidate) throws IOException {
     ArrayList<StorageDirectory> al = new ArrayList<StorageDirectory>(2);
     Iterator<StorageDirectory> it = fi.getStorage().dirIterator();
     while(it.hasNext()) {
       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);
       }
     }
@@ -140,7 +150,7 @@ public class TestStorageRestore extends TestCase {
    * test
    */
   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(); ) {
       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.
    */
   @SuppressWarnings("deprecation")
+  @Test
   public void testStorageRestore() throws Exception {
     int numDatanodes = 0;
     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...");
   
-    invalidateStorage(cluster.getNameNode().getFSImage());
+    invalidateStorage(cluster.getNameNode().getFSImage(), ImmutableSet.of(path2, path3));
     printStorages(cluster.getNameNode().getFSImage());
     System.out.println("****testStorageRestore: storage invalidated");
 
@@ -265,6 +276,7 @@ public class TestStorageRestore extends TestCase {
    * Test dfsadmin -restoreFailedStorage command
    * @throws Exception
    */
+  @Test
   public void testDfsAdminCmd() throws Exception {
     cluster = new MiniDFSCluster.Builder(config).
                                  numDataNodes(2).
@@ -296,7 +308,7 @@ public class TestStorageRestore extends TestCase {
       restore = fsi.getStorage().getRestoreFailedStorage();
       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";
       CommandExecutor.Result cmdResult = executor.executeCommand(cmd);
       restore = fsi.getStorage().getRestoreFailedStorage();
@@ -310,4 +322,67 @@ public class TestStorageRestore extends TestCase {
       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 java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Enumeration;
 import java.util.List;
 import java.util.Vector;
 
+import javax.servlet.ServletContext;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.http.HttpServletRequest;
 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.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;
 
 /*
@@ -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
   private static byte[] getOutputArray(int start, int count) {
     byte[] a = new byte[count];
@@ -200,6 +234,7 @@ public class TestStreamFile extends TestCase {
     return a;
   }
   
+  @Test
   public void testWriteTo() throws IOException, InterruptedException {
 
     FSInputStream fsin = new MockFSInputStream();
@@ -219,7 +254,7 @@ public class TestStreamFile extends TestCase {
     assertTrue("Pairs array must be even", pairs.length % 2 == 0);
     
     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]
                         + " bytes from offset " + pairs[i],
                         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});
     Enumeration<?> e = (new Vector<String>(l)).elements();
     return InclusiveByteRange.satisfiableRanges(e, contentLength);
   }
   
+  @Test
   public void testSendPartialData() throws IOException, InterruptedException {
     FSInputStream in = new MockFSInputStream();
     ByteArrayOutputStream os = new ByteArrayOutputStream();
 
     // 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();
       StreamFile.sendPartialData(in, os, response, 500, ranges);
       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
     { 
-      List<?> ranges = strToRanges("600-800", 500);
+      List<InclusiveByteRange> ranges = strToRanges("600-800", 500);
       MockHttpServletResponse response = new MockHttpServletResponse();
       StreamFile.sendPartialData(in, os, response, 500, ranges);
       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
     { 
-      List<?> ranges = strToRanges("100-300", 500);
+      List<InclusiveByteRange> ranges = strToRanges("100-300", 500);
       MockHttpServletResponse response = new MockHttpServletResponse();
       StreamFile.sendPartialData(in, os, response, 500, ranges);
       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;
 
+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.IOException;
 import java.util.Random;
 
 import junit.framework.TestCase;
 
+import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 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.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 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.NameNodeAdapter;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.test.MetricsAsserts;
 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
  */

+ 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 org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 
 public class TestNetworkTopology extends TestCase {
   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;
 
 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.LogFactory;
 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.protocol.AlreadyBeingCreatedException;
 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.NamenodeRole;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 import org.junit.After;
-
-import static org.junit.Assert.*;
 import org.junit.Before;
 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 {
   private static final Log LOG = LogFactory.getLog(TestNNLeaseRecovery.class);

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

@@ -59,19 +59,9 @@
 %>
 <%@ page
   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.common.JspHelper"
-  import="org.apache.hadoop.util.ServletUtil"
   import="org.znerd.xmlenc.*"
-  
 %>
 <%!
   //for java.io.Serializable