Bläddra i källkod

HADOOP-4687 Moving src directories on branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/HADOOP-4687/hdfs@776174 13f79535-47bb-0310-9956-ffa450edef68
Owen O'Malley 16 år sedan
incheckning
6a3ac690e4
100 ändrade filer med 38073 tillägg och 0 borttagningar
  1. 426 0
      src/java/hdfs-default.xml
  2. 3300 0
      src/java/org/apache/hadoop/hdfs/DFSClient.java
  3. 81 0
      src/java/org/apache/hadoop/hdfs/DFSUtil.java
  4. 522 0
      src/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  5. 50 0
      src/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java
  6. 322 0
      src/java/org/apache/hadoop/hdfs/HftpFileSystem.java
  7. 135 0
      src/java/org/apache/hadoop/hdfs/HsftpFileSystem.java
  8. 34 0
      src/java/org/apache/hadoop/hdfs/package.html
  9. 32 0
      src/java/org/apache/hadoop/hdfs/protocol/AlreadyBeingCreatedException.java
  10. 212 0
      src/java/org/apache/hadoop/hdfs/protocol/Block.java
  11. 127 0
      src/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
  12. 47 0
      src/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
  13. 489 0
      src/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  14. 61 0
      src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java
  15. 188 0
      src/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
  16. 351 0
      src/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  17. 86 0
      src/java/org/apache/hadoop/hdfs/protocol/FSConstants.java
  18. 148 0
      src/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
  19. 187 0
      src/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
  20. 67 0
      src/java/org/apache/hadoop/hdfs/protocol/QuotaExceededException.java
  21. 49 0
      src/java/org/apache/hadoop/hdfs/protocol/UnregisteredNodeException.java
  22. 1651 0
      src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  23. 114 0
      src/java/org/apache/hadoop/hdfs/server/common/GenerationStamp.java
  24. 83 0
      src/java/org/apache/hadoop/hdfs/server/common/HdfsConstants.java
  25. 47 0
      src/java/org/apache/hadoop/hdfs/server/common/InconsistentFSStateException.java
  26. 44 0
      src/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java
  27. 774 0
      src/java/org/apache/hadoop/hdfs/server/common/Storage.java
  28. 72 0
      src/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
  29. 89 0
      src/java/org/apache/hadoop/hdfs/server/common/UpgradeManager.java
  30. 66 0
      src/java/org/apache/hadoop/hdfs/server/common/UpgradeObject.java
  31. 130 0
      src/java/org/apache/hadoop/hdfs/server/common/UpgradeObjectCollection.java
  32. 124 0
      src/java/org/apache/hadoop/hdfs/server/common/UpgradeStatusReport.java
  33. 98 0
      src/java/org/apache/hadoop/hdfs/server/common/Upgradeable.java
  34. 28 0
      src/java/org/apache/hadoop/hdfs/server/common/Util.java
  35. 38 0
      src/java/org/apache/hadoop/hdfs/server/datanode/BlockAlreadyExistsException.java
  36. 130 0
      src/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  37. 1014 0
      src/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  38. 430 0
      src/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  39. 111 0
      src/java/org/apache/hadoop/hdfs/server/datanode/BlockTransferThrottler.java
  40. 985 0
      src/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
  41. 1683 0
      src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  42. 438 0
      src/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  43. 680 0
      src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  44. 175 0
      src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
  45. 138 0
      src/java/org/apache/hadoop/hdfs/server/datanode/DatanodeBlockInfo.java
  46. 314 0
      src/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  47. 1622 0
      src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
  48. 267 0
      src/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
  49. 151 0
      src/java/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java
  50. 134 0
      src/java/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java
  51. 76 0
      src/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeActivityMBean.java
  52. 135 0
      src/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
  53. 0 0
      src/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeStatistics.java
  54. 0 0
      src/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeStatisticsMBean.java
  55. 65 0
      src/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java
  56. 342 0
      src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  57. 388 0
      src/java/org/apache/hadoop/hdfs/server/namenode/BackupStorage.java
  58. 1375 0
      src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java
  59. 424 0
      src/java/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java
  60. 117 0
      src/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java
  61. 255 0
      src/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
  62. 137 0
      src/java/org/apache/hadoop/hdfs/server/namenode/CorruptReplicasMap.java
  63. 466 0
      src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java
  64. 93 0
      src/java/org/apache/hadoop/hdfs/server/namenode/DecommissionManager.java
  65. 98 0
      src/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
  66. 118 0
      src/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
  67. 207 0
      src/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
  68. 73 0
      src/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
  69. 187 0
      src/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
  70. 57 0
      src/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
  71. 116 0
      src/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
  72. 1289 0
      src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  73. 1434 0
      src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  74. 1880 0
      src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  75. 3726 0
      src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  76. 185 0
      src/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
  77. 102 0
      src/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
  78. 107 0
      src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
  79. 63 0
      src/java/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java
  80. 70 0
      src/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
  81. 188 0
      src/java/org/apache/hadoop/hdfs/server/namenode/Host2NodesMap.java
  82. 426 0
      src/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  83. 348 0
      src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  84. 164 0
      src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java
  85. 194 0
      src/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  86. 181 0
      src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  87. 56 0
      src/java/org/apache/hadoop/hdfs/server/namenode/JournalStream.java
  88. 429 0
      src/java/org/apache/hadoop/hdfs/server/namenode/JspHelper.java
  89. 32 0
      src/java/org/apache/hadoop/hdfs/server/namenode/LeaseExpiredException.java
  90. 399 0
      src/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  91. 178 0
      src/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java
  92. 1159 0
      src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  93. 618 0
      src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  94. 32 0
      src/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java
  95. 249 0
      src/java/org/apache/hadoop/hdfs/server/namenode/PendingReplicationBlocks.java
  96. 514 0
      src/java/org/apache/hadoop/hdfs/server/namenode/ReplicationTargetChooser.java
  97. 35 0
      src/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
  98. 585 0
      src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  99. 72 0
      src/java/org/apache/hadoop/hdfs/server/namenode/SerialNumberManager.java
  100. 85 0
      src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java

+ 426 - 0
src/java/hdfs-default.xml

@@ -0,0 +1,426 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!-- Do not modify this file directly.  Instead, copy entries that you -->
+<!-- wish to modify from this file into hdfs-site.xml and change them -->
+<!-- there.  If hdfs-site.xml does not already exist, create it.      -->
+
+<configuration>
+
+<property>
+  <name>dfs.namenode.logging.level</name>
+  <value>info</value>
+  <description>The logging level for dfs namenode. Other values are "dir"(trac
+e namespace mutations), "block"(trace block under/over replications and block
+creations/deletions), or "all".</description>
+</property>
+
+<property>
+  <name>dfs.secondary.http.address</name>
+  <value>0.0.0.0:50090</value>
+  <description>
+    The secondary namenode http server address and port.
+    If the port is 0 then the server will start on a free port.
+  </description>
+</property>
+
+<property>
+  <name>dfs.datanode.address</name>
+  <value>0.0.0.0:50010</value>
+  <description>
+    The address where the datanode server will listen to.
+    If the port is 0 then the server will start on a free port.
+  </description>
+</property>
+
+<property>
+  <name>dfs.datanode.http.address</name>
+  <value>0.0.0.0:50075</value>
+  <description>
+    The datanode http server address and port.
+    If the port is 0 then the server will start on a free port.
+  </description>
+</property>
+
+<property>
+  <name>dfs.datanode.ipc.address</name>
+  <value>0.0.0.0:50020</value>
+  <description>
+    The datanode ipc server address and port.
+    If the port is 0 then the server will start on a free port.
+  </description>
+</property>
+
+<property>
+  <name>dfs.datanode.handler.count</name>
+  <value>3</value>
+  <description>The number of server threads for the datanode.</description>
+</property>
+
+<property>
+  <name>dfs.http.address</name>
+  <value>0.0.0.0:50070</value>
+  <description>
+    The address and the base port where the dfs namenode web ui will listen on.
+    If the port is 0 then the server will start on a free port.
+  </description>
+</property>
+
+<property>
+  <name>dfs.https.enable</name>
+  <value>false</value>
+  <description>Decide if HTTPS(SSL) is supported on HDFS
+  </description>
+</property>
+
+<property>
+  <name>dfs.https.need.client.auth</name>
+  <value>false</value>
+  <description>Whether SSL client certificate authentication is required
+  </description>
+</property>
+
+<property>
+  <name>dfs.https.server.keystore.resource</name>
+  <value>ssl-server.xml</value>
+  <description>Resource file from which ssl server keystore
+  information will be extracted
+  </description>
+</property>
+
+<property>
+  <name>dfs.https.client.keystore.resource</name>
+  <value>ssl-client.xml</value>
+  <description>Resource file from which ssl client keystore
+  information will be extracted
+  </description>
+</property>
+
+<property>
+  <name>dfs.datanode.https.address</name>
+  <value>0.0.0.0:50475</value>
+</property>
+
+<property>
+  <name>dfs.https.address</name>
+  <value>0.0.0.0:50470</value>
+</property>
+
+ <property>
+  <name>dfs.datanode.dns.interface</name>
+  <value>default</value>
+  <description>The name of the Network Interface from which a data node should 
+  report its IP address.
+  </description>
+ </property>
+ 
+<property>
+  <name>dfs.datanode.dns.nameserver</name>
+  <value>default</value>
+  <description>The host name or IP address of the name server (DNS)
+  which a DataNode should use to determine the host name used by the
+  NameNode for communication and display purposes.
+  </description>
+ </property>
+ 
+ <property>
+  <name>dfs.backup.address</name>
+  <value>0.0.0.0:50100</value>
+  <description>
+    The backup node server address and port.
+    If the port is 0 then the server will start on a free port.
+  </description>
+</property>
+ 
+ <property>
+  <name>dfs.backup.http.address</name>
+  <value>0.0.0.0:50105</value>
+  <description>
+    The backup node http server address and port.
+    If the port is 0 then the server will start on a free port.
+  </description>
+</property>
+
+<property>
+  <name>dfs.replication.considerLoad</name>
+  <value>true</value>
+  <description>Decide if chooseTarget considers the target's load or not
+  </description>
+</property>
+<property>
+  <name>dfs.default.chunk.view.size</name>
+  <value>32768</value>
+  <description>The number of bytes to view for a file on the browser.
+  </description>
+</property>
+
+<property>
+  <name>dfs.datanode.du.reserved</name>
+  <value>0</value>
+  <description>Reserved space in bytes per volume. Always leave this much space free for non dfs use.
+  </description>
+</property>
+
+<property>
+  <name>dfs.name.dir</name>
+  <value>${hadoop.tmp.dir}/dfs/name</value>
+  <description>Determines where on the local filesystem the DFS name node
+      should store the name table(fsimage).  If this is a comma-delimited list
+      of directories then the name table is replicated in all of the
+      directories, for redundancy. </description>
+</property>
+
+<property>
+  <name>dfs.name.edits.dir</name>
+  <value>${dfs.name.dir}</value>
+  <description>Determines where on the local filesystem the DFS name node
+      should store the transaction (edits) file. If this is a comma-delimited list
+      of directories then the transaction file is replicated in all of the 
+      directories, for redundancy. Default value is same as dfs.name.dir
+  </description>
+</property>
+<property>
+  <name>dfs.web.ugi</name>
+  <value>webuser,webgroup</value>
+  <description>The user account used by the web interface.
+    Syntax: USERNAME,GROUP1,GROUP2, ...
+  </description>
+</property>
+
+<property>
+  <name>dfs.permissions</name>
+  <value>true</value>
+  <description>
+    If "true", enable permission checking in HDFS.
+    If "false", permission checking is turned off,
+    but all other behavior is unchanged.
+    Switching from one parameter value to the other does not change the mode,
+    owner or group of files or directories.
+  </description>
+</property>
+
+<property>
+  <name>dfs.permissions.supergroup</name>
+  <value>supergroup</value>
+  <description>The name of the group of super-users.</description>
+</property>
+
+<property>
+  <name>dfs.access.token.enable</name>
+  <value>false</value>
+  <description>
+    If "true", access tokens are used as capabilities for accessing datanodes.
+    If "false", no access tokens are checked on accessing datanodes.
+  </description>
+</property>
+
+<property>
+  <name>dfs.access.key.update.interval</name>
+  <value>600</value>
+  <description>
+    Interval in minutes at which namenode updates its access keys.
+  </description>
+</property>
+
+<property>
+  <name>dfs.access.token.lifetime</name>
+  <value>600</value>
+  <description>The lifetime of access tokens in minutes.</description>
+</property>
+
+<property>
+  <name>dfs.data.dir</name>
+  <value>${hadoop.tmp.dir}/dfs/data</value>
+  <description>Determines where on the local filesystem an DFS data node
+  should store its blocks.  If this is a comma-delimited
+  list of directories, then data will be stored in all named
+  directories, typically on different devices.
+  Directories that do not exist are ignored.
+  </description>
+</property>
+
+<property>
+  <name>dfs.replication</name>
+  <value>3</value>
+  <description>Default block replication. 
+  The actual number of replications can be specified when the file is created.
+  The default is used if replication is not specified in create time.
+  </description>
+</property>
+
+<property>
+  <name>dfs.replication.max</name>
+  <value>512</value>
+  <description>Maximal block replication. 
+  </description>
+</property>
+
+<property>
+  <name>dfs.replication.min</name>
+  <value>1</value>
+  <description>Minimal block replication. 
+  </description>
+</property>
+
+<property>
+  <name>dfs.block.size</name>
+  <value>67108864</value>
+  <description>The default block size for new files.</description>
+</property>
+
+<property>
+  <name>dfs.df.interval</name>
+  <value>60000</value>
+  <description>Disk usage statistics refresh interval in msec.</description>
+</property>
+
+<property>
+  <name>dfs.client.block.write.retries</name>
+  <value>3</value>
+  <description>The number of retries for writing blocks to the data nodes, 
+  before we signal failure to the application.
+  </description>
+</property>
+
+<property>
+  <name>dfs.blockreport.intervalMsec</name>
+  <value>3600000</value>
+  <description>Determines block reporting interval in milliseconds.</description>
+</property>
+
+<property>
+  <name>dfs.blockreport.initialDelay</name>  <value>0</value>
+  <description>Delay for first block report in seconds.</description>
+</property>
+
+<property>
+  <name>dfs.datanode.directoryscan.interval</name>
+  <value>21600</value>
+  <description>Interval in seconds for Datanode to scan data directories and
+  reconcile the difference between blocks in memory and on the disk.
+  </description>
+</property>
+
+<property>
+  <name>dfs.heartbeat.interval</name>
+  <value>3</value>
+  <description>Determines datanode heartbeat interval in seconds.</description>
+</property>
+
+<property>
+  <name>dfs.namenode.handler.count</name>
+  <value>10</value>
+  <description>The number of server threads for the namenode.</description>
+</property>
+
+<property>
+  <name>dfs.safemode.threshold.pct</name>
+  <value>0.999f</value>
+  <description>
+    Specifies the percentage of blocks that should satisfy 
+    the minimal replication requirement defined by dfs.replication.min.
+    Values less than or equal to 0 mean not to start in safe mode.
+    Values greater than 1 will make safe mode permanent.
+  </description>
+</property>
+
+<property>
+  <name>dfs.safemode.extension</name>
+  <value>30000</value>
+  <description>
+    Determines extension of safe mode in milliseconds 
+    after the threshold level is reached.
+  </description>
+</property>
+
+<property>
+  <name>dfs.balance.bandwidthPerSec</name>
+  <value>1048576</value>
+  <description>
+        Specifies the maximum amount of bandwidth that each datanode
+        can utilize for the balancing purpose in term of
+        the number of bytes per second.
+  </description>
+</property>
+
+<property>
+  <name>dfs.hosts</name>
+  <value></value>
+  <description>Names a file that contains a list of hosts that are
+  permitted to connect to the namenode. The full pathname of the file
+  must be specified.  If the value is empty, all hosts are
+  permitted.</description>
+</property>
+
+<property>
+  <name>dfs.hosts.exclude</name>
+  <value></value>
+  <description>Names a file that contains a list of hosts that are
+  not permitted to connect to the namenode.  The full pathname of the
+  file must be specified.  If the value is empty, no hosts are
+  excluded.</description>
+</property> 
+
+<property>
+  <name>dfs.max.objects</name>
+  <value>0</value>
+  <description>The maximum number of files, directories and blocks
+  dfs supports. A value of zero indicates no limit to the number
+  of objects that dfs supports.
+  </description>
+</property>
+
+<property>
+  <name>dfs.namenode.decommission.interval</name>
+  <value>30</value>
+  <description>Namenode periodicity in seconds to check if decommission is 
+  complete.</description>
+</property>
+
+<property>
+  <name>dfs.namenode.decommission.nodes.per.interval</name>
+  <value>5</value>
+  <description>The number of nodes namenode checks if decommission is complete
+  in each dfs.namenode.decommission.interval.</description>
+</property>
+
+<property>
+  <name>dfs.replication.interval</name>
+  <value>3</value>
+  <description>The periodicity in seconds with which the namenode computes 
+  repliaction work for datanodes. </description>
+</property>
+
+<property>
+  <name>dfs.access.time.precision</name>
+  <value>3600000</value>
+  <description>The access time for HDFS file is precise upto this value. 
+               The default value is 1 hour. Setting a value of 0 disables
+               access times for HDFS.
+  </description>
+</property>
+
+<property>
+  <name>dfs.support.append</name>
+  <value>false</value>
+  <description>Does HDFS allow appends to files?
+               This is currently set to false because there are bugs in the
+               "append code" and is not supported in any prodction cluster.
+  </description>
+</property>
+
+<property>
+  <name>dfs.datanode.plugins</name>
+  <value></value>
+  <description>Comma-separated list of datanode plug-ins to be activated.
+  </description>
+</property>
+
+<property>
+  <name>dfs.namenode.plugins</name>
+  <value></value>
+  <description>Comma-separated list of namenode plug-ins to be activated.
+  </description>
+</property>
+
+</configuration>

+ 3300 - 0
src/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -0,0 +1,3300 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.ipc.*;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.NodeBase;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+import org.apache.hadoop.security.InvalidAccessTokenException;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.AccessToken;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.util.*;
+
+import org.apache.commons.logging.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.zip.CRC32;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.ConcurrentHashMap;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+
+import javax.net.SocketFactory;
+import javax.security.auth.login.LoginException;
+
+/********************************************************
+ * DFSClient can connect to a Hadoop Filesystem and 
+ * perform basic file tasks.  It uses the ClientProtocol
+ * to communicate with a NameNode daemon, and connects 
+ * directly to DataNodes to read/write block data.
+ *
+ * Hadoop DFS users should obtain an instance of 
+ * DistributedFileSystem, which uses DFSClient to handle
+ * filesystem tasks.
+ *
+ ********************************************************/
+public class DFSClient implements FSConstants, java.io.Closeable {
+  public static final Log LOG = LogFactory.getLog(DFSClient.class);
+  public static final int MAX_BLOCK_ACQUIRE_FAILURES = 3;
+  private static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
+  public final ClientProtocol namenode;
+  final private ClientProtocol rpcNamenode;
+  final UnixUserGroupInformation ugi;
+  volatile boolean clientRunning = true;
+  Random r = new Random();
+  final String clientName;
+  final LeaseChecker leasechecker = new LeaseChecker();
+  private Configuration conf;
+  private long defaultBlockSize;
+  private short defaultReplication;
+  private SocketFactory socketFactory;
+  private int socketTimeout;
+  private int datanodeWriteTimeout;
+  final int writePacketSize;
+  private final FileSystem.Statistics stats;
+  private int maxBlockAcquireFailures;
+    
+ 
+  public static ClientProtocol createNamenode(Configuration conf) throws IOException {
+    return createNamenode(NameNode.getAddress(conf), conf);
+  }
+
+  public static ClientProtocol createNamenode( InetSocketAddress nameNodeAddr,
+      Configuration conf) throws IOException {
+    try {
+      return createNamenode(createRPCNamenode(nameNodeAddr, conf,
+        UnixUserGroupInformation.login(conf, true)));
+    } catch (LoginException e) {
+      throw (IOException)(new IOException().initCause(e));
+    }
+  }
+
+  private static ClientProtocol createRPCNamenode(InetSocketAddress nameNodeAddr,
+      Configuration conf, UnixUserGroupInformation ugi) 
+    throws IOException {
+    return (ClientProtocol)RPC.getProxy(ClientProtocol.class,
+        ClientProtocol.versionID, nameNodeAddr, ugi, conf,
+        NetUtils.getSocketFactory(conf, ClientProtocol.class));
+  }
+
+  private static ClientProtocol createNamenode(ClientProtocol rpcNamenode)
+    throws IOException {
+    RetryPolicy createPolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep(
+        5, LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
+    
+    Map<Class<? extends Exception>,RetryPolicy> remoteExceptionToPolicyMap =
+      new HashMap<Class<? extends Exception>, RetryPolicy>();
+    remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class, createPolicy);
+
+    Map<Class<? extends Exception>,RetryPolicy> exceptionToPolicyMap =
+      new HashMap<Class<? extends Exception>, RetryPolicy>();
+    exceptionToPolicyMap.put(RemoteException.class, 
+        RetryPolicies.retryByRemoteException(
+            RetryPolicies.TRY_ONCE_THEN_FAIL, remoteExceptionToPolicyMap));
+    RetryPolicy methodPolicy = RetryPolicies.retryByException(
+        RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
+    Map<String,RetryPolicy> methodNameToPolicyMap = new HashMap<String,RetryPolicy>();
+    
+    methodNameToPolicyMap.put("create", methodPolicy);
+
+    return (ClientProtocol) RetryProxy.create(ClientProtocol.class,
+        rpcNamenode, methodNameToPolicyMap);
+  }
+
+  static ClientDatanodeProtocol createClientDatanodeProtocolProxy (
+      DatanodeID datanodeid, Configuration conf) throws IOException {
+    InetSocketAddress addr = NetUtils.createSocketAddr(
+      datanodeid.getHost() + ":" + datanodeid.getIpcPort());
+    if (ClientDatanodeProtocol.LOG.isDebugEnabled()) {
+      ClientDatanodeProtocol.LOG.info("ClientDatanodeProtocol addr=" + addr);
+    }
+    return (ClientDatanodeProtocol)RPC.getProxy(ClientDatanodeProtocol.class,
+        ClientDatanodeProtocol.versionID, addr, conf);
+  }
+        
+  /** 
+   * Create a new DFSClient connected to the default namenode.
+   */
+  public DFSClient(Configuration conf) throws IOException {
+    this(NameNode.getAddress(conf), conf, null);
+  }
+
+  /** 
+   * Create a new DFSClient connected to the given namenode server.
+   */
+  public DFSClient(InetSocketAddress nameNodeAddr, Configuration conf,
+                   FileSystem.Statistics stats)
+    throws IOException {
+    this.conf = conf;
+    this.stats = stats;
+    this.socketTimeout = conf.getInt("dfs.socket.timeout", 
+                                     HdfsConstants.READ_TIMEOUT);
+    this.datanodeWriteTimeout = conf.getInt("dfs.datanode.socket.write.timeout",
+                                            HdfsConstants.WRITE_TIMEOUT);
+    this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
+    // dfs.write.packet.size is an internal config variable
+    this.writePacketSize = conf.getInt("dfs.write.packet.size", 64*1024);
+    this.maxBlockAcquireFailures = 
+                          conf.getInt("dfs.client.max.block.acquire.failures",
+                                      MAX_BLOCK_ACQUIRE_FAILURES);
+    
+    try {
+      this.ugi = UnixUserGroupInformation.login(conf, true);
+    } catch (LoginException e) {
+      throw (IOException)(new IOException().initCause(e));
+    }
+
+    this.rpcNamenode = createRPCNamenode(nameNodeAddr, conf, ugi);
+    this.namenode = createNamenode(rpcNamenode);
+
+    String taskId = conf.get("mapred.task.id");
+    if (taskId != null) {
+      this.clientName = "DFSClient_" + taskId; 
+    } else {
+      this.clientName = "DFSClient_" + r.nextInt();
+    }
+    defaultBlockSize = conf.getLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    defaultReplication = (short) conf.getInt("dfs.replication", 3);
+  }
+
+  public DFSClient(InetSocketAddress nameNodeAddr, 
+                   Configuration conf) throws IOException {
+    this(nameNodeAddr, conf, null);
+  }
+
+  private void checkOpen() throws IOException {
+    if (!clientRunning) {
+      IOException result = new IOException("Filesystem closed");
+      throw result;
+    }
+  }
+    
+  /**
+   * Close the file system, abandoning all of the leases and files being
+   * created and close connections to the namenode.
+   */
+  public synchronized void close() throws IOException {
+    if(clientRunning) {
+      leasechecker.close();
+      clientRunning = false;
+      try {
+        leasechecker.interruptAndJoin();
+      } catch (InterruptedException ie) {
+      }
+  
+      // close connections to the namenode
+      RPC.stopProxy(rpcNamenode);
+    }
+  }
+
+  /**
+   * Get the default block size for this cluster
+   * @return the default block size in bytes
+   */
+  public long getDefaultBlockSize() {
+    return defaultBlockSize;
+  }
+    
+  public long getBlockSize(String f) throws IOException {
+    try {
+      return namenode.getPreferredBlockSize(f);
+    } catch (IOException ie) {
+      LOG.warn("Problem getting block size: " + 
+          StringUtils.stringifyException(ie));
+      throw ie;
+    }
+  }
+
+  /**
+   * Report corrupt blocks that were discovered by the client.
+   */
+  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+    namenode.reportBadBlocks(blocks);
+  }
+  
+  public short getDefaultReplication() {
+    return defaultReplication;
+  }
+
+  private static LocatedBlocks callGetBlockLocations(ClientProtocol namenode,
+      String src, long start, long length) throws IOException {
+    try {
+      return namenode.getBlockLocations(src, start, length);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                    FileNotFoundException.class);
+    }
+  }
+
+  /**
+   * Get block location info about file
+   * 
+   * getBlockLocations() returns a list of hostnames that store 
+   * data for a specific file region.  It returns a set of hostnames
+   * for every block within the indicated region.
+   *
+   * This function is very useful when writing code that considers
+   * data-placement when performing operations.  For example, the
+   * MapReduce system tries to schedule tasks on the same machines
+   * as the data-block the task processes. 
+   */
+  public BlockLocation[] getBlockLocations(String src, long start, 
+    long length) throws IOException {
+    LocatedBlocks blocks = callGetBlockLocations(namenode, src, start, length);
+    if (blocks == null) {
+      return new BlockLocation[0];
+    }
+    int nrBlocks = blocks.locatedBlockCount();
+    BlockLocation[] blkLocations = new BlockLocation[nrBlocks];
+    int idx = 0;
+    for (LocatedBlock blk : blocks.getLocatedBlocks()) {
+      assert idx < nrBlocks : "Incorrect index";
+      DatanodeInfo[] locations = blk.getLocations();
+      String[] hosts = new String[locations.length];
+      String[] names = new String[locations.length];
+      String[] racks = new String[locations.length];
+      for (int hCnt = 0; hCnt < locations.length; hCnt++) {
+        hosts[hCnt] = locations[hCnt].getHostName();
+        names[hCnt] = locations[hCnt].getName();
+        NodeBase node = new NodeBase(names[hCnt], 
+                                     locations[hCnt].getNetworkLocation());
+        racks[hCnt] = node.toString();
+      }
+      blkLocations[idx] = new BlockLocation(names, hosts, racks,
+                                            blk.getStartOffset(),
+                                            blk.getBlockSize());
+      idx++;
+    }
+    return blkLocations;
+  }
+
+  public DFSInputStream open(String src) throws IOException {
+    return open(src, conf.getInt("io.file.buffer.size", 4096), true, null);
+  }
+
+  /**
+   * Create an input stream that obtains a nodelist from the
+   * namenode, and then reads from all the right places.  Creates
+   * inner subclass of InputStream that does the right out-of-band
+   * work.
+   */
+  DFSInputStream open(String src, int buffersize, boolean verifyChecksum,
+                      FileSystem.Statistics stats
+      ) throws IOException {
+    checkOpen();
+    //    Get block info from namenode
+    return new DFSInputStream(src, buffersize, verifyChecksum);
+  }
+
+  /**
+   * Create a new dfs file and return an output stream for writing into it. 
+   * 
+   * @param src stream name
+   * @param overwrite do not check for file existence if true
+   * @return output stream
+   * @throws IOException
+   */
+  public OutputStream create(String src, 
+                             boolean overwrite
+                             ) throws IOException {
+    return create(src, overwrite, defaultReplication, defaultBlockSize, null);
+  }
+    
+  /**
+   * Create a new dfs file and return an output stream for writing into it
+   * with write-progress reporting. 
+   * 
+   * @param src stream name
+   * @param overwrite do not check for file existence if true
+   * @return output stream
+   * @throws IOException
+   */
+  public OutputStream create(String src, 
+                             boolean overwrite,
+                             Progressable progress
+                             ) throws IOException {
+    return create(src, overwrite, defaultReplication, defaultBlockSize, null);
+  }
+    
+  /**
+   * Create a new dfs file with the specified block replication 
+   * and return an output stream for writing into the file.  
+   * 
+   * @param src stream name
+   * @param overwrite do not check for file existence if true
+   * @param replication block replication
+   * @return output stream
+   * @throws IOException
+   */
+  public OutputStream create(String src, 
+                             boolean overwrite, 
+                             short replication,
+                             long blockSize
+                             ) throws IOException {
+    return create(src, overwrite, replication, blockSize, null);
+  }
+
+  
+  /**
+   * Create a new dfs file with the specified block replication 
+   * with write-progress reporting and return an output stream for writing
+   * into the file.  
+   * 
+   * @param src stream name
+   * @param overwrite do not check for file existence if true
+   * @param replication block replication
+   * @return output stream
+   * @throws IOException
+   */
+  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));
+  }
+  /**
+   * Call
+   * {@link #create(String,FsPermission,boolean,short,long,Progressable,int)}
+   * with default permission.
+   * @see FsPermission#getDefault()
+   */
+  public OutputStream create(String src,
+      boolean overwrite,
+      short replication,
+      long blockSize,
+      Progressable progress,
+      int buffersize
+      ) throws IOException {
+    return create(src, FsPermission.getDefault(),
+        overwrite, replication, blockSize, progress, buffersize);
+  }
+  /**
+   * Create a new dfs file with the specified block replication 
+   * with write-progress reporting and return an output stream for writing
+   * into the file.  
+   * 
+   * @param src stream name
+   * @param permission The permission of the directory being created.
+   * If permission == null, use {@link FsPermission#getDefault()}.
+   * @param overwrite do not check for file existence if true
+   * @param replication block replication
+   * @return output stream
+   * @throws IOException
+   * @see ClientProtocol#create(String, FsPermission, String, boolean, short, long)
+   */
+  public OutputStream create(String src, 
+                             FsPermission permission,
+                             boolean overwrite, 
+                             short replication,
+                             long blockSize,
+                             Progressable progress,
+                             int buffersize
+                             ) throws IOException {
+    checkOpen();
+    if (permission == null) {
+      permission = FsPermission.getDefault();
+    }
+    FsPermission masked = permission.applyUMask(FsPermission.getUMask(conf));
+    LOG.debug(src + ": masked=" + masked);
+    OutputStream result = new DFSOutputStream(src, masked,
+        overwrite, replication, blockSize, progress, buffersize,
+        conf.getInt("io.bytes.per.checksum", 512));
+    leasechecker.put(src, result);
+    return result;
+  }
+
+  /**
+   * Append to an existing HDFS file.  
+   * 
+   * @param src file name
+   * @param buffersize buffer size
+   * @param progress for reporting write-progress
+   * @return an output stream for writing into the file
+   * @throws IOException
+   * @see ClientProtocol#append(String, String)
+   */
+  OutputStream append(String src, int buffersize, Progressable progress
+      ) throws IOException {
+    checkOpen();
+    FileStatus stat = null;
+    LocatedBlock lastBlock = null;
+    try {
+      stat = getFileInfo(src);
+      lastBlock = namenode.append(src, clientName);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(FileNotFoundException.class,
+                                     AccessControlException.class,
+                                     QuotaExceededException.class);
+    }
+    OutputStream result = new DFSOutputStream(src, buffersize, progress,
+        lastBlock, stat, conf.getInt("io.bytes.per.checksum", 512));
+    leasechecker.put(src, result);
+    return result;
+  }
+
+  /**
+   * Set replication for an existing file.
+   * 
+   * @see ClientProtocol#setReplication(String, short)
+   * @param replication
+   * @throws IOException
+   * @return true is successful or false if file does not exist 
+   */
+  public boolean setReplication(String src, 
+                                short replication
+                                ) throws IOException {
+    try {
+      return namenode.setReplication(src, replication);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     QuotaExceededException.class);
+    }
+  }
+
+  /**
+   * Rename file or directory.
+   * See {@link ClientProtocol#rename(String, String)}. 
+   */
+  public boolean rename(String src, String dst) throws IOException {
+    checkOpen();
+    try {
+      return namenode.rename(src, dst);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     QuotaExceededException.class);
+    }
+  }
+
+  /**
+   * Delete file or directory.
+   * See {@link ClientProtocol#delete(String)}. 
+   */
+  @Deprecated
+  public boolean delete(String src) throws IOException {
+    checkOpen();
+    return namenode.delete(src, true);
+  }
+
+  /**
+   * delete file or directory.
+   * delete contents of the directory if non empty and recursive 
+   * set to true
+   */
+  public boolean delete(String src, boolean recursive) throws IOException {
+    checkOpen();
+    try {
+      return namenode.delete(src, recursive);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class);
+    }
+  }
+  
+  /** Implemented using getFileInfo(src)
+   */
+  public boolean exists(String src) throws IOException {
+    checkOpen();
+    return getFileInfo(src) != null;
+  }
+
+  public FileStatus[] listPaths(String src) throws IOException {
+    checkOpen();
+    try {
+      return namenode.getListing(src);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class);
+    }
+  }
+
+  public FileStatus getFileInfo(String src) throws IOException {
+    checkOpen();
+    try {
+      return namenode.getFileInfo(src);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class);
+    }
+  }
+
+  /**
+   * Get the checksum of a file.
+   * @param src The file path
+   * @return The checksum 
+   * @see DistributedFileSystem#getFileChecksum(Path)
+   */
+  MD5MD5CRC32FileChecksum getFileChecksum(String src) throws IOException {
+    checkOpen();
+    return getFileChecksum(src, namenode, socketFactory, socketTimeout);    
+  }
+
+  /**
+   * Get the checksum of a file.
+   * @param src The file path
+   * @return The checksum 
+   */
+  public static MD5MD5CRC32FileChecksum getFileChecksum(String src,
+      ClientProtocol namenode, SocketFactory socketFactory, int socketTimeout
+      ) throws IOException {
+    //get all block locations
+    List<LocatedBlock> locatedblocks
+        = callGetBlockLocations(namenode, src, 0, Long.MAX_VALUE).getLocatedBlocks();
+    final DataOutputBuffer md5out = new DataOutputBuffer();
+    int bytesPerCRC = 0;
+    long crcPerBlock = 0;
+    boolean refetchBlocks = false;
+    int lastRetriedIndex = -1;
+
+    //get block checksum for each block
+    for(int i = 0; i < locatedblocks.size(); i++) {
+      if (refetchBlocks) {  // refetch to get fresh tokens
+        locatedblocks = callGetBlockLocations(namenode, src, 0, Long.MAX_VALUE)
+            .getLocatedBlocks();
+        refetchBlocks = false;
+      }
+      LocatedBlock lb = locatedblocks.get(i);
+      final Block block = lb.getBlock();
+      final DatanodeInfo[] datanodes = lb.getLocations();
+      
+      //try each datanode location of the block
+      final int timeout = 3000 * datanodes.length + socketTimeout;
+      boolean done = false;
+      for(int j = 0; !done && j < datanodes.length; j++) {
+        //connect to a datanode
+        final Socket sock = socketFactory.createSocket();
+        NetUtils.connect(sock, 
+                         NetUtils.createSocketAddr(datanodes[j].getName()),
+                         timeout);
+        sock.setSoTimeout(timeout);
+
+        DataOutputStream out = new DataOutputStream(
+            new BufferedOutputStream(NetUtils.getOutputStream(sock), 
+                                     DataNode.SMALL_BUFFER_SIZE));
+        DataInputStream in = new DataInputStream(NetUtils.getInputStream(sock));
+
+        // get block MD5
+        try {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("write to " + datanodes[j].getName() + ": "
+                + DataTransferProtocol.OP_BLOCK_CHECKSUM +
+                ", block=" + block);
+          }
+          out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
+          out.write(DataTransferProtocol.OP_BLOCK_CHECKSUM);
+          out.writeLong(block.getBlockId());
+          out.writeLong(block.getGenerationStamp());
+          lb.getAccessToken().write(out);
+          out.flush();
+         
+          final short reply = in.readShort();
+          if (reply != DataTransferProtocol.OP_STATUS_SUCCESS) {
+            if (reply == DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN
+                && i > lastRetriedIndex) {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
+                    + "for file " + src + " for block " + block
+                    + " from datanode " + datanodes[j].getName()
+                    + ". Will retry the block once.");
+              }
+              lastRetriedIndex = i;
+              done = true; // actually it's not done; but we'll retry
+              i--; // repeat at i-th block
+              refetchBlocks = true;
+              break;
+            } else {
+              throw new IOException("Bad response " + reply + " for block "
+                  + block + " from datanode " + datanodes[j].getName());
+            }
+          }
+
+          //read byte-per-checksum
+          final int bpc = in.readInt(); 
+          if (i == 0) { //first block
+            bytesPerCRC = bpc;
+          }
+          else if (bpc != bytesPerCRC) {
+            throw new IOException("Byte-per-checksum not matched: bpc=" + bpc
+                + " but bytesPerCRC=" + bytesPerCRC);
+          }
+          
+          //read crc-per-block
+          final long cpb = in.readLong();
+          if (locatedblocks.size() > 1 && i == 0) {
+            crcPerBlock = cpb;
+          }
+
+          //read md5
+          final MD5Hash md5 = MD5Hash.read(in);
+          md5.write(md5out);
+          
+          done = true;
+
+          if (LOG.isDebugEnabled()) {
+            if (i == 0) {
+              LOG.debug("set bytesPerCRC=" + bytesPerCRC
+                  + ", crcPerBlock=" + crcPerBlock);
+            }
+            LOG.debug("got reply from " + datanodes[j].getName()
+                + ": md5=" + md5);
+          }
+        } catch (IOException ie) {
+          LOG.warn("src=" + src + ", datanodes[" + j + "].getName()="
+              + datanodes[j].getName(), ie);
+        } finally {
+          IOUtils.closeStream(in);
+          IOUtils.closeStream(out);
+          IOUtils.closeSocket(sock);        
+        }
+      }
+
+      if (!done) {
+        throw new IOException("Fail to get block MD5 for " + block);
+      }
+    }
+
+    //compute file MD5
+    final MD5Hash fileMD5 = MD5Hash.digest(md5out.getData()); 
+    return new MD5MD5CRC32FileChecksum(bytesPerCRC, crcPerBlock, fileMD5);
+  }
+
+  /**
+   * Set permissions to a file or directory.
+   * @param src path name.
+   * @param permission
+   * @throws <code>FileNotFoundException</code> is file does not exist.
+   */
+  public void setPermission(String src, FsPermission permission
+                            ) throws IOException {
+    checkOpen();
+    try {
+      namenode.setPermission(src, permission);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class);
+    }
+  }
+
+  /**
+   * Set file or directory owner.
+   * @param src path name.
+   * @param username user id.
+   * @param groupname user group.
+   * @throws <code>FileNotFoundException</code> is file does not exist.
+   */
+  public void setOwner(String src, String username, String groupname
+                      ) throws IOException {
+    checkOpen();
+    try {
+      namenode.setOwner(src, username, groupname);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class);
+    }
+  }
+
+  public FsStatus getDiskStatus() throws IOException {
+    long rawNums[] = namenode.getStats();
+    return new FsStatus(rawNums[0], rawNums[1], rawNums[2]);
+  }
+
+  /**
+   * Returns count of blocks with no good replicas left. Normally should be 
+   * zero.
+   * @throws IOException
+   */ 
+  public long getMissingBlocksCount() throws IOException {
+    return namenode.getStats()[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX];
+  }
+  
+  /**
+   * Returns count of blocks with one of more replica missing.
+   * @throws IOException
+   */ 
+  public long getUnderReplicatedBlocksCount() throws IOException {
+    return namenode.getStats()[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX];
+  }
+  
+  /**
+   * Returns count of blocks with at least one replica marked corrupt. 
+   * @throws IOException
+   */ 
+  public long getCorruptBlocksCount() throws IOException {
+    return namenode.getStats()[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX];
+  }
+  
+  public DatanodeInfo[] datanodeReport(DatanodeReportType type)
+  throws IOException {
+    return namenode.getDatanodeReport(type);
+  }
+    
+  /**
+   * Enter, leave or get safe mode.
+   * See {@link ClientProtocol#setSafeMode(FSConstants.SafeModeAction)} 
+   * for more details.
+   * 
+   * @see ClientProtocol#setSafeMode(FSConstants.SafeModeAction)
+   */
+  public boolean setSafeMode(SafeModeAction action) throws IOException {
+    return namenode.setSafeMode(action);
+  }
+
+  /**
+   * Save namespace image.
+   * See {@link ClientProtocol#saveNamespace()} 
+   * for more details.
+   * 
+   * @see ClientProtocol#saveNamespace()
+   */
+  void saveNamespace() throws AccessControlException, IOException {
+    try {
+      namenode.saveNamespace();
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class);
+    }
+  }
+  
+  /**
+   * enable/disable restore failed storage.
+   * See {@link ClientProtocol#restoreFailedStorage(String arg)} 
+   * for more details.
+   * 
+   */
+  boolean restoreFailedStorage(String arg) throws AccessControlException {
+    return namenode.restoreFailedStorage(arg);
+  }
+
+  /**
+   * Refresh the hosts and exclude files.  (Rereads them.)
+   * See {@link ClientProtocol#refreshNodes()} 
+   * for more details.
+   * 
+   * @see ClientProtocol#refreshNodes()
+   */
+  public void refreshNodes() throws IOException {
+    namenode.refreshNodes();
+  }
+
+  /**
+   * Dumps DFS data structures into specified file.
+   * See {@link ClientProtocol#metaSave(String)} 
+   * for more details.
+   * 
+   * @see ClientProtocol#metaSave(String)
+   */
+  public void metaSave(String pathname) throws IOException {
+    namenode.metaSave(pathname);
+  }
+    
+  /**
+   * @see ClientProtocol#finalizeUpgrade()
+   */
+  public void finalizeUpgrade() throws IOException {
+    namenode.finalizeUpgrade();
+  }
+
+  /**
+   * @see ClientProtocol#distributedUpgradeProgress(FSConstants.UpgradeAction)
+   */
+  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action
+                                                        ) throws IOException {
+    return namenode.distributedUpgradeProgress(action);
+  }
+
+  /**
+   */
+  public boolean mkdirs(String src) throws IOException {
+    return mkdirs(src, null);
+  }
+
+  /**
+   * Create a directory (or hierarchy of directories) with the given
+   * name and permission.
+   *
+   * @param src The path of the directory being created
+   * @param permission The permission of the directory being created.
+   * If permission == null, use {@link FsPermission#getDefault()}.
+   * @return True if the operation success.
+   * @see ClientProtocol#mkdirs(String, FsPermission)
+   */
+  public boolean mkdirs(String src, FsPermission permission)throws IOException{
+    checkOpen();
+    if (permission == null) {
+      permission = FsPermission.getDefault();
+    }
+    FsPermission masked = permission.applyUMask(FsPermission.getUMask(conf));
+    LOG.debug(src + ": masked=" + masked);
+    try {
+      return namenode.mkdirs(src, masked);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     QuotaExceededException.class);
+    }
+  }
+
+  ContentSummary getContentSummary(String src) throws IOException {
+    try {
+      return namenode.getContentSummary(src);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class);
+    }
+  }
+
+  /**
+   * Sets or resets quotas for a directory.
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long)
+   */
+  void setQuota(String src, long namespaceQuota, long diskspaceQuota) 
+                                                 throws IOException {
+    // sanity check
+    if ((namespaceQuota <= 0 && namespaceQuota != FSConstants.QUOTA_DONT_SET &&
+         namespaceQuota != FSConstants.QUOTA_RESET) ||
+        (diskspaceQuota <= 0 && diskspaceQuota != FSConstants.QUOTA_DONT_SET &&
+         diskspaceQuota != FSConstants.QUOTA_RESET)) {
+      throw new IllegalArgumentException("Invalid values for quota : " +
+                                         namespaceQuota + " and " + 
+                                         diskspaceQuota);
+                                         
+    }
+    
+    try {
+      namenode.setQuota(src, namespaceQuota, diskspaceQuota);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     QuotaExceededException.class);
+    }
+  }
+
+  /**
+   * set the modification and access time of a file
+   * @throws FileNotFoundException if the path is not a file
+   */
+  public void setTimes(String src, long mtime, long atime) throws IOException {
+    checkOpen();
+    try {
+      namenode.setTimes(src, mtime, atime);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class);
+    }
+  }
+
+  /**
+   * Pick the best node from which to stream the data.
+   * Entries in <i>nodes</i> are already in the priority order
+   */
+  private DatanodeInfo bestNode(DatanodeInfo nodes[], 
+                                AbstractMap<DatanodeInfo, DatanodeInfo> deadNodes)
+                                throws IOException {
+    if (nodes != null) { 
+      for (int i = 0; i < nodes.length; i++) {
+        if (!deadNodes.containsKey(nodes[i])) {
+          return nodes[i];
+        }
+      }
+    }
+    throw new IOException("No live nodes contain current block");
+  }
+
+  boolean isLeaseCheckerStarted() {
+    return leasechecker.daemon != null;
+  }
+
+  /** Lease management*/
+  class LeaseChecker implements Runnable {
+    /** A map from src -> DFSOutputStream of files that are currently being
+     * written by this client.
+     */
+    private final SortedMap<String, OutputStream> pendingCreates
+        = new TreeMap<String, OutputStream>();
+
+    private Daemon daemon = null;
+    
+    synchronized void put(String src, OutputStream out) {
+      if (clientRunning) {
+        if (daemon == null) {
+          daemon = new Daemon(this);
+          daemon.start();
+        }
+        pendingCreates.put(src, out);
+      }
+    }
+    
+    synchronized void remove(String src) {
+      pendingCreates.remove(src);
+    }
+    
+    void interruptAndJoin() throws InterruptedException {
+      Daemon daemonCopy = null;
+      synchronized (this) {
+        if (daemon != null) {
+          daemon.interrupt();
+          daemonCopy = daemon;
+        }
+      }
+     
+      if (daemonCopy != null) {
+        LOG.debug("Wait for lease checker to terminate");
+        daemonCopy.join();
+      }
+    }
+
+    synchronized void close() {
+      while (!pendingCreates.isEmpty()) {
+        String src = pendingCreates.firstKey();
+        OutputStream out = pendingCreates.remove(src);
+        if (out != null) {
+          try {
+            out.close();
+          } catch (IOException ie) {
+            LOG.error("Exception closing file " + src+ " : " + ie, ie);
+          }
+        }
+      }
+    }
+
+    private void renew() throws IOException {
+      synchronized(this) {
+        if (pendingCreates.isEmpty()) {
+          return;
+        }
+      }
+      namenode.renewLease(clientName);
+    }
+
+    /**
+     * Periodically check in with the namenode and renew all the leases
+     * when the lease period is half over.
+     */
+    public void run() {
+      long lastRenewed = 0;
+      while (clientRunning && !Thread.interrupted()) {
+        if (System.currentTimeMillis() - lastRenewed > (LEASE_SOFTLIMIT_PERIOD / 2)) {
+          try {
+            renew();
+            lastRenewed = System.currentTimeMillis();
+          } catch (IOException ie) {
+            LOG.warn("Problem renewing lease for " + clientName, ie);
+          }
+        }
+
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException ie) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(this + " is interrupted.", ie);
+          }
+          return;
+        }
+      }
+    }
+
+    /** {@inheritDoc} */
+    public String toString() {
+      String s = getClass().getSimpleName();
+      if (LOG.isTraceEnabled()) {
+        return s + "@" + DFSClient.this + ": "
+               + StringUtils.stringifyException(new Throwable("for testing"));
+      }
+      return s;
+    }
+  }
+
+  /** Utility class to encapsulate data node info and its ip address. */
+  private static class DNAddrPair {
+    DatanodeInfo info;
+    InetSocketAddress addr;
+    DNAddrPair(DatanodeInfo info, InetSocketAddress addr) {
+      this.info = info;
+      this.addr = addr;
+    }
+  }
+
+  /** This is a wrapper around connection to datadone
+   * and understands checksum, offset etc
+   */
+  public static class BlockReader extends FSInputChecker {
+
+    private Socket dnSock; //for now just sending checksumOk.
+    private DataInputStream in;
+    private DataChecksum checksum;
+    private long lastChunkOffset = -1;
+    private long lastChunkLen = -1;
+    private long lastSeqNo = -1;
+
+    private long startOffset;
+    private long firstChunkOffset;
+    private int bytesPerChecksum;
+    private int checksumSize;
+    private boolean gotEOS = false;
+    
+    byte[] skipBuf = null;
+    ByteBuffer checksumBytes = null;
+    int dataLeft = 0;
+    boolean isLastPacket = false;
+    
+    /* FSInputChecker interface */
+    
+    /* same interface as inputStream java.io.InputStream#read()
+     * used by DFSInputStream#read()
+     * This violates one rule when there is a checksum error:
+     * "Read should not modify user buffer before successful read"
+     * because it first reads the data to user buffer and then checks
+     * the checksum.
+     */
+    @Override
+    public synchronized int read(byte[] buf, int off, int len) 
+                                 throws IOException {
+      
+      //for the first read, skip the extra bytes at the front.
+      if (lastChunkLen < 0 && startOffset > firstChunkOffset && len > 0) {
+        // Skip these bytes. But don't call this.skip()!
+        int toSkip = (int)(startOffset - firstChunkOffset);
+        if ( skipBuf == null ) {
+          skipBuf = new byte[bytesPerChecksum];
+        }
+        if ( super.read(skipBuf, 0, toSkip) != toSkip ) {
+          // should never happen
+          throw new IOException("Could not skip required number of bytes");
+        }
+      }
+      
+      boolean eosBefore = gotEOS;
+      int nRead = super.read(buf, off, len);
+      
+      // if gotEOS was set in the previous read and checksum is enabled :
+      if (gotEOS && !eosBefore && nRead >= 0 && needChecksum()) {
+        //checksum is verified and there are no errors.
+        checksumOk(dnSock);
+      }
+      return nRead;
+    }
+
+    @Override
+    public synchronized long skip(long n) throws IOException {
+      /* How can we make sure we don't throw a ChecksumException, at least
+       * in majority of the cases?. This one throws. */  
+      if ( skipBuf == null ) {
+        skipBuf = new byte[bytesPerChecksum]; 
+      }
+
+      long nSkipped = 0;
+      while ( nSkipped < n ) {
+        int toSkip = (int)Math.min(n-nSkipped, skipBuf.length);
+        int ret = read(skipBuf, 0, toSkip);
+        if ( ret <= 0 ) {
+          return nSkipped;
+        }
+        nSkipped += ret;
+      }
+      return nSkipped;
+    }
+
+    @Override
+    public int read() throws IOException {
+      throw new IOException("read() is not expected to be invoked. " +
+                            "Use read(buf, off, len) instead.");
+    }
+    
+    @Override
+    public boolean seekToNewSource(long targetPos) throws IOException {
+      /* Checksum errors are handled outside the BlockReader. 
+       * DFSInputStream does not always call 'seekToNewSource'. In the 
+       * case of pread(), it just tries a different replica without seeking.
+       */ 
+      return false;
+    }
+    
+    @Override
+    public void seek(long pos) throws IOException {
+      throw new IOException("Seek() is not supported in BlockInputChecker");
+    }
+
+    @Override
+    protected long getChunkPosition(long pos) {
+      throw new RuntimeException("getChunkPosition() is not supported, " +
+                                 "since seek is not required");
+    }
+    
+    /**
+     * Makes sure that checksumBytes has enough capacity 
+     * and limit is set to the number of checksum bytes needed 
+     * to be read.
+     */
+    private void adjustChecksumBytes(int dataLen) {
+      int requiredSize = 
+        ((dataLen + bytesPerChecksum - 1)/bytesPerChecksum)*checksumSize;
+      if (checksumBytes == null || requiredSize > checksumBytes.capacity()) {
+        checksumBytes =  ByteBuffer.wrap(new byte[requiredSize]);
+      } else {
+        checksumBytes.clear();
+      }
+      checksumBytes.limit(requiredSize);
+    }
+    
+    @Override
+    protected synchronized int readChunk(long pos, byte[] buf, int offset, 
+                                         int len, byte[] checksumBuf) 
+                                         throws IOException {
+      // Read one chunk.
+      
+      if ( gotEOS ) {
+        if ( startOffset < 0 ) {
+          //This is mainly for debugging. can be removed.
+          throw new IOException( "BlockRead: already got EOS or an error" );
+        }
+        startOffset = -1;
+        return -1;
+      }
+      
+      // Read one DATA_CHUNK.
+      long chunkOffset = lastChunkOffset;
+      if ( lastChunkLen > 0 ) {
+        chunkOffset += lastChunkLen;
+      }
+      
+      if ( (pos + firstChunkOffset) != chunkOffset ) {
+        throw new IOException("Mismatch in pos : " + pos + " + " + 
+                              firstChunkOffset + " != " + chunkOffset);
+      }
+
+      // Read next packet if the previous packet has been read completely.
+      if (dataLeft <= 0) {
+        //Read packet headers.
+        int packetLen = in.readInt();
+        long offsetInBlock = in.readLong();
+        long seqno = in.readLong();
+        boolean lastPacketInBlock = in.readBoolean();
+      
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("DFSClient readChunk got seqno " + seqno +
+                    " offsetInBlock " + offsetInBlock +
+                    " lastPacketInBlock " + lastPacketInBlock +
+                    " packetLen " + packetLen);
+        }
+        
+        int dataLen = in.readInt();
+      
+        // Sanity check the lengths
+        if ( dataLen < 0 || 
+             ( (dataLen % bytesPerChecksum) != 0 && !lastPacketInBlock ) ||
+             (seqno != (lastSeqNo + 1)) ) {
+             throw new IOException("BlockReader: error in packet header" +
+                                   "(chunkOffset : " + chunkOffset + 
+                                   ", dataLen : " + dataLen +
+                                   ", seqno : " + seqno + 
+                                   " (last: " + lastSeqNo + "))");
+        }
+        
+        lastSeqNo = seqno;
+        isLastPacket = lastPacketInBlock;
+        dataLeft = dataLen;
+        adjustChecksumBytes(dataLen);
+        if (dataLen > 0) {
+          IOUtils.readFully(in, checksumBytes.array(), 0,
+                            checksumBytes.limit());
+        }
+      }
+
+      int chunkLen = Math.min(dataLeft, bytesPerChecksum);
+      
+      if ( chunkLen > 0 ) {
+        // len should be >= chunkLen
+        IOUtils.readFully(in, buf, offset, chunkLen);
+        checksumBytes.get(checksumBuf, 0, checksumSize);
+      }
+      
+      dataLeft -= chunkLen;
+      lastChunkOffset = chunkOffset;
+      lastChunkLen = chunkLen;
+      
+      if ((dataLeft == 0 && isLastPacket) || chunkLen == 0) {
+        gotEOS = true;
+      }
+      if ( chunkLen == 0 ) {
+        return -1;
+      }
+      
+      return chunkLen;
+    }
+    
+    private BlockReader( String file, long blockId, DataInputStream in, 
+                         DataChecksum checksum, boolean verifyChecksum,
+                         long startOffset, long firstChunkOffset, 
+                         Socket dnSock ) {
+      super(new Path("/blk_" + blockId + ":of:" + file)/*too non path-like?*/,
+            1, verifyChecksum,
+            checksum.getChecksumSize() > 0? checksum : null, 
+            checksum.getBytesPerChecksum(),
+            checksum.getChecksumSize());
+      
+      this.dnSock = dnSock;
+      this.in = in;
+      this.checksum = checksum;
+      this.startOffset = Math.max( startOffset, 0 );
+
+      this.firstChunkOffset = firstChunkOffset;
+      lastChunkOffset = firstChunkOffset;
+      lastChunkLen = -1;
+
+      bytesPerChecksum = this.checksum.getBytesPerChecksum();
+      checksumSize = this.checksum.getChecksumSize();
+    }
+
+    public static BlockReader newBlockReader(Socket sock, String file, long blockId, AccessToken accessToken, 
+        long genStamp, long startOffset, long len, int bufferSize) throws IOException {
+      return newBlockReader(sock, file, blockId, accessToken, genStamp, startOffset, len, bufferSize,
+          true);
+    }
+
+    /** Java Doc required */
+    public static BlockReader newBlockReader( Socket sock, String file, long blockId, 
+                                       AccessToken accessToken,
+                                       long genStamp,
+                                       long startOffset, long len,
+                                       int bufferSize, boolean verifyChecksum)
+                                       throws IOException {
+      return newBlockReader(sock, file, blockId, accessToken, genStamp, startOffset,
+                            len, bufferSize, verifyChecksum, "");
+    }
+
+    public static BlockReader newBlockReader( Socket sock, String file,
+                                       long blockId, 
+                                       AccessToken accessToken,
+                                       long genStamp,
+                                       long startOffset, long len,
+                                       int bufferSize, boolean verifyChecksum,
+                                       String clientName)
+                                       throws IOException {
+      // in and out will be closed when sock is closed (by the caller)
+      DataOutputStream out = new DataOutputStream(
+        new BufferedOutputStream(NetUtils.getOutputStream(sock,HdfsConstants.WRITE_TIMEOUT)));
+
+      //write the header.
+      out.writeShort( DataTransferProtocol.DATA_TRANSFER_VERSION );
+      out.write( DataTransferProtocol.OP_READ_BLOCK );
+      out.writeLong( blockId );
+      out.writeLong( genStamp );
+      out.writeLong( startOffset );
+      out.writeLong( len );
+      Text.writeString(out, clientName);
+      accessToken.write(out);
+      out.flush();
+      
+      //
+      // Get bytes in block, set streams
+      //
+
+      DataInputStream in = new DataInputStream(
+          new BufferedInputStream(NetUtils.getInputStream(sock), 
+                                  bufferSize));
+      
+      short status = in.readShort();
+      if (status != DataTransferProtocol.OP_STATUS_SUCCESS) {
+        if (status == DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN) {
+          throw new InvalidAccessTokenException(
+              "Got access token error in response to OP_READ_BLOCK "
+                  + "for file " + file + " for block " + blockId);
+        } else {
+          throw new IOException("Got error in response to OP_READ_BLOCK "
+              + "for file " + file + " for block " + blockId);
+        }
+      }
+      DataChecksum checksum = DataChecksum.newDataChecksum( in );
+      //Warning when we get CHECKSUM_NULL?
+      
+      // Read the first chunk offset.
+      long firstChunkOffset = in.readLong();
+      
+      if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
+          firstChunkOffset >= (startOffset + checksum.getBytesPerChecksum())) {
+        throw new IOException("BlockReader: error in first chunk offset (" +
+                              firstChunkOffset + ") startOffset is " + 
+                              startOffset + " for file " + file);
+      }
+
+      return new BlockReader( file, blockId, in, checksum, verifyChecksum,
+                              startOffset, firstChunkOffset, sock );
+    }
+
+    @Override
+    public synchronized void close() throws IOException {
+      startOffset = -1;
+      checksum = null;
+      // in will be closed when its Socket is closed.
+    }
+    
+    /** kind of like readFully(). Only reads as much as possible.
+     * And allows use of protected readFully().
+     */
+    public int readAll(byte[] buf, int offset, int len) throws IOException {
+      return readFully(this, buf, offset, len);
+    }
+    
+    /* When the reader reaches end of a block and there are no checksum
+     * errors, we send OP_STATUS_CHECKSUM_OK to datanode to inform that 
+     * checksum was verified and there was no error.
+     */ 
+    private void checksumOk(Socket sock) {
+      try {
+        OutputStream out = NetUtils.getOutputStream(sock, HdfsConstants.WRITE_TIMEOUT);
+        byte buf[] = { (DataTransferProtocol.OP_STATUS_CHECKSUM_OK >>> 8) & 0xff,
+                       (DataTransferProtocol.OP_STATUS_CHECKSUM_OK) & 0xff };
+        out.write(buf);
+        out.flush();
+      } catch (IOException e) {
+        // its ok not to be able to send this.
+        LOG.debug("Could not write to datanode " + sock.getInetAddress() +
+                  ": " + e.getMessage());
+      }
+    }
+  }
+    
+  /****************************************************************
+   * DFSInputStream provides bytes from a named file.  It handles 
+   * negotiation of the namenode and various datanodes as necessary.
+   ****************************************************************/
+  class DFSInputStream extends FSInputStream {
+    private Socket s = null;
+    private boolean closed = false;
+
+    private String src;
+    private long prefetchSize = 10 * defaultBlockSize;
+    private BlockReader blockReader = null;
+    private boolean verifyChecksum;
+    private LocatedBlocks locatedBlocks = null;
+    private DatanodeInfo currentNode = null;
+    private Block currentBlock = null;
+    private long pos = 0;
+    private long blockEnd = -1;
+    private int failures = 0;
+
+    /* XXX Use of CocurrentHashMap is temp fix. Need to fix 
+     * parallel accesses to DFSInputStream (through ptreads) properly */
+    private ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes = 
+               new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
+    private int buffersize = 1;
+    
+    private byte[] oneByteBuf = new byte[1]; // used for 'int read()'
+    
+    void addToDeadNodes(DatanodeInfo dnInfo) {
+      deadNodes.put(dnInfo, dnInfo);
+    }
+    
+    DFSInputStream(String src, int buffersize, boolean verifyChecksum
+                   ) throws IOException {
+      this.verifyChecksum = verifyChecksum;
+      this.buffersize = buffersize;
+      this.src = src;
+      prefetchSize = conf.getLong("dfs.read.prefetch.size", prefetchSize);
+      openInfo();
+    }
+
+    /**
+     * Grab the open-file info from namenode
+     */
+    synchronized void openInfo() throws IOException {
+      LocatedBlocks newInfo = callGetBlockLocations(namenode, src, 0, prefetchSize);
+      if (newInfo == null) {
+        throw new IOException("Cannot open filename " + src);
+      }
+
+      if (locatedBlocks != null) {
+        Iterator<LocatedBlock> oldIter = locatedBlocks.getLocatedBlocks().iterator();
+        Iterator<LocatedBlock> newIter = newInfo.getLocatedBlocks().iterator();
+        while (oldIter.hasNext() && newIter.hasNext()) {
+          if (! oldIter.next().getBlock().equals(newIter.next().getBlock())) {
+            throw new IOException("Blocklist for " + src + " has changed!");
+          }
+        }
+      }
+      this.locatedBlocks = newInfo;
+      this.currentNode = null;
+    }
+    
+    public synchronized long getFileLength() {
+      return (locatedBlocks == null) ? 0 : locatedBlocks.getFileLength();
+    }
+
+    /**
+     * Returns the datanode from which the stream is currently reading.
+     */
+    public DatanodeInfo getCurrentDatanode() {
+      return currentNode;
+    }
+
+    /**
+     * Returns the block containing the target position. 
+     */
+    public Block getCurrentBlock() {
+      return currentBlock;
+    }
+
+    /**
+     * Return collection of blocks that has already been located.
+     */
+    synchronized List<LocatedBlock> getAllBlocks() throws IOException {
+      return getBlockRange(0, this.getFileLength());
+    }
+
+    /**
+     * Get block at the specified position.
+     * Fetch it from the namenode if not cached.
+     * 
+     * @param offset
+     * @return located block
+     * @throws IOException
+     */
+    private synchronized LocatedBlock getBlockAt(long offset) throws IOException {
+      assert (locatedBlocks != null) : "locatedBlocks is null";
+      // search cached blocks first
+      int targetBlockIdx = locatedBlocks.findBlock(offset);
+      if (targetBlockIdx < 0) { // block is not cached
+        targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
+        // fetch more blocks
+        LocatedBlocks newBlocks;
+        newBlocks = callGetBlockLocations(namenode, src, offset, prefetchSize);
+        assert (newBlocks != null) : "Could not find target position " + offset;
+        locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
+      }
+      LocatedBlock blk = locatedBlocks.get(targetBlockIdx);
+      // update current position
+      this.pos = offset;
+      this.blockEnd = blk.getStartOffset() + blk.getBlockSize() - 1;
+      this.currentBlock = blk.getBlock();
+      return blk;
+    }
+
+    /** Fetch a block from namenode and cache it */
+    private synchronized void fetchAndCacheBlockAt(long offset) throws IOException {
+      int targetBlockIdx = locatedBlocks.findBlock(offset);
+      if (targetBlockIdx < 0) { // block is not cached
+        targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
+      }
+      // fetch blocks
+      LocatedBlocks newBlocks;
+      newBlocks = callGetBlockLocations(namenode, src, offset, prefetchSize);
+      if (newBlocks == null) {
+        throw new IOException("Could not find target position " + offset);
+      }
+      locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
+    }
+
+    /** Fetch a block without caching */
+    private LocatedBlock fetchBlockAt(long offset) throws IOException {
+      LocatedBlocks newBlocks;
+      newBlocks = callGetBlockLocations(namenode, src, offset, prefetchSize);
+      if (newBlocks == null) {
+        throw new IOException("Could not find target position " + offset);
+      }
+      int index = newBlocks.findBlock(offset);
+      return newBlocks.get(index);
+    }
+    
+    /**
+     * Get blocks in the specified range.
+     * Fetch them from the namenode if not cached.
+     * 
+     * @param offset
+     * @param length
+     * @return consequent segment of located blocks
+     * @throws IOException
+     */
+    private synchronized List<LocatedBlock> getBlockRange(long offset, 
+                                                          long length) 
+                                                        throws IOException {
+      assert (locatedBlocks != null) : "locatedBlocks is null";
+      List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
+      // search cached blocks first
+      int blockIdx = locatedBlocks.findBlock(offset);
+      if (blockIdx < 0) { // block is not cached
+        blockIdx = LocatedBlocks.getInsertIndex(blockIdx);
+      }
+      long remaining = length;
+      long curOff = offset;
+      while(remaining > 0) {
+        LocatedBlock blk = null;
+        if(blockIdx < locatedBlocks.locatedBlockCount())
+          blk = locatedBlocks.get(blockIdx);
+        if (blk == null || curOff < blk.getStartOffset()) {
+          LocatedBlocks newBlocks;
+          newBlocks = callGetBlockLocations(namenode, src, curOff, remaining);
+          locatedBlocks.insertRange(blockIdx, newBlocks.getLocatedBlocks());
+          continue;
+        }
+        assert curOff >= blk.getStartOffset() : "Block not found";
+        blockRange.add(blk);
+        long bytesRead = blk.getStartOffset() + blk.getBlockSize() - curOff;
+        remaining -= bytesRead;
+        curOff += bytesRead;
+        blockIdx++;
+      }
+      return blockRange;
+    }
+
+    /**
+     * Open a DataInputStream to a DataNode so that it can be read from.
+     * We get block ID and the IDs of the destinations at startup, from the namenode.
+     */
+    private synchronized DatanodeInfo blockSeekTo(long target) throws IOException {
+      if (target >= getFileLength()) {
+        throw new IOException("Attempted to read past end of file");
+      }
+
+      if ( blockReader != null ) {
+        blockReader.close(); 
+        blockReader = null;
+      }
+      
+      if (s != null) {
+        s.close();
+        s = null;
+      }
+
+      //
+      // Connect to best DataNode for desired Block, with potential offset
+      //
+      DatanodeInfo chosenNode = null;
+      int refetchToken = 1; // only need to get a new access token once
+      while (true) {
+        //
+        // Compute desired block
+        //
+        LocatedBlock targetBlock = getBlockAt(target);
+        assert (target==this.pos) : "Wrong postion " + pos + " expect " + target;
+        long offsetIntoBlock = target - targetBlock.getStartOffset();
+
+        DNAddrPair retval = chooseDataNode(targetBlock);
+        chosenNode = retval.info;
+        InetSocketAddress targetAddr = retval.addr;
+
+        try {
+          s = socketFactory.createSocket();
+          NetUtils.connect(s, targetAddr, socketTimeout);
+          s.setSoTimeout(socketTimeout);
+          Block blk = targetBlock.getBlock();
+          AccessToken accessToken = targetBlock.getAccessToken();
+          
+          blockReader = BlockReader.newBlockReader(s, src, blk.getBlockId(), 
+              accessToken, 
+              blk.getGenerationStamp(),
+              offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock,
+              buffersize, verifyChecksum, clientName);
+          return chosenNode;
+        } catch (IOException ex) {
+          LOG.debug("Failed to connect to " + targetAddr + ":" 
+                    + StringUtils.stringifyException(ex));
+          if (ex instanceof InvalidAccessTokenException && refetchToken-- > 0) {
+            /*
+             * Get a new access token and retry. Retry is needed in 2 cases. 1)
+             * When both NN and DN re-started while DFSClient holding a cached
+             * access token. 2) In the case that NN fails to update its
+             * access key at pre-set interval (by a wide margin) and
+             * subsequently restarts. In this case, DN re-registers itself with
+             * NN and receives a new access key, but DN will delete the old
+             * access key from its memory since it's considered expired based on
+             * the estimated expiration date.
+             */
+            fetchAndCacheBlockAt(target);
+          } else {
+            // Put chosen node into dead list, continue
+            addToDeadNodes(chosenNode);
+          }
+          if (s != null) {
+            try {
+              s.close();
+            } catch (IOException iex) {
+            }                        
+          }
+          s = null;
+        }
+      }
+    }
+
+    /**
+     * Close it down!
+     */
+    @Override
+    public synchronized void close() throws IOException {
+      if (closed) {
+        return;
+      }
+      checkOpen();
+      
+      if ( blockReader != null ) {
+        blockReader.close();
+        blockReader = null;
+      }
+      
+      if (s != null) {
+        s.close();
+        s = null;
+      }
+      super.close();
+      closed = true;
+    }
+
+    @Override
+    public synchronized int read() throws IOException {
+      int ret = read( oneByteBuf, 0, 1 );
+      return ( ret <= 0 ) ? -1 : (oneByteBuf[0] & 0xff);
+    }
+
+    /* This is a used by regular read() and handles ChecksumExceptions.
+     * name readBuffer() is chosen to imply similarity to readBuffer() in
+     * ChecksuFileSystem
+     */ 
+    private synchronized int readBuffer(byte buf[], int off, int len) 
+                                                    throws IOException {
+      IOException ioe;
+      
+      /* we retry current node only once. So this is set to true only here.
+       * Intention is to handle one common case of an error that is not a
+       * failure on datanode or client : when DataNode closes the connection
+       * since client is idle. If there are other cases of "non-errors" then
+       * then a datanode might be retried by setting this to true again.
+       */
+      boolean retryCurrentNode = true;
+ 
+      while (true) {
+        // retry as many times as seekToNewSource allows.
+        try {
+          return blockReader.read(buf, off, len);
+        } catch ( ChecksumException ce ) {
+          LOG.warn("Found Checksum error for " + currentBlock + " from " +
+                   currentNode.getName() + " at " + ce.getPos());          
+          reportChecksumFailure(src, currentBlock, currentNode);
+          ioe = ce;
+          retryCurrentNode = false;
+        } catch ( IOException e ) {
+          if (!retryCurrentNode) {
+            LOG.warn("Exception while reading from " + currentBlock +
+                     " of " + src + " from " + currentNode + ": " +
+                     StringUtils.stringifyException(e));
+          }
+          ioe = e;
+        }
+        boolean sourceFound = false;
+        if (retryCurrentNode) {
+          /* possibly retry the same node so that transient errors don't
+           * result in application level failures (e.g. Datanode could have
+           * closed the connection because the client is idle for too long).
+           */ 
+          sourceFound = seekToBlockSource(pos);
+        } else {
+          addToDeadNodes(currentNode);
+          sourceFound = seekToNewSource(pos);
+        }
+        if (!sourceFound) {
+          throw ioe;
+        }
+        retryCurrentNode = false;
+      }
+    }
+
+    /**
+     * Read the entire buffer.
+     */
+    @Override
+    public synchronized int read(byte buf[], int off, int len) throws IOException {
+      checkOpen();
+      if (closed) {
+        throw new IOException("Stream closed");
+      }
+      if (pos < getFileLength()) {
+        int retries = 2;
+        while (retries > 0) {
+          try {
+            if (pos > blockEnd) {
+              currentNode = blockSeekTo(pos);
+            }
+            int realLen = Math.min(len, (int) (blockEnd - pos + 1));
+            int result = readBuffer(buf, off, realLen);
+            
+            if (result >= 0) {
+              pos += result;
+            } else {
+              // got a EOS from reader though we expect more data on it.
+              throw new IOException("Unexpected EOS from the reader");
+            }
+            if (stats != null && result != -1) {
+              stats.incrementBytesRead(result);
+            }
+            return result;
+          } catch (ChecksumException ce) {
+            throw ce;            
+          } catch (IOException e) {
+            if (retries == 1) {
+              LOG.warn("DFS Read: " + StringUtils.stringifyException(e));
+            }
+            blockEnd = -1;
+            if (currentNode != null) { addToDeadNodes(currentNode); }
+            if (--retries == 0) {
+              throw e;
+            }
+          }
+        }
+      }
+      return -1;
+    }
+
+        
+    private DNAddrPair chooseDataNode(LocatedBlock block)
+      throws IOException {
+      while (true) {
+        DatanodeInfo[] nodes = block.getLocations();
+        try {
+          DatanodeInfo chosenNode = bestNode(nodes, deadNodes);
+          InetSocketAddress targetAddr = 
+                            NetUtils.createSocketAddr(chosenNode.getName());
+          return new DNAddrPair(chosenNode, targetAddr);
+        } catch (IOException ie) {
+          String blockInfo = block.getBlock() + " file=" + src;
+          if (failures >= maxBlockAcquireFailures) {
+            throw new IOException("Could not obtain block: " + blockInfo);
+          }
+          
+          if (nodes == null || nodes.length == 0) {
+            LOG.info("No node available for block: " + blockInfo);
+          }
+          LOG.info("Could not obtain block " + block.getBlock() + " from any node:  " + ie);
+          try {
+            Thread.sleep(3000);
+          } catch (InterruptedException iex) {
+          }
+          deadNodes.clear(); //2nd option is to remove only nodes[blockId]
+          openInfo();
+          failures++;
+          continue;
+        }
+      }
+    } 
+        
+    private void fetchBlockByteRange(LocatedBlock block, long start,
+                                     long end, byte[] buf, int offset) throws IOException {
+      //
+      // Connect to best DataNode for desired Block, with potential offset
+      //
+      Socket dn = null;
+      int numAttempts = block.getLocations().length;
+      IOException ioe = null;
+      int refetchToken = 1; // only need to get a new access token once
+      
+      while (dn == null && numAttempts-- > 0 ) {
+        DNAddrPair retval = chooseDataNode(block);
+        DatanodeInfo chosenNode = retval.info;
+        InetSocketAddress targetAddr = retval.addr;
+        BlockReader reader = null;
+            
+        try {
+          dn = socketFactory.createSocket();
+          NetUtils.connect(dn, targetAddr, socketTimeout);
+          dn.setSoTimeout(socketTimeout);
+          AccessToken accessToken = block.getAccessToken();
+              
+          int len = (int) (end - start + 1);
+              
+          reader = BlockReader.newBlockReader(dn, src, 
+                                              block.getBlock().getBlockId(),
+                                              accessToken,
+                                              block.getBlock().getGenerationStamp(),
+                                              start, len, buffersize, 
+                                              verifyChecksum, clientName);
+          int nread = reader.readAll(buf, offset, len);
+          if (nread != len) {
+            throw new IOException("truncated return from reader.read(): " +
+                                  "excpected " + len + ", got " + nread);
+          }
+          return;
+        } catch (ChecksumException e) {
+          ioe = e;
+          LOG.warn("fetchBlockByteRange(). Got a checksum exception for " +
+                   src + " at " + block.getBlock() + ":" + 
+                   e.getPos() + " from " + chosenNode.getName());
+          reportChecksumFailure(src, block.getBlock(), chosenNode);
+        } catch (IOException e) {
+          ioe = e;
+          if (e instanceof InvalidAccessTokenException && refetchToken-- > 0) {
+            LOG.info("Invalid access token when connecting to " + targetAddr
+                + " for file " + src + " for block "
+                + block.getBlock() + ":"
+                + StringUtils.stringifyException(e)
+                + ", get a new access token and retry...");
+            block = fetchBlockAt(block.getStartOffset());
+            numAttempts = block.getLocations().length;
+            continue;
+          } else {
+            LOG.warn("Failed to connect to " + targetAddr + " for file " + src
+                + " for block " + block.getBlock() + ":"
+                + StringUtils.stringifyException(e));
+          }
+        } finally {
+          IOUtils.closeStream(reader);
+          IOUtils.closeSocket(dn);
+          dn = null;
+        }
+        // Put chosen node into dead list, continue
+        addToDeadNodes(chosenNode);
+      }
+      throw (ioe == null) ? new IOException("Could not read data") : ioe;
+    }
+
+    /**
+     * Read bytes starting from the specified position.
+     * 
+     * @param position start read from this position
+     * @param buffer read buffer
+     * @param offset offset into buffer
+     * @param length number of bytes to read
+     * 
+     * @return actual number of bytes read
+     */
+    @Override
+    public int read(long position, byte[] buffer, int offset, int length)
+      throws IOException {
+      // sanity checks
+      checkOpen();
+      if (closed) {
+        throw new IOException("Stream closed");
+      }
+      long filelen = getFileLength();
+      if ((position < 0) || (position >= filelen)) {
+        return -1;
+      }
+      int realLen = length;
+      if ((position + length) > filelen) {
+        realLen = (int)(filelen - position);
+      }
+      
+      // determine the block and byte range within the block
+      // corresponding to position and realLen
+      List<LocatedBlock> blockRange = getBlockRange(position, realLen);
+      int remaining = realLen;
+      for (LocatedBlock blk : blockRange) {
+        long targetStart = position - blk.getStartOffset();
+        long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
+        fetchBlockByteRange(blk, targetStart, 
+                            targetStart + bytesToRead - 1, buffer, offset);
+        remaining -= bytesToRead;
+        position += bytesToRead;
+        offset += bytesToRead;
+      }
+      assert remaining == 0 : "Wrong number of bytes read.";
+      if (stats != null) {
+        stats.incrementBytesRead(realLen);
+      }
+      return realLen;
+    }
+     
+    @Override
+    public long skip(long n) throws IOException {
+      if ( n > 0 ) {
+        long curPos = getPos();
+        long fileLen = getFileLength();
+        if( n+curPos > fileLen ) {
+          n = fileLen - curPos;
+        }
+        seek(curPos+n);
+        return n;
+      }
+      return n < 0 ? -1 : 0;
+    }
+
+    /**
+     * Seek to a new arbitrary location
+     */
+    @Override
+    public synchronized void seek(long targetPos) throws IOException {
+      if (targetPos > getFileLength()) {
+        throw new IOException("Cannot seek after EOF");
+      }
+      boolean done = false;
+      if (pos <= targetPos && targetPos <= blockEnd) {
+        //
+        // If this seek is to a positive position in the current
+        // block, and this piece of data might already be lying in
+        // the TCP buffer, then just eat up the intervening data.
+        //
+        int diff = (int)(targetPos - pos);
+        if (diff <= TCP_WINDOW_SIZE) {
+          try {
+            pos += blockReader.skip(diff);
+            if (pos == targetPos) {
+              done = true;
+            }
+          } catch (IOException e) {//make following read to retry
+            LOG.debug("Exception while seek to " + targetPos + " from "
+                      + currentBlock +" of " + src + " from " + currentNode + 
+                      ": " + StringUtils.stringifyException(e));
+          }
+        }
+      }
+      if (!done) {
+        pos = targetPos;
+        blockEnd = -1;
+      }
+    }
+
+    /**
+     * Same as {@link #seekToNewSource(long)} except that it does not exclude
+     * the current datanode and might connect to the same node.
+     */
+    private synchronized boolean seekToBlockSource(long targetPos)
+                                                   throws IOException {
+      currentNode = blockSeekTo(targetPos);
+      return true;
+    }
+    
+    /**
+     * Seek to given position on a node other than the current node.  If
+     * a node other than the current node is found, then returns true. 
+     * If another node could not be found, then returns false.
+     */
+    @Override
+    public synchronized boolean seekToNewSource(long targetPos) throws IOException {
+      boolean markedDead = deadNodes.containsKey(currentNode);
+      addToDeadNodes(currentNode);
+      DatanodeInfo oldNode = currentNode;
+      DatanodeInfo newNode = blockSeekTo(targetPos);
+      if (!markedDead) {
+        /* remove it from deadNodes. blockSeekTo could have cleared 
+         * deadNodes and added currentNode again. Thats ok. */
+        deadNodes.remove(oldNode);
+      }
+      if (!oldNode.getStorageID().equals(newNode.getStorageID())) {
+        currentNode = newNode;
+        return true;
+      } else {
+        return false;
+      }
+    }
+        
+    /**
+     */
+    @Override
+    public synchronized long getPos() throws IOException {
+      return pos;
+    }
+
+    /**
+     */
+    @Override
+    public synchronized int available() throws IOException {
+      if (closed) {
+        throw new IOException("Stream closed");
+      }
+      return (int) (getFileLength() - pos);
+    }
+
+    /**
+     * We definitely don't support marks
+     */
+    @Override
+    public boolean markSupported() {
+      return false;
+    }
+    @Override
+    public void mark(int readLimit) {
+    }
+    @Override
+    public void reset() throws IOException {
+      throw new IOException("Mark/reset not supported");
+    }
+  }
+    
+  static class DFSDataInputStream extends FSDataInputStream {
+    DFSDataInputStream(DFSInputStream in)
+      throws IOException {
+      super(in);
+    }
+      
+    /**
+     * Returns the datanode from which the stream is currently reading.
+     */
+    public DatanodeInfo getCurrentDatanode() {
+      return ((DFSInputStream)in).getCurrentDatanode();
+    }
+      
+    /**
+     * Returns the block containing the target position. 
+     */
+    public Block getCurrentBlock() {
+      return ((DFSInputStream)in).getCurrentBlock();
+    }
+
+    /**
+     * Return collection of blocks that has already been located.
+     */
+    synchronized List<LocatedBlock> getAllBlocks() throws IOException {
+      return ((DFSInputStream)in).getAllBlocks();
+    }
+
+  }
+
+  /****************************************************************
+   * DFSOutputStream creates files from a stream of bytes.
+   *
+   * The client application writes data that is cached internally by
+   * this stream. Data is broken up into packets, each packet is
+   * typically 64K in size. A packet comprises of chunks. Each chunk
+   * is typically 512 bytes and has an associated checksum with it.
+   *
+   * When a client application fills up the currentPacket, it is
+   * enqueued into dataQueue.  The DataStreamer thread picks up
+   * packets from the dataQueue, sends it to the first datanode in
+   * the pipeline and moves it from the dataQueue to the ackQueue.
+   * The ResponseProcessor receives acks from the datanodes. When an
+   * successful ack for a packet is received from all datanodes, the
+   * ResponseProcessor removes the corresponding packet from the
+   * ackQueue.
+   *
+   * In case of error, all outstanding packets and moved from
+   * ackQueue. A new pipeline is setup by eliminating the bad
+   * datanode from the original pipeline. The DataStreamer now
+   * starts sending packets from the dataQueue.
+  ****************************************************************/
+  class DFSOutputStream extends FSOutputSummer implements Syncable {
+    private Socket s;
+    // closed is accessed by different threads under different locks.
+    volatile boolean closed = false;
+  
+    private String src;
+    private DataOutputStream blockStream;
+    private DataInputStream blockReplyStream;
+    private Block block;
+    private AccessToken accessToken;
+    final private long blockSize;
+    private DataChecksum checksum;
+    private LinkedList<Packet> dataQueue = new LinkedList<Packet>();
+    private LinkedList<Packet> ackQueue = new LinkedList<Packet>();
+    private Packet currentPacket = null;
+    private int maxPackets = 80; // each packet 64K, total 5MB
+    // private int maxPackets = 1000; // each packet 64K, total 64MB
+    private DataStreamer streamer = new DataStreamer();;
+    private ResponseProcessor response = null;
+    private long currentSeqno = 0;
+    private long bytesCurBlock = 0; // bytes writen in current block
+    private int packetSize = 0; // write packet size, including the header.
+    private int chunksPerPacket = 0;
+    private DatanodeInfo[] nodes = null; // list of targets for current block
+    private volatile boolean hasError = false;
+    private volatile int errorIndex = 0;
+    private volatile IOException lastException = null;
+    private long artificialSlowdown = 0;
+    private long lastFlushOffset = -1; // offset when flush was invoked
+    //volatile: written holding dataQueue and read holding DFSOutputStream 
+    private volatile boolean persistBlocks = false;//persist blocks on namenode
+    private int recoveryErrorCount = 0; // number of times block recovery failed
+    private int maxRecoveryErrorCount = 5; // try block recovery 5 times
+    private volatile boolean appendChunk = false;   // appending to existing partial block
+    private long initialFileSize = 0; // at time of file open
+
+    private void setLastException(IOException e) {
+      if (lastException == null) {
+        lastException = e;
+      }
+    }
+    
+    private class Packet {
+      ByteBuffer buffer;           // only one of buf and buffer is non-null
+      byte[]  buf;
+      long    seqno;               // sequencenumber of buffer in block
+      long    offsetInBlock;       // offset in block
+      boolean lastPacketInBlock;   // is this the last packet in block?
+      int     numChunks;           // number of chunks currently in packet
+      int     maxChunks;           // max chunks in packet
+      int     dataStart;
+      int     dataPos;
+      int     checksumStart;
+      int     checksumPos;      
+  
+      // create a new packet
+      Packet(int pktSize, int chunksPerPkt, long offsetInBlock) {
+        this.lastPacketInBlock = false;
+        this.numChunks = 0;
+        this.offsetInBlock = offsetInBlock;
+        this.seqno = currentSeqno;
+        currentSeqno++;
+        
+        buffer = null;
+        buf = new byte[pktSize];
+        
+        checksumStart = DataNode.PKT_HEADER_LEN + SIZE_OF_INTEGER;
+        checksumPos = checksumStart;
+        dataStart = checksumStart + chunksPerPkt * checksum.getChecksumSize();
+        dataPos = dataStart;
+        maxChunks = chunksPerPkt;
+      }
+
+      void writeData(byte[] inarray, int off, int len) {
+        if ( dataPos + len > buf.length) {
+          throw new BufferOverflowException();
+        }
+        System.arraycopy(inarray, off, buf, dataPos, len);
+        dataPos += len;
+      }
+  
+      void  writeChecksum(byte[] inarray, int off, int len) {
+        if (checksumPos + len > dataStart) {
+          throw new BufferOverflowException();
+        }
+        System.arraycopy(inarray, off, buf, checksumPos, len);
+        checksumPos += len;
+      }
+      
+      /**
+       * Returns ByteBuffer that contains one full packet, including header.
+       */
+      ByteBuffer getBuffer() {
+        /* Once this is called, no more data can be added to the packet.
+         * setting 'buf' to null ensures that.
+         * This is called only when the packet is ready to be sent.
+         */
+        if (buffer != null) {
+          return buffer;
+        }
+        
+        //prepare the header and close any gap between checksum and data.
+        
+        int dataLen = dataPos - dataStart;
+        int checksumLen = checksumPos - checksumStart;
+        
+        if (checksumPos != dataStart) {
+          /* move the checksum to cover the gap.
+           * This can happen for the last packet.
+           */
+          System.arraycopy(buf, checksumStart, buf, 
+                           dataStart - checksumLen , checksumLen); 
+        }
+        
+        int pktLen = SIZE_OF_INTEGER + dataLen + checksumLen;
+        
+        //normally dataStart == checksumPos, i.e., offset is zero.
+        buffer = ByteBuffer.wrap(buf, dataStart - checksumPos,
+                                 DataNode.PKT_HEADER_LEN + pktLen);
+        buf = null;
+        buffer.mark();
+        
+        /* write the header and data length.
+         * The format is described in comment before DataNode.BlockSender
+         */
+        buffer.putInt(pktLen);  // pktSize
+        buffer.putLong(offsetInBlock); 
+        buffer.putLong(seqno);
+        buffer.put((byte) ((lastPacketInBlock) ? 1 : 0));
+        //end of pkt header
+        buffer.putInt(dataLen); // actual data length, excluding checksum.
+        
+        buffer.reset();
+        return buffer;
+      }
+    }
+  
+    //
+    // The DataStreamer class is responsible for sending data packets to the
+    // datanodes in the pipeline. It retrieves a new blockid and block locations
+    // from the namenode, and starts streaming packets to the pipeline of
+    // Datanodes. Every packet has a sequence number associated with
+    // it. When all the packets for a block are sent out and acks for each
+    // if them are received, the DataStreamer closes the current block.
+    //
+    private class DataStreamer extends Daemon {
+
+      private volatile boolean closed = false;
+  
+      public void run() {
+        while (!closed && clientRunning) {
+
+          // if the Responder encountered an error, shutdown Responder
+          if (hasError && response != null) {
+            try {
+              response.close();
+              response.join();
+              response = null;
+            } catch (InterruptedException  e) {
+            }
+          }
+
+          Packet one = null;
+          synchronized (dataQueue) {
+
+            // process IO errors if any
+            boolean doSleep = processDatanodeError(hasError, false);
+
+            // wait for a packet to be sent.
+            while ((!closed && !hasError && clientRunning 
+                   && dataQueue.size() == 0) || doSleep) {
+              try {
+                dataQueue.wait(1000);
+              } catch (InterruptedException  e) {
+              }
+              doSleep = false;
+            }
+            if (closed || hasError || dataQueue.size() == 0 || !clientRunning) {
+              continue;
+            }
+
+            try {
+              // get packet to be sent.
+              one = dataQueue.getFirst();
+              long offsetInBlock = one.offsetInBlock;
+  
+              // get new block from namenode.
+              if (blockStream == null) {
+                LOG.debug("Allocating new block");
+                nodes = nextBlockOutputStream(src); 
+                this.setName("DataStreamer for file " + src +
+                             " block " + block);
+                response = new ResponseProcessor(nodes);
+                response.start();
+              }
+
+              if (offsetInBlock >= blockSize) {
+                throw new IOException("BlockSize " + blockSize +
+                                      " is smaller than data size. " +
+                                      " Offset of packet in block " + 
+                                      offsetInBlock +
+                                      " Aborting file " + src);
+              }
+
+              ByteBuffer buf = one.getBuffer();
+              
+              // move packet from dataQueue to ackQueue
+              dataQueue.removeFirst();
+              dataQueue.notifyAll();
+              synchronized (ackQueue) {
+                ackQueue.addLast(one);
+                ackQueue.notifyAll();
+              } 
+              
+              // write out data to remote datanode
+              blockStream.write(buf.array(), buf.position(), buf.remaining());
+              
+              if (one.lastPacketInBlock) {
+                blockStream.writeInt(0); // indicate end-of-block 
+              }
+              blockStream.flush();
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("DataStreamer block " + block +
+                          " wrote packet seqno:" + one.seqno +
+                          " size:" + buf.remaining() +
+                          " offsetInBlock:" + one.offsetInBlock + 
+                          " lastPacketInBlock:" + one.lastPacketInBlock);
+              }
+            } catch (Throwable e) {
+              LOG.warn("DataStreamer Exception: " + 
+                       StringUtils.stringifyException(e));
+              if (e instanceof IOException) {
+                setLastException((IOException)e);
+              }
+              hasError = true;
+            }
+          }
+
+          if (closed || hasError || !clientRunning) {
+            continue;
+          }
+
+          // Is this block full?
+          if (one.lastPacketInBlock) {
+            synchronized (ackQueue) {
+              while (!hasError && ackQueue.size() != 0 && clientRunning) {
+                try {
+                  ackQueue.wait();   // wait for acks to arrive from datanodes
+                } catch (InterruptedException  e) {
+                }
+              }
+            }
+            LOG.debug("Closing old block " + block);
+            this.setName("DataStreamer for file " + src);
+
+            response.close();        // ignore all errors in Response
+            try {
+              response.join();
+              response = null;
+            } catch (InterruptedException  e) {
+            }
+
+            if (closed || hasError || !clientRunning) {
+              continue;
+            }
+
+            synchronized (dataQueue) {
+              try {
+                blockStream.close();
+                blockReplyStream.close();
+              } catch (IOException e) {
+              }
+              nodes = null;
+              response = null;
+              blockStream = null;
+              blockReplyStream = null;
+            }
+          }
+          if (progress != null) { progress.progress(); }
+
+          // This is used by unit test to trigger race conditions.
+          if (artificialSlowdown != 0 && clientRunning) {
+            try { 
+              Thread.sleep(artificialSlowdown); 
+            } catch (InterruptedException e) {}
+          }
+        }
+      }
+
+      // shutdown thread
+      void close() {
+        closed = true;
+        synchronized (dataQueue) {
+          dataQueue.notifyAll();
+        }
+        synchronized (ackQueue) {
+          ackQueue.notifyAll();
+        }
+        this.interrupt();
+      }
+    }
+                  
+    //
+    // Processes reponses from the datanodes.  A packet is removed 
+    // from the ackQueue when its response arrives.
+    //
+    private class ResponseProcessor extends Thread {
+
+      private volatile boolean closed = false;
+      private DatanodeInfo[] targets = null;
+      private boolean lastPacketInBlock = false;
+
+      ResponseProcessor (DatanodeInfo[] targets) {
+        this.targets = targets;
+      }
+
+      public void run() {
+
+        this.setName("ResponseProcessor for block " + block);
+  
+        while (!closed && clientRunning && !lastPacketInBlock) {
+          // process responses from datanodes.
+          try {
+            // verify seqno from datanode
+            long seqno = blockReplyStream.readLong();
+            LOG.debug("DFSClient received ack for seqno " + seqno);
+            if (seqno == -1) {
+              continue;
+            } else if (seqno == -2) {
+              // no nothing
+            } else {
+              Packet one = null;
+              synchronized (ackQueue) {
+                one = ackQueue.getFirst();
+              }
+              if (one.seqno != seqno) {
+                throw new IOException("Responseprocessor: Expecting seqno " + 
+                                      " for block " + block +
+                                      one.seqno + " but received " + seqno);
+              }
+              lastPacketInBlock = one.lastPacketInBlock;
+            }
+
+            // processes response status from all datanodes.
+            for (int i = 0; i < targets.length && clientRunning; i++) {
+              short reply = blockReplyStream.readShort();
+              if (reply != DataTransferProtocol.OP_STATUS_SUCCESS) {
+                errorIndex = i; // first bad datanode
+                throw new IOException("Bad response " + reply +
+                                      " for block " + block +
+                                      " from datanode " + 
+                                      targets[i].getName());
+              }
+            }
+
+            synchronized (ackQueue) {
+              ackQueue.removeFirst();
+              ackQueue.notifyAll();
+            }
+          } catch (Exception e) {
+            if (!closed) {
+              hasError = true;
+              if (e instanceof IOException) {
+                setLastException((IOException)e);
+              }
+              LOG.warn("DFSOutputStream ResponseProcessor exception " + 
+                       " for block " + block +
+                        StringUtils.stringifyException(e));
+              closed = true;
+            }
+          }
+
+          synchronized (dataQueue) {
+            dataQueue.notifyAll();
+          }
+          synchronized (ackQueue) {
+            ackQueue.notifyAll();
+          }
+        }
+      }
+
+      void close() {
+        closed = true;
+        this.interrupt();
+      }
+    }
+
+    // If this stream has encountered any errors so far, shutdown 
+    // threads and mark stream as closed. Returns true if we should
+    // sleep for a while after returning from this call.
+    //
+    private boolean processDatanodeError(boolean hasError, boolean isAppend) {
+      if (!hasError) {
+        return false;
+      }
+      if (response != null) {
+        LOG.info("Error Recovery for block " + block +
+                 " waiting for responder to exit. ");
+        return true;
+      }
+      if (errorIndex >= 0) {
+        LOG.warn("Error Recovery for block " + block
+            + " bad datanode[" + errorIndex + "] "
+            + (nodes == null? "nodes == null": nodes[errorIndex].getName()));
+      }
+
+      if (blockStream != null) {
+        try {
+          blockStream.close();
+          blockReplyStream.close();
+        } catch (IOException e) {
+        }
+      }
+      blockStream = null;
+      blockReplyStream = null;
+
+      // move packets from ack queue to front of the data queue
+      synchronized (ackQueue) {
+        dataQueue.addAll(0, ackQueue);
+        ackQueue.clear();
+      }
+
+      boolean success = false;
+      while (!success && clientRunning) {
+        DatanodeInfo[] newnodes = null;
+        if (nodes == null) {
+          String msg = "Could not get block locations. " +
+                                          "Source file \"" + src
+                                          + "\" - Aborting...";
+          LOG.warn(msg);
+          setLastException(new IOException(msg));
+          closed = true;
+          if (streamer != null) streamer.close();
+          return false;
+        }
+        StringBuilder pipelineMsg = new StringBuilder();
+        for (int j = 0; j < nodes.length; j++) {
+          pipelineMsg.append(nodes[j].getName());
+          if (j < nodes.length - 1) {
+            pipelineMsg.append(", ");
+          }
+        }
+        // remove bad datanode from list of datanodes.
+        // If errorIndex was not set (i.e. appends), then do not remove 
+        // any datanodes
+        // 
+        if (errorIndex < 0) {
+          newnodes = nodes;
+        } else {
+          if (nodes.length <= 1) {
+            lastException = new IOException("All datanodes " + pipelineMsg + 
+                                            " are bad. Aborting...");
+            closed = true;
+            if (streamer != null) streamer.close();
+            return false;
+          }
+          LOG.warn("Error Recovery for block " + block +
+                   " in pipeline " + pipelineMsg + 
+                   ": bad datanode " + nodes[errorIndex].getName());
+          newnodes =  new DatanodeInfo[nodes.length-1];
+          System.arraycopy(nodes, 0, newnodes, 0, errorIndex);
+          System.arraycopy(nodes, errorIndex+1, newnodes, errorIndex,
+              newnodes.length-errorIndex);
+        }
+
+        // Tell the primary datanode to do error recovery 
+        // by stamping appropriate generation stamps.
+        //
+        LocatedBlock newBlock = null;
+        ClientDatanodeProtocol primary =  null;
+        DatanodeInfo primaryNode = null;
+        try {
+          // Pick the "least" datanode as the primary datanode to avoid deadlock.
+          primaryNode = Collections.min(Arrays.asList(newnodes));
+          primary = createClientDatanodeProtocolProxy(primaryNode, conf);
+          newBlock = primary.recoverBlock(block, isAppend, newnodes);
+        } catch (IOException e) {
+          recoveryErrorCount++;
+          if (recoveryErrorCount > maxRecoveryErrorCount) {
+            if (nodes.length > 1) {
+              // if the primary datanode failed, remove it from the list.
+              // The original bad datanode is left in the list because it is
+              // conservative to remove only one datanode in one iteration.
+              for (int j = 0; j < nodes.length; j++) {
+                if (nodes[j].equals(primaryNode)) {
+                  errorIndex = j; // forget original bad node.
+                }
+              }
+              // remove primary node from list
+              newnodes =  new DatanodeInfo[nodes.length-1];
+              System.arraycopy(nodes, 0, newnodes, 0, errorIndex);
+              System.arraycopy(nodes, errorIndex+1, newnodes, errorIndex,
+                               newnodes.length-errorIndex);
+              nodes = newnodes;
+              LOG.warn("Error Recovery for block " + block + " failed " +
+                       " because recovery from primary datanode " +
+                       primaryNode + " failed " + recoveryErrorCount +
+                       " times. " + " Pipeline was " + pipelineMsg +
+                       ". Marking primary datanode as bad.");
+              recoveryErrorCount = 0; 
+              errorIndex = -1;
+              return true;          // sleep when we return from here
+            }
+            String emsg = "Error Recovery for block " + block + " failed " +
+                          " because recovery from primary datanode " +
+                          primaryNode + " failed " + recoveryErrorCount + 
+                          " times. "  + " Pipeline was " + pipelineMsg +
+                          ". Aborting...";
+            LOG.warn(emsg);
+            lastException = new IOException(emsg);
+            closed = true;
+            if (streamer != null) streamer.close();
+            return false;       // abort with IOexception
+          } 
+          LOG.warn("Error Recovery for block " + block + " failed " +
+                   " because recovery from primary datanode " +
+                   primaryNode + " failed " + recoveryErrorCount +
+                   " times. "  + " Pipeline was " + pipelineMsg +
+                   ". Will retry...");
+          return true;          // sleep when we return from here
+        } finally {
+          RPC.stopProxy(primary);
+        }
+        recoveryErrorCount = 0; // block recovery successful
+
+        // If the block recovery generated a new generation stamp, use that
+        // from now on.  Also, setup new pipeline
+        //
+        if (newBlock != null) {
+          block = newBlock.getBlock();
+          accessToken = newBlock.getAccessToken();
+          nodes = newBlock.getLocations();
+        }
+
+        this.hasError = false;
+        lastException = null;
+        errorIndex = 0;
+        success = createBlockOutputStream(nodes, clientName, true);
+      }
+
+      response = new ResponseProcessor(nodes);
+      response.start();
+      return false; // do not sleep, continue processing
+    }
+
+    private void isClosed() throws IOException {
+      if (closed && lastException != null) {
+          throw lastException;
+      }
+    }
+
+    //
+    // returns the list of targets, if any, that is being currently used.
+    //
+    DatanodeInfo[] getPipeline() {
+      synchronized (dataQueue) {
+        if (nodes == null) {
+          return null;
+        }
+        DatanodeInfo[] value = new DatanodeInfo[nodes.length];
+        for (int i = 0; i < nodes.length; i++) {
+          value[i] = nodes[i];
+        }
+        return value;
+      }
+    }
+
+    private Progressable progress;
+
+    private DFSOutputStream(String src, long blockSize, Progressable progress,
+        int bytesPerChecksum) throws IOException {
+      super(new CRC32(), bytesPerChecksum, 4);
+      this.src = src;
+      this.blockSize = blockSize;
+      this.progress = progress;
+      if (progress != null) {
+        LOG.debug("Set non-null progress callback on DFSOutputStream "+src);
+      }
+      
+      if ( bytesPerChecksum < 1 || blockSize % bytesPerChecksum != 0) {
+        throw new IOException("io.bytes.per.checksum(" + bytesPerChecksum +
+                              ") and blockSize(" + blockSize + 
+                              ") do not match. " + "blockSize should be a " +
+                              "multiple of io.bytes.per.checksum");
+                              
+      }
+      checksum = DataChecksum.newDataChecksum(DataChecksum.CHECKSUM_CRC32, 
+                                              bytesPerChecksum);
+    }
+
+    /**
+     * Create a new output stream to the given DataNode.
+     * @see ClientProtocol#create(String, FsPermission, String, boolean, short, long)
+     */
+    DFSOutputStream(String src, FsPermission masked, boolean overwrite,
+        short replication, long blockSize, Progressable progress,
+        int buffersize, int bytesPerChecksum) throws IOException {
+      this(src, blockSize, progress, bytesPerChecksum);
+
+      computePacketChunkSize(writePacketSize, bytesPerChecksum);
+
+      try {
+        namenode.create(
+            src, masked, clientName, overwrite, replication, blockSize);
+      } catch(RemoteException re) {
+        throw re.unwrapRemoteException(AccessControlException.class,
+                                       QuotaExceededException.class);
+      }
+      streamer.start();
+    }
+  
+    /**
+     * Create a new output stream to the given DataNode.
+     * @see ClientProtocol#create(String, FsPermission, String, boolean, short, long)
+     */
+    DFSOutputStream(String src, int buffersize, Progressable progress,
+        LocatedBlock lastBlock, FileStatus stat,
+        int bytesPerChecksum) throws IOException {
+      this(src, stat.getBlockSize(), progress, bytesPerChecksum);
+      initialFileSize = stat.getLen(); // length of file when opened
+
+      //
+      // The last partial block of the file has to be filled.
+      //
+      if (lastBlock != null) {
+        block = lastBlock.getBlock();
+        long usedInLastBlock = stat.getLen() % blockSize;
+        int freeInLastBlock = (int)(blockSize - usedInLastBlock);
+
+        // calculate the amount of free space in the pre-existing 
+        // last crc chunk
+        int usedInCksum = (int)(stat.getLen() % bytesPerChecksum);
+        int freeInCksum = bytesPerChecksum - usedInCksum;
+
+        // if there is space in the last block, then we have to 
+        // append to that block
+        if (freeInLastBlock > blockSize) {
+          throw new IOException("The last block for file " + 
+                                src + " is full.");
+        }
+
+        // indicate that we are appending to an existing block
+        bytesCurBlock = lastBlock.getBlockSize();
+
+        if (usedInCksum > 0 && freeInCksum > 0) {
+          // if there is space in the last partial chunk, then 
+          // setup in such a way that the next packet will have only 
+          // one chunk that fills up the partial chunk.
+          //
+          computePacketChunkSize(0, freeInCksum);
+          resetChecksumChunk(freeInCksum);
+          this.appendChunk = true;
+        } else {
+          // if the remaining space in the block is smaller than 
+          // that expected size of of a packet, then create 
+          // smaller size packet.
+          //
+          computePacketChunkSize(Math.min(writePacketSize, freeInLastBlock), 
+                                 bytesPerChecksum);
+        }
+
+        // setup pipeline to append to the last block XXX retries??
+        nodes = lastBlock.getLocations();
+        errorIndex = -1;   // no errors yet.
+        if (nodes.length < 1) {
+          throw new IOException("Unable to retrieve blocks locations " +
+                                " for last block " + block +
+                                "of file " + src);
+                        
+        }
+        processDatanodeError(true, true);
+        streamer.start();
+      }
+      else {
+        computePacketChunkSize(writePacketSize, bytesPerChecksum);
+        streamer.start();
+      }
+    }
+
+    private void computePacketChunkSize(int psize, int csize) {
+      int chunkSize = csize + checksum.getChecksumSize();
+      int n = DataNode.PKT_HEADER_LEN + SIZE_OF_INTEGER;
+      chunksPerPacket = Math.max((psize - n + chunkSize-1)/chunkSize, 1);
+      packetSize = n + chunkSize*chunksPerPacket;
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("computePacketChunkSize: src=" + src +
+                  ", chunkSize=" + chunkSize +
+                  ", chunksPerPacket=" + chunksPerPacket +
+                  ", packetSize=" + packetSize);
+      }
+    }
+
+    /**
+     * Open a DataOutputStream to a DataNode so that it can be written to.
+     * This happens when a file is created and each time a new block is allocated.
+     * Must get block ID and the IDs of the destinations from the namenode.
+     * Returns the list of target datanodes.
+     */
+    private DatanodeInfo[] nextBlockOutputStream(String client) throws IOException {
+      LocatedBlock lb = null;
+      boolean retry = false;
+      DatanodeInfo[] nodes;
+      int count = conf.getInt("dfs.client.block.write.retries", 3);
+      boolean success;
+      do {
+        hasError = false;
+        lastException = null;
+        errorIndex = 0;
+        retry = false;
+        nodes = null;
+        success = false;
+                
+        long startTime = System.currentTimeMillis();
+        lb = locateFollowingBlock(startTime);
+        block = lb.getBlock();
+        accessToken = lb.getAccessToken();
+        nodes = lb.getLocations();
+  
+        //
+        // Connect to first DataNode in the list.
+        //
+        success = createBlockOutputStream(nodes, clientName, false);
+
+        if (!success) {
+          LOG.info("Abandoning block " + block);
+          namenode.abandonBlock(block, src, clientName);
+
+          // Connection failed.  Let's wait a little bit and retry
+          retry = true;
+          try {
+            if (System.currentTimeMillis() - startTime > 5000) {
+              LOG.info("Waiting to find target node: " + nodes[0].getName());
+            }
+            Thread.sleep(6000);
+          } catch (InterruptedException iex) {
+          }
+        }
+      } while (retry && --count >= 0);
+
+      if (!success) {
+        throw new IOException("Unable to create new block.");
+      }
+      return nodes;
+    }
+
+    // connects to the first datanode in the pipeline
+    // Returns true if success, otherwise return failure.
+    //
+    private boolean createBlockOutputStream(DatanodeInfo[] nodes, String client,
+                    boolean recoveryFlag) {
+      String firstBadLink = "";
+      if (LOG.isDebugEnabled()) {
+        for (int i = 0; i < nodes.length; i++) {
+          LOG.debug("pipeline = " + nodes[i].getName());
+        }
+      }
+
+      // persist blocks on namenode on next flush
+      persistBlocks = true;
+
+      try {
+        LOG.debug("Connecting to " + nodes[0].getName());
+        InetSocketAddress target = NetUtils.createSocketAddr(nodes[0].getName());
+        s = socketFactory.createSocket();
+        int timeoutValue = (socketTimeout > 0) ? 
+                           (3000 * nodes.length + socketTimeout) : 0;
+        NetUtils.connect(s, target, timeoutValue);
+        s.setSoTimeout(timeoutValue);
+        s.setSendBufferSize(DEFAULT_DATA_SOCKET_SIZE);
+        LOG.debug("Send buf size " + s.getSendBufferSize());
+        long writeTimeout = (datanodeWriteTimeout > 0) ? 
+             (HdfsConstants.WRITE_TIMEOUT_EXTENSION * nodes.length +
+              datanodeWriteTimeout) : 0;
+
+        //
+        // Xmit header info to datanode
+        //
+        DataOutputStream out = new DataOutputStream(
+            new BufferedOutputStream(NetUtils.getOutputStream(s, writeTimeout), 
+                                     DataNode.SMALL_BUFFER_SIZE));
+        blockReplyStream = new DataInputStream(NetUtils.getInputStream(s));
+
+        out.writeShort( DataTransferProtocol.DATA_TRANSFER_VERSION );
+        out.write( DataTransferProtocol.OP_WRITE_BLOCK );
+        out.writeLong( block.getBlockId() );
+        out.writeLong( block.getGenerationStamp() );
+        out.writeInt( nodes.length );
+        out.writeBoolean( recoveryFlag );       // recovery flag
+        Text.writeString( out, client );
+        out.writeBoolean(false); // Not sending src node information
+        out.writeInt( nodes.length - 1 );
+        for (int i = 1; i < nodes.length; i++) {
+          nodes[i].write(out);
+        }
+        accessToken.write(out);
+        checksum.writeHeader( out );
+        out.flush();
+
+        // receive ack for connect
+        firstBadLink = Text.readString(blockReplyStream);
+        if (firstBadLink.length() != 0) {
+          throw new IOException("Bad connect ack with firstBadLink " + firstBadLink);
+        }
+
+        blockStream = out;
+        return true;     // success
+
+      } catch (IOException ie) {
+
+        LOG.info("Exception in createBlockOutputStream " + ie);
+
+        // find the datanode that matches
+        if (firstBadLink.length() != 0) {
+          for (int i = 0; i < nodes.length; i++) {
+            if (nodes[i].getName().equals(firstBadLink)) {
+              errorIndex = i;
+              break;
+            }
+          }
+        }
+        hasError = true;
+        setLastException(ie);
+        blockReplyStream = null;
+        return false;  // error
+      }
+    }
+  
+    private LocatedBlock locateFollowingBlock(long start
+                                              ) throws IOException {     
+      int retries = 5;
+      long sleeptime = 400;
+      while (true) {
+        long localstart = System.currentTimeMillis();
+        while (true) {
+          try {
+            return namenode.addBlock(src, clientName);
+          } catch (RemoteException e) {
+            IOException ue = 
+              e.unwrapRemoteException(FileNotFoundException.class,
+                                      AccessControlException.class,
+                                      QuotaExceededException.class);
+            if (ue != e) { 
+              throw ue; // no need to retry these exceptions
+            }
+            
+            if (--retries == 0 && 
+                !NotReplicatedYetException.class.getName().
+                equals(e.getClassName())) {
+              throw e;
+            } else {
+              LOG.info(StringUtils.stringifyException(e));
+              if (System.currentTimeMillis() - localstart > 5000) {
+                LOG.info("Waiting for replication for " + 
+                         (System.currentTimeMillis() - localstart)/1000 + 
+                         " seconds");
+              }
+              try {
+                LOG.warn("NotReplicatedYetException sleeping " + src +
+                          " retries left " + retries);
+                Thread.sleep(sleeptime);
+                sleeptime *= 2;
+              } catch (InterruptedException ie) {
+              }
+            }                
+          }
+        }
+      } 
+    }
+  
+    // @see FSOutputSummer#writeChunk()
+    @Override
+    protected synchronized void writeChunk(byte[] b, int offset, int len, byte[] checksum) 
+                                                          throws IOException {
+      checkOpen();
+      isClosed();
+  
+      int cklen = checksum.length;
+      int bytesPerChecksum = this.checksum.getBytesPerChecksum(); 
+      if (len > bytesPerChecksum) {
+        throw new IOException("writeChunk() buffer size is " + len +
+                              " is larger than supported  bytesPerChecksum " +
+                              bytesPerChecksum);
+      }
+      if (checksum.length != this.checksum.getChecksumSize()) {
+        throw new IOException("writeChunk() checksum size is supposed to be " +
+                              this.checksum.getChecksumSize() + 
+                              " but found to be " + checksum.length);
+      }
+
+      synchronized (dataQueue) {
+  
+        // If queue is full, then wait till we can create  enough space
+        while (!closed && dataQueue.size() + ackQueue.size()  > maxPackets) {
+          try {
+            dataQueue.wait();
+          } catch (InterruptedException  e) {
+          }
+        }
+        isClosed();
+  
+        if (currentPacket == null) {
+          currentPacket = new Packet(packetSize, chunksPerPacket, 
+                                     bytesCurBlock);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("DFSClient writeChunk allocating new packet seqno=" + 
+                      currentPacket.seqno +
+                      ", src=" + src +
+                      ", packetSize=" + packetSize +
+                      ", chunksPerPacket=" + chunksPerPacket +
+                      ", bytesCurBlock=" + bytesCurBlock);
+          }
+        }
+
+        currentPacket.writeChecksum(checksum, 0, cklen);
+        currentPacket.writeData(b, offset, len);
+        currentPacket.numChunks++;
+        bytesCurBlock += len;
+
+        // If packet is full, enqueue it for transmission
+        //
+        if (currentPacket.numChunks == currentPacket.maxChunks ||
+            bytesCurBlock == blockSize) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("DFSClient writeChunk packet full seqno=" +
+                      currentPacket.seqno +
+                      ", src=" + src +
+                      ", bytesCurBlock=" + bytesCurBlock +
+                      ", blockSize=" + blockSize +
+                      ", appendChunk=" + appendChunk);
+          }
+          //
+          // if we allocated a new packet because we encountered a block
+          // boundary, reset bytesCurBlock.
+          //
+          if (bytesCurBlock == blockSize) {
+            currentPacket.lastPacketInBlock = true;
+            bytesCurBlock = 0;
+            lastFlushOffset = -1;
+          }
+          dataQueue.addLast(currentPacket);
+          dataQueue.notifyAll();
+          currentPacket = null;
+ 
+          // If this was the first write after reopening a file, then the above
+          // write filled up any partial chunk. Tell the summer to generate full 
+          // crc chunks from now on.
+          if (appendChunk) {
+            appendChunk = false;
+            resetChecksumChunk(bytesPerChecksum);
+          }
+          int psize = Math.min((int)(blockSize-bytesCurBlock), writePacketSize);
+          computePacketChunkSize(psize, bytesPerChecksum);
+        }
+      }
+      //LOG.debug("DFSClient writeChunk done length " + len +
+      //          " checksum length " + cklen);
+    }
+  
+    /**
+     * All data is written out to datanodes. It is not guaranteed 
+     * that data has been flushed to persistent store on the 
+     * datanode. Block allocations are persisted on namenode.
+     */
+    public synchronized void sync() throws IOException {
+      try {
+        /* Record current blockOffset. This might be changed inside
+         * flushBuffer() where a partial checksum chunk might be flushed.
+         * After the flush, reset the bytesCurBlock back to its previous value,
+         * any partial checksum chunk will be sent now and in next packet.
+         */
+        long saveOffset = bytesCurBlock;
+
+        // flush checksum buffer, but keep checksum buffer intact
+        flushBuffer(true);
+
+        LOG.debug("DFSClient flush() : saveOffset " + saveOffset +  
+                  " bytesCurBlock " + bytesCurBlock +
+                  " lastFlushOffset " + lastFlushOffset);
+        
+        // Flush only if we haven't already flushed till this offset.
+        if (lastFlushOffset != bytesCurBlock) {
+
+          // record the valid offset of this flush
+          lastFlushOffset = bytesCurBlock;
+
+          // wait for all packets to be sent and acknowledged
+          flushInternal();
+        } else {
+          // just discard the current packet since it is already been sent.
+          currentPacket = null;
+        }
+        
+        // Restore state of stream. Record the last flush offset 
+        // of the last full chunk that was flushed.
+        //
+        bytesCurBlock = saveOffset;
+
+        // If any new blocks were allocated since the last flush, 
+        // then persist block locations on namenode. 
+        //
+        if (persistBlocks) {
+          namenode.fsync(src, clientName);
+          persistBlocks = false;
+        }
+      } catch (IOException e) {
+          lastException = new IOException("IOException flush:" + e);
+          closed = true;
+          closeThreads();
+          throw e;
+      }
+    }
+
+    /**
+     * Waits till all existing data is flushed and confirmations 
+     * received from datanodes. 
+     */
+    private synchronized void flushInternal() throws IOException {
+      checkOpen();
+      isClosed();
+
+      while (!closed) {
+        synchronized (dataQueue) {
+          isClosed();
+          //
+          // If there is data in the current buffer, send it across
+          //
+          if (currentPacket != null) {
+            dataQueue.addLast(currentPacket);
+            dataQueue.notifyAll();
+            currentPacket = null;
+          }
+
+          // wait for all buffers to be flushed to datanodes
+          if (!closed && dataQueue.size() != 0) {
+            try {
+              dataQueue.wait();
+            } catch (InterruptedException e) {
+            }
+            continue;
+          }
+        }
+
+        // wait for all acks to be received back from datanodes
+        synchronized (ackQueue) {
+          if (!closed && ackQueue.size() != 0) {
+            try {
+              ackQueue.wait();
+            } catch (InterruptedException e) {
+            }
+            continue;
+          }
+        }
+
+        // acquire both the locks and verify that we are
+        // *really done*. In the case of error recovery, 
+        // packets might move back from ackQueue to dataQueue.
+        //
+        synchronized (dataQueue) {
+          synchronized (ackQueue) {
+            if (dataQueue.size() + ackQueue.size() == 0) {
+              break;       // we are done
+            }
+          }
+        }
+      }
+    }
+  
+    /**
+     * Closes this output stream and releases any system 
+     * resources associated with this stream.
+     */
+    @Override
+    public void close() throws IOException {
+      if(closed)
+        return;
+      closeInternal();
+      leasechecker.remove(src);
+      
+      if (s != null) {
+        s.close();
+        s = null;
+      }
+    }
+ 
+    // shutdown datastreamer and responseprocessor threads.
+    private void closeThreads() throws IOException {
+      try {
+        streamer.close();
+        streamer.join();
+        
+        // shutdown response after streamer has exited.
+        if (response != null) {
+          response.close();
+          response.join();
+          response = null;
+        }
+      } catch (InterruptedException e) {
+        throw new IOException("Failed to shutdown response thread");
+      }
+    }
+    
+    /**
+     * Closes this output stream and releases any system 
+     * resources associated with this stream.
+     */
+    private synchronized void closeInternal() throws IOException {
+      checkOpen();
+      isClosed();
+
+      try {
+          flushBuffer();       // flush from all upper layers
+      
+          // Mark that this packet is the last packet in block.
+          // If there are no outstanding packets and the last packet
+          // was not the last one in the current block, then create a
+          // packet with empty payload.
+          synchronized (dataQueue) {
+            if (currentPacket == null && bytesCurBlock != 0) {
+              currentPacket = new Packet(packetSize, chunksPerPacket,
+                                         bytesCurBlock);
+            }
+            if (currentPacket != null) { 
+              currentPacket.lastPacketInBlock = true;
+            }
+          }
+
+        flushInternal();             // flush all data to Datanodes
+        isClosed(); // check to see if flushInternal had any exceptions
+        closed = true; // allow closeThreads() to showdown threads
+
+        closeThreads();
+        
+        synchronized (dataQueue) {
+          if (blockStream != null) {
+            blockStream.writeInt(0); // indicate end-of-block to datanode
+            blockStream.close();
+            blockReplyStream.close();
+          }
+          if (s != null) {
+            s.close();
+            s = null;
+          }
+        }
+
+        streamer = null;
+        blockStream = null;
+        blockReplyStream = null;
+
+        long localstart = System.currentTimeMillis();
+        boolean fileComplete = false;
+        while (!fileComplete) {
+          fileComplete = namenode.complete(src, clientName);
+          if (!fileComplete) {
+            try {
+              Thread.sleep(400);
+              if (System.currentTimeMillis() - localstart > 5000) {
+                LOG.info("Could not complete file " + src + " retrying...");
+              }
+            } catch (InterruptedException ie) {
+            }
+          }
+        }
+      } finally {
+        closed = true;
+      }
+    }
+
+    void setArtificialSlowdown(long period) {
+      artificialSlowdown = period;
+    }
+
+    synchronized void setChunksPerPacket(int value) {
+      chunksPerPacket = Math.min(chunksPerPacket, value);
+      packetSize = DataNode.PKT_HEADER_LEN + SIZE_OF_INTEGER +
+                   (checksum.getBytesPerChecksum() + 
+                    checksum.getChecksumSize()) * chunksPerPacket;
+    }
+
+    synchronized void setTestFilename(String newname) {
+      src = newname;
+    }
+
+    /**
+     * Returns the size of a file as it was when this stream was opened
+     */
+    long getInitialLen() {
+      return initialFileSize;
+    }
+  }
+
+  void reportChecksumFailure(String file, Block blk, DatanodeInfo dn) {
+    DatanodeInfo [] dnArr = { dn };
+    LocatedBlock [] lblocks = { new LocatedBlock(blk, dnArr) };
+    reportChecksumFailure(file, lblocks);
+  }
+    
+  // just reports checksum failure and ignores any exception during the report.
+  void reportChecksumFailure(String file, LocatedBlock lblocks[]) {
+    try {
+      reportBadBlocks(lblocks);
+    } catch (IOException ie) {
+      LOG.info("Found corruption while reading " + file 
+               + ".  Error repairing corrupt blocks.  Bad blocks remain. " 
+               + StringUtils.stringifyException(ie));
+    }
+  }
+
+  /** {@inheritDoc} */
+  public String toString() {
+    return getClass().getSimpleName() + "[clientName=" + clientName
+        + ", ugi=" + ugi + "]"; 
+  }
+}

+ 81 - 0
src/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.util.StringTokenizer;
+import org.apache.hadoop.fs.Path;
+
+public class DFSUtil {
+  /**
+   * Whether the pathname is valid.  Currently prohibits relative paths, 
+   * and names which contain a ":" or "/" 
+   */
+  public static boolean isValidName(String src) {
+      
+    // Path must be absolute.
+    if (!src.startsWith(Path.SEPARATOR)) {
+      return false;
+    }
+      
+    // Check for ".." "." ":" "/"
+    StringTokenizer tokens = new StringTokenizer(src, Path.SEPARATOR);
+    while(tokens.hasMoreTokens()) {
+      String element = tokens.nextToken();
+      if (element.equals("..") || 
+          element.equals(".")  ||
+          (element.indexOf(":") >= 0)  ||
+          (element.indexOf("/") >= 0)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Utility class to facilitate junit test error simulation.
+   */
+  public static class ErrorSimulator {
+    private static boolean[] simulation = null; // error simulation events
+    public static void initializeErrorSimulationEvent(int numberOfEvents) {
+      simulation = new boolean[numberOfEvents]; 
+      for (int i = 0; i < numberOfEvents; i++) {
+        simulation[i] = false;
+      }
+    }
+    
+    public static boolean getErrorSimulation(int index) {
+      if(simulation == null)
+        return false;
+      assert(index < simulation.length);
+      return simulation[index];
+    }
+    
+    public static void setErrorSimulation(int index) {
+      assert(index < simulation.length);
+      simulation[index] = true;
+    }
+    
+    public static void clearErrorSimulation(int index) {
+      assert(index < simulation.length);
+      simulation[index] = false;
+    }
+  }
+
+}
+

+ 522 - 0
src/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -0,0 +1,522 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.*;
+import java.net.*;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.DFSClient.DFSOutputStream;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.util.Progressable;
+
+
+/****************************************************************
+ * Implementation of the abstract FileSystem for the DFS system.
+ * This object is the way end-user code interacts with a Hadoop
+ * DistributedFileSystem.
+ *
+ *****************************************************************/
+public class DistributedFileSystem extends FileSystem {
+  private Path workingDir;
+  private URI uri;
+
+  DFSClient dfs;
+  private boolean verifyChecksum = true;
+  
+  static{
+    Configuration.addDefaultResource("hdfs-default.xml");
+    Configuration.addDefaultResource("hdfs-site.xml");
+  }
+
+  public DistributedFileSystem() {
+  }
+
+  @Deprecated
+  public DistributedFileSystem(InetSocketAddress namenode,
+    Configuration conf) throws IOException {
+    initialize(NameNode.getUri(namenode), conf);
+  }
+
+  @Override
+  public URI getUri() { return uri; }
+
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    setConf(conf);
+
+    String host = uri.getHost();
+    if (host == null) {
+      throw new IOException("Incomplete HDFS URI, no host: "+ uri);
+    }
+
+    InetSocketAddress namenode = NameNode.getAddress(uri.getAuthority());
+    this.dfs = new DFSClient(namenode, conf, statistics);
+    this.uri = URI.create("hdfs://" + uri.getAuthority());
+    this.workingDir = getHomeDirectory();
+  }
+
+  /** Permit paths which explicitly specify the default port. */
+  @Override
+  protected void checkPath(Path path) {
+    URI thisUri = this.getUri();
+    URI thatUri = path.toUri();
+    String thatAuthority = thatUri.getAuthority();
+    if (thatUri.getScheme() != null
+        && thatUri.getScheme().equalsIgnoreCase(thisUri.getScheme())
+        && thatUri.getPort() == NameNode.DEFAULT_PORT
+        && (thisUri.getPort() == -1 || 
+            thisUri.getPort() == NameNode.DEFAULT_PORT)
+        && thatAuthority.substring(0,thatAuthority.indexOf(":"))
+        .equalsIgnoreCase(thisUri.getAuthority()))
+      return;
+    super.checkPath(path);
+  }
+
+  /** Normalize paths that explicitly specify the default port. */
+  @Override
+  public Path makeQualified(Path path) {
+    URI thisUri = this.getUri();
+    URI thatUri = path.toUri();
+    String thatAuthority = thatUri.getAuthority();
+    if (thatUri.getScheme() != null
+        && thatUri.getScheme().equalsIgnoreCase(thisUri.getScheme())
+        && thatUri.getPort() == NameNode.DEFAULT_PORT
+        && thisUri.getPort() == -1
+        && thatAuthority.substring(0,thatAuthority.indexOf(":"))
+        .equalsIgnoreCase(thisUri.getAuthority())) {
+      path = new Path(thisUri.getScheme(), thisUri.getAuthority(),
+                      thatUri.getPath());
+    }
+    return super.makeQualified(path);
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return workingDir;
+  }
+
+  @Override
+  public long getDefaultBlockSize() {
+    return dfs.getDefaultBlockSize();
+  }
+
+  @Override
+  public short getDefaultReplication() {
+    return dfs.getDefaultReplication();
+  }
+
+  private Path makeAbsolute(Path f) {
+    if (f.isAbsolute()) {
+      return f;
+    } else {
+      return new Path(workingDir, f);
+    }
+  }
+
+  @Override
+  public void setWorkingDirectory(Path dir) {
+    String result = makeAbsolute(dir).toUri().getPath();
+    if (!DFSUtil.isValidName(result)) {
+      throw new IllegalArgumentException("Invalid DFS directory name " + 
+                                         result);
+    }
+    workingDir = makeAbsolute(dir);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public Path getHomeDirectory() {
+    return new Path("/user/" + dfs.ugi.getUserName()).makeQualified(this);
+  }
+
+  private String getPathName(Path file) {
+    checkPath(file);
+    String result = makeAbsolute(file).toUri().getPath();
+    if (!DFSUtil.isValidName(result)) {
+      throw new IllegalArgumentException("Pathname " + result + " from " +
+                                         file+" is not a valid DFS filename.");
+    }
+    return result;
+  }
+  
+  @Override
+  public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
+      long len) throws IOException {
+    if (file == null) {
+      return null;
+    }
+    return dfs.getBlockLocations(getPathName(file.getPath()), start, len);
+  }
+
+  @Override
+  public void setVerifyChecksum(boolean verifyChecksum) {
+    this.verifyChecksum = verifyChecksum;
+  }
+
+  @Override
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    return new DFSClient.DFSDataInputStream(
+          dfs.open(getPathName(f), bufferSize, verifyChecksum, statistics));
+  }
+
+  /** This optional operation is not yet supported. */
+  @Override
+  public FSDataOutputStream append(Path f, int bufferSize,
+      Progressable progress) throws IOException {
+
+    DFSOutputStream op = (DFSOutputStream)dfs.append(getPathName(f), bufferSize, progress);
+    return new FSDataOutputStream(op, statistics, op.getInitialLen());
+  }
+
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+    boolean overwrite,
+    int bufferSize, short replication, long blockSize,
+    Progressable progress) throws IOException {
+
+    return new FSDataOutputStream
+       (dfs.create(getPathName(f), permission,
+                   overwrite, replication, blockSize, progress, bufferSize),
+        statistics);
+  }
+
+  @Override
+  public boolean setReplication(Path src, 
+                                short replication
+                               ) throws IOException {
+    return dfs.setReplication(getPathName(src), replication);
+  }
+
+  /**
+   * Rename files/dirs
+   */
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+    return dfs.rename(getPathName(src), getPathName(dst));
+  }
+
+  /**
+   * requires a boolean check to delete a non 
+   * empty directory recursively.
+   */
+  @Override
+  public boolean delete(Path f, boolean recursive) throws IOException {
+   return dfs.delete(getPathName(f), recursive);
+  }
+  
+  /** {@inheritDoc} */
+  @Override
+  public ContentSummary getContentSummary(Path f) throws IOException {
+    return dfs.getContentSummary(getPathName(f));
+  }
+
+  /** Set a directory's quotas
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long) 
+   */
+  public void setQuota(Path src, long namespaceQuota, long diskspaceQuota) 
+                       throws IOException {
+    dfs.setQuota(getPathName(src), namespaceQuota, diskspaceQuota);
+  }
+  
+  private FileStatus makeQualified(FileStatus f) {
+    return new FileStatus(f.getLen(), f.isDir(), f.getReplication(),
+        f.getBlockSize(), f.getModificationTime(),
+        f.getAccessTime(),
+        f.getPermission(), f.getOwner(), f.getGroup(),
+        f.getPath().makeQualified(this)); // fully-qualify path
+  }
+
+  @Override
+  public FileStatus[] listStatus(Path p) throws IOException {
+    FileStatus[] infos = dfs.listPaths(getPathName(p));
+    if (infos == null) return null;
+    FileStatus[] stats = new FileStatus[infos.length];
+    for (int i = 0; i < infos.length; i++) {
+      stats[i] = makeQualified(infos[i]);
+    }
+    return stats;
+  }
+
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    return dfs.mkdirs(getPathName(f), permission);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void close() throws IOException {
+    try {
+      super.processDeleteOnExit();
+      dfs.close();
+    } finally {
+      super.close();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "DFS[" + dfs + "]";
+  }
+
+  public DFSClient getClient() {
+    return dfs;
+  }        
+  
+  /** @deprecated Use {@link org.apache.hadoop.fs.FsStatus} instead */
+  @Deprecated
+  public static class DiskStatus extends FsStatus {
+    public DiskStatus(FsStatus stats) {
+      super(stats.getCapacity(), stats.getUsed(), stats.getRemaining());
+    }
+
+    public DiskStatus(long capacity, long dfsUsed, long remaining) {
+      super(capacity, dfsUsed, remaining);
+    }
+
+    public long getDfsUsed() {
+      return super.getUsed();
+    }
+  }
+  
+  /** {@inheritDoc} */
+  @Override
+  public FsStatus getStatus(Path p) throws IOException {
+    return dfs.getDiskStatus();
+  }
+
+  /** Return the disk usage of the filesystem, including total capacity,
+   * used space, and remaining space 
+   * @deprecated Use {@link org.apache.hadoop.fs.FileSystem#getStatus()} 
+   * instead */
+   @Deprecated
+  public DiskStatus getDiskStatus() throws IOException {
+    return new DiskStatus(dfs.getDiskStatus());
+  }
+  
+  /** Return the total raw capacity of the filesystem, disregarding
+   * replication.
+   * @deprecated Use {@link org.apache.hadoop.fs.FileSystem#getStatus()} 
+   * instead */
+   @Deprecated
+  public long getRawCapacity() throws IOException{
+    return dfs.getDiskStatus().getCapacity();
+  }
+
+  /** Return the total raw used space in the filesystem, disregarding
+   * replication.
+   * @deprecated Use {@link org.apache.hadoop.fs.FileSystem#getStatus()} 
+   * instead */
+   @Deprecated
+  public long getRawUsed() throws IOException{
+    return dfs.getDiskStatus().getUsed();
+  }
+   
+  /**
+   * Returns count of blocks with no good replicas left. Normally should be
+   * zero.
+   * 
+   * @throws IOException
+   */
+  public long getMissingBlocksCount() throws IOException {
+    return dfs.getMissingBlocksCount();
+  }
+
+  /**
+   * Returns count of blocks with one of more replica missing.
+   * 
+   * @throws IOException
+   */
+  public long getUnderReplicatedBlocksCount() throws IOException {
+    return dfs.getUnderReplicatedBlocksCount();
+  }
+
+  /**
+   * Returns count of blocks with at least one replica marked corrupt.
+   * 
+   * @throws IOException
+   */
+  public long getCorruptBlocksCount() throws IOException {
+    return dfs.getCorruptBlocksCount();
+  }
+
+  /** Return statistics for each datanode. */
+  public DatanodeInfo[] getDataNodeStats() throws IOException {
+    return dfs.datanodeReport(DatanodeReportType.ALL);
+  }
+
+  /**
+   * Enter, leave or get safe mode.
+   *  
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(
+   *    FSConstants.SafeModeAction)
+   */
+  public boolean setSafeMode(FSConstants.SafeModeAction action) 
+  throws IOException {
+    return dfs.setSafeMode(action);
+  }
+
+  /**
+   * Save namespace image.
+   * 
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#saveNamespace()
+   */
+  public void saveNamespace() throws AccessControlException, IOException {
+    dfs.saveNamespace();
+  }
+
+  /**
+   * enable/disable/check restoreFaileStorage
+   * 
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String arg)
+   */
+  public boolean restoreFailedStorage(String arg) throws AccessControlException {
+    return dfs.restoreFailedStorage(arg);
+  }
+  
+
+  /**
+   * Refreshes the list of hosts and excluded hosts from the configured 
+   * files.  
+   */
+  public void refreshNodes() throws IOException {
+    dfs.refreshNodes();
+  }
+
+  /**
+   * Finalize previously upgraded files system state.
+   * @throws IOException
+   */
+  public void finalizeUpgrade() throws IOException {
+    dfs.finalizeUpgrade();
+  }
+
+  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action
+  ) throws IOException {
+    return dfs.distributedUpgradeProgress(action);
+  }
+
+  /*
+   * Requests the namenode to dump data strcutures into specified 
+   * file.
+   */
+  public void metaSave(String pathname) throws IOException {
+    dfs.metaSave(pathname);
+  }
+
+  /**
+   * We need to find the blocks that didn't match.  Likely only one 
+   * is corrupt but we will report both to the namenode.  In the future,
+   * we can consider figuring out exactly which block is corrupt.
+   */
+  public boolean reportChecksumFailure(Path f, 
+    FSDataInputStream in, long inPos, 
+    FSDataInputStream sums, long sumsPos) {
+
+    LocatedBlock lblocks[] = new LocatedBlock[2];
+
+    // Find block in data stream.
+    DFSClient.DFSDataInputStream dfsIn = (DFSClient.DFSDataInputStream) in;
+    Block dataBlock = dfsIn.getCurrentBlock();
+    if (dataBlock == null) {
+      LOG.error("Error: Current block in data stream is null! ");
+      return false;
+    }
+    DatanodeInfo[] dataNode = {dfsIn.getCurrentDatanode()}; 
+    lblocks[0] = new LocatedBlock(dataBlock, dataNode);
+    LOG.info("Found checksum error in data stream at block="
+        + dataBlock + " on datanode="
+        + dataNode[0].getName());
+
+    // Find block in checksum stream
+    DFSClient.DFSDataInputStream dfsSums = (DFSClient.DFSDataInputStream) sums;
+    Block sumsBlock = dfsSums.getCurrentBlock();
+    if (sumsBlock == null) {
+      LOG.error("Error: Current block in checksum stream is null! ");
+      return false;
+    }
+    DatanodeInfo[] sumsNode = {dfsSums.getCurrentDatanode()}; 
+    lblocks[1] = new LocatedBlock(sumsBlock, sumsNode);
+    LOG.info("Found checksum error in checksum stream at block="
+        + sumsBlock + " on datanode="
+        + sumsNode[0].getName());
+
+    // Ask client to delete blocks.
+    dfs.reportChecksumFailure(f.toString(), lblocks);
+
+    return true;
+  }
+
+  /**
+   * Returns the stat information about the file.
+   * @throws FileNotFoundException if the file does not exist.
+   */
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    FileStatus fi = dfs.getFileInfo(getPathName(f));
+    if (fi != null) {
+      return makeQualified(fi);
+    } else {
+      throw new FileNotFoundException("File does not exist: " + f);
+    }
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public MD5MD5CRC32FileChecksum getFileChecksum(Path f) throws IOException {
+    return dfs.getFileChecksum(getPathName(f));
+  }
+
+  /** {@inheritDoc }*/
+  @Override
+  public void setPermission(Path p, FsPermission permission
+      ) throws IOException {
+    dfs.setPermission(getPathName(p), permission);
+  }
+
+  /** {@inheritDoc }*/
+  @Override
+  public void setOwner(Path p, String username, String groupname
+      ) throws IOException {
+    if (username == null && groupname == null) {
+      throw new IOException("username == null && groupname == null");
+    }
+    dfs.setOwner(getPathName(p), username, groupname);
+  }
+
+  /** {@inheritDoc }*/
+  @Override
+  public void setTimes(Path p, long mtime, long atime
+      ) throws IOException {
+    dfs.setTimes(getPathName(p), mtime, atime);
+  }
+  
+  
+}

+ 50 - 0
src/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java

@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
+import org.apache.hadoop.security.authorize.Service;
+
+/**
+ * {@link PolicyProvider} for HDFS protocols.
+ */
+public class HDFSPolicyProvider extends PolicyProvider {
+  private static final Service[] hdfsServices =
+    new Service[] {
+    new Service("security.client.protocol.acl", ClientProtocol.class),
+    new Service("security.client.datanode.protocol.acl", 
+                ClientDatanodeProtocol.class),
+    new Service("security.datanode.protocol.acl", DatanodeProtocol.class),
+    new Service("security.inter.datanode.protocol.acl", 
+                InterDatanodeProtocol.class),
+    new Service("security.namenode.protocol.acl", NamenodeProtocol.class),
+    new Service("security.refresh.policy.protocol.acl", 
+                RefreshAuthorizationPolicyProtocol.class),
+  };
+  
+  @Override
+  public Service[] getServices() {
+    return hdfsServices;
+  }
+}

+ 322 - 0
src/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -0,0 +1,322 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Random;
+
+import javax.security.auth.login.LoginException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.namenode.ListPathsServlet;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.StringUtils;
+import org.xml.sax.Attributes;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+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.
+ * The following implementation provides a limited, read-only interface
+ * to a filesystem over HTTP.
+ * @see org.apache.hadoop.hdfs.server.namenode.ListPathsServlet
+ * @see org.apache.hadoop.hdfs.server.namenode.FileDataServlet
+ */
+public class HftpFileSystem extends FileSystem {
+  static {
+    HttpURLConnection.setFollowRedirects(true);
+  }
+
+  protected InetSocketAddress nnAddr;
+  protected UserGroupInformation ugi; 
+  protected final Random ran = new Random();
+
+  protected static final SimpleDateFormat df = ListPathsServlet.df;
+
+  @Override
+  public void initialize(URI name, Configuration conf) throws IOException {
+    super.initialize(name, conf);
+    setConf(conf);
+    try {
+      this.ugi = UnixUserGroupInformation.login(conf, true);
+    } catch (LoginException le) {
+      throw new IOException(StringUtils.stringifyException(le));
+    }
+
+    nnAddr = NetUtils.createSocketAddr(name.toString());
+  }
+  
+
+  @Override
+  public URI getUri() {
+    try {
+      return new URI("hftp", null, nnAddr.getHostName(), nnAddr.getPort(),
+                     null, null, null);
+    } catch (URISyntaxException e) {
+      return null;
+    } 
+  }
+
+  /**
+   * Open an HTTP connection to the namenode to read file data and metadata.
+   * @param path The path component of the URL
+   * @param query The query component of the URL
+   */
+  protected HttpURLConnection openConnection(String path, String query)
+      throws IOException {
+    try {
+      final URL url = new URI("http", null, nnAddr.getHostName(),
+          nnAddr.getPort(), path, query, null).toURL();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("url=" + url);
+      }
+      HttpURLConnection connection = (HttpURLConnection)url.openConnection();
+      connection.setRequestMethod("GET");
+      connection.connect();
+      return connection;
+    } catch (URISyntaxException e) {
+      throw (IOException)new IOException().initCause(e);
+    }
+  }
+
+  @Override
+  public FSDataInputStream open(Path f, int buffersize) throws IOException {
+    HttpURLConnection connection = null;
+    connection = openConnection("/data" + f.toUri().getPath(), "ugi=" + ugi);
+    final InputStream in = connection.getInputStream();
+    return new FSDataInputStream(new FSInputStream() {
+        public int read() throws IOException {
+          return in.read();
+        }
+        public int read(byte[] b, int off, int len) throws IOException {
+          return in.read(b, off, len);
+        }
+
+        public void close() throws IOException {
+          in.close();
+        }
+
+        public void seek(long pos) throws IOException {
+          throw new IOException("Can't seek!");
+        }
+        public long getPos() throws IOException {
+          throw new IOException("Position unknown!");
+        }
+        public boolean seekToNewSource(long targetPos) throws IOException {
+          return false;
+        }
+      });
+  }
+
+  /** Class to parse and store a listing reply from the server. */
+  class LsParser extends DefaultHandler {
+
+    ArrayList<FileStatus> fslist = new ArrayList<FileStatus>();
+
+    public void startElement(String ns, String localname, String qname,
+                Attributes attrs) throws SAXException {
+      if ("listing".equals(qname)) return;
+      if (!"file".equals(qname) && !"directory".equals(qname)) {
+        if (RemoteException.class.getSimpleName().equals(qname)) {
+          throw new SAXException(RemoteException.valueOf(attrs));
+        }
+        throw new SAXException("Unrecognized entry: " + qname);
+      }
+      long modif;
+      long atime = 0;
+      try {
+        modif = df.parse(attrs.getValue("modified")).getTime();
+        String astr = attrs.getValue("accesstime");
+        if (astr != null) {
+          atime = df.parse(astr).getTime();
+        }
+      } catch (ParseException e) { throw new SAXException(e); }
+      FileStatus fs = "file".equals(qname)
+        ? new FileStatus(
+              Long.valueOf(attrs.getValue("size")).longValue(), false,
+              Short.valueOf(attrs.getValue("replication")).shortValue(),
+              Long.valueOf(attrs.getValue("blocksize")).longValue(),
+              modif, atime, FsPermission.valueOf(attrs.getValue("permission")),
+              attrs.getValue("owner"), attrs.getValue("group"),
+              new Path(getUri().toString(), attrs.getValue("path"))
+                .makeQualified(HftpFileSystem.this))
+        : new FileStatus(0L, true, 0, 0L,
+              modif, atime, FsPermission.valueOf(attrs.getValue("permission")),
+              attrs.getValue("owner"), attrs.getValue("group"),
+              new Path(getUri().toString(), attrs.getValue("path"))
+                .makeQualified(HftpFileSystem.this));
+      fslist.add(fs);
+    }
+
+    private void fetchList(String path, boolean recur) throws IOException {
+      try {
+        XMLReader xr = XMLReaderFactory.createXMLReader();
+        xr.setContentHandler(this);
+        HttpURLConnection connection = openConnection("/listPaths" + path,
+            "ugi=" + ugi + (recur? "&recursive=yes" : ""));
+
+        InputStream resp = connection.getInputStream();
+        xr.parse(new InputSource(resp));
+      } catch(SAXException e) {
+        final Exception embedded = e.getException();
+        if (embedded != null && embedded instanceof IOException) {
+          throw (IOException)embedded;
+        }
+        throw new IOException("invalid xml directory content", e);
+      }
+    }
+
+    public FileStatus getFileStatus(Path f) throws IOException {
+      fetchList(f.toUri().getPath(), false);
+      if (fslist.size() == 0) {
+        throw new FileNotFoundException("File does not exist: " + f);
+      }
+      return fslist.get(0);
+    }
+
+    public FileStatus[] listStatus(Path f, boolean recur) throws IOException {
+      fetchList(f.toUri().getPath(), recur);
+      if (fslist.size() > 0 && (fslist.size() != 1 || fslist.get(0).isDir())) {
+        fslist.remove(0);
+      }
+      return fslist.toArray(new FileStatus[0]);
+    }
+
+    public FileStatus[] listStatus(Path f) throws IOException {
+      return listStatus(f, false);
+    }
+  }
+
+  @Override
+  public FileStatus[] listStatus(Path f) throws IOException {
+    LsParser lsparser = new LsParser();
+    return lsparser.listStatus(f);
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    LsParser lsparser = new LsParser();
+    return lsparser.getFileStatus(f);
+  }
+
+  private class ChecksumParser extends DefaultHandler {
+    private FileChecksum filechecksum;
+
+    /** {@inheritDoc} */
+    public void startElement(String ns, String localname, String qname,
+                Attributes attrs) throws SAXException {
+      if (!MD5MD5CRC32FileChecksum.class.getName().equals(qname)) {
+        if (RemoteException.class.getSimpleName().equals(qname)) {
+          throw new SAXException(RemoteException.valueOf(attrs));
+        }
+        throw new SAXException("Unrecognized entry: " + qname);
+      }
+
+      filechecksum = MD5MD5CRC32FileChecksum.valueOf(attrs);
+    }
+
+    private FileChecksum getFileChecksum(String f) throws IOException {
+      final HttpURLConnection connection = openConnection(
+          "/fileChecksum" + f, "ugi=" + ugi);
+      try {
+        final XMLReader xr = XMLReaderFactory.createXMLReader();
+        xr.setContentHandler(this);
+        xr.parse(new InputSource(connection.getInputStream()));
+      } catch(SAXException e) {
+        final Exception embedded = e.getException();
+        if (embedded != null && embedded instanceof IOException) {
+          throw (IOException)embedded;
+        }
+        throw new IOException("invalid xml directory content", e);
+      } finally {
+        connection.disconnect();
+      }
+      return filechecksum;
+    }
+  }
+
+  /** {@inheritDoc} */
+  public FileChecksum getFileChecksum(Path f) throws IOException {
+    final String s = makeQualified(f).toUri().getPath();
+    return new ChecksumParser().getFileChecksum(s);
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return new Path("/").makeQualified(this);
+  }
+
+  @Override
+  public void setWorkingDirectory(Path f) { }
+
+  /** This optional operation is not yet supported. */
+  public FSDataOutputStream append(Path f, int bufferSize,
+      Progressable progress) throws IOException {
+    throw new IOException("Not supported");
+  }
+
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+                                   boolean overwrite, int bufferSize,
+                                   short replication, long blockSize,
+                                   Progressable progress) throws IOException {
+    throw new IOException("Not supported");
+  }
+
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+    throw new IOException("Not supported");
+  }
+
+  @Override 
+  public boolean delete(Path f, boolean recursive) throws IOException {
+    throw new IOException("Not supported");
+  }
+  
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    throw new IOException("Not supported");
+  }
+
+}

+ 135 - 0
src/java/org/apache/hadoop/hdfs/HsftpFileSystem.java

@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.security.cert.X509Certificate;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSession;
+
+import org.apache.hadoop.conf.Configuration;
+
+
+
+/** An implementation of a protocol for accessing filesystems over HTTPS.
+ * The following implementation provides a limited, read-only interface
+ * to a filesystem over HTTPS.
+ * @see org.apache.hadoop.hdfs.server.namenode.ListPathsServlet
+ * @see org.apache.hadoop.hdfs.server.namenode.FileDataServlet
+ */
+public class HsftpFileSystem extends HftpFileSystem {
+
+  private static final long MM_SECONDS_PER_DAY = 1000 * 60 * 60 * 24;
+  private volatile int ExpWarnDays = 0;  
+  
+  
+  @Override
+  public void initialize(URI name, Configuration conf) throws IOException {
+    super.initialize(name, conf);
+    setupSsl(conf);
+    ExpWarnDays = conf.getInt("ssl.expiration.warn.days", 30);
+  }
+
+  /** Set up SSL resources */
+  private static void setupSsl(Configuration conf) {
+    Configuration sslConf = new Configuration(false);
+    sslConf.addResource(conf.get("dfs.https.client.keystore.resource",
+        "ssl-client.xml"));
+    System.setProperty("javax.net.ssl.trustStore", sslConf.get(
+        "ssl.client.truststore.location", ""));
+    System.setProperty("javax.net.ssl.trustStorePassword", sslConf.get(
+        "ssl.client.truststore.password", ""));
+    System.setProperty("javax.net.ssl.trustStoreType", sslConf.get(
+        "ssl.client.truststore.type", "jks"));
+    System.setProperty("javax.net.ssl.keyStore", sslConf.get(
+        "ssl.client.keystore.location", ""));
+    System.setProperty("javax.net.ssl.keyStorePassword", sslConf.get(
+        "ssl.client.keystore.password", ""));
+    System.setProperty("javax.net.ssl.keyPassword", sslConf.get(
+        "ssl.client.keystore.keypassword", ""));
+    System.setProperty("javax.net.ssl.keyStoreType", sslConf.get(
+        "ssl.client.keystore.type", "jks"));
+  }
+  
+  @Override
+  protected HttpURLConnection openConnection(String path, String query)
+      throws IOException {
+    try {
+      final URL url = new URI("https", null, nnAddr.getHostName(),
+          nnAddr.getPort(), path, query, null).toURL();
+      HttpsURLConnection conn = (HttpsURLConnection)url.openConnection();
+      // bypass hostname verification
+      conn.setHostnameVerifier(new DummyHostnameVerifier());
+      conn.setRequestMethod("GET");
+      conn.connect();
+     
+   // check cert expiration date
+      final int warnDays = ExpWarnDays;
+      if (warnDays > 0) { // make sure only check once
+        ExpWarnDays = 0;
+        long expTimeThreshold = warnDays * MM_SECONDS_PER_DAY
+            + System.currentTimeMillis();
+        X509Certificate[] clientCerts = (X509Certificate[]) conn
+            .getLocalCertificates();
+        if (clientCerts != null) {
+          for (X509Certificate cert : clientCerts) {
+            long expTime = cert.getNotAfter().getTime();
+            if (expTime < expTimeThreshold) {
+              StringBuffer sb = new StringBuffer();
+              sb.append("\n Client certificate "
+                  + cert.getSubjectX500Principal().getName());
+              int dayOffSet = (int) ((expTime - System.currentTimeMillis())/MM_SECONDS_PER_DAY);
+              sb.append(" have " + dayOffSet + " days to expire");
+              LOG.warn(sb.toString());
+            }
+          }
+        }        
+      }
+      return (HttpURLConnection)conn;
+    } catch (URISyntaxException e) {
+      throw (IOException)new IOException().initCause(e);
+    }
+  }
+
+  @Override
+  public URI getUri() {
+    try {
+      return new URI("hsftp", null, nnAddr.getHostName(), nnAddr.getPort(),
+                     null, null, null);
+    } catch (URISyntaxException e) {
+      return null;
+    } 
+  }
+
+  /**
+   * Dummy hostname verifier that is used to bypass hostname checking
+   */
+  protected static class DummyHostnameVerifier implements HostnameVerifier {
+    public boolean verify(String hostname, SSLSession session) {
+      return true;
+    }
+  }
+
+}

+ 34 - 0
src/java/org/apache/hadoop/hdfs/package.html

@@ -0,0 +1,34 @@
+<html>
+
+<!--
+   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.
+-->
+
+<body>
+
+<p>A distributed implementation of {@link
+org.apache.hadoop.fs.FileSystem}.  This is loosely modelled after
+Google's <a href="http://labs.google.com/papers/gfs.html">GFS</a>.</p>
+
+<p>The most important difference is that unlike GFS, Hadoop DFS files 
+have strictly one writer at any one time.  Bytes are always appended 
+to the end of the writer's stream.  There is no notion of "record appends"
+or "mutations" that are then checked or reordered.  Writers simply emit 
+a byte stream.  That byte stream is guaranteed to be stored in the 
+order written.</p>
+
+</body>
+</html>

+ 32 - 0
src/java/org/apache/hadoop/hdfs/protocol/AlreadyBeingCreatedException.java

@@ -0,0 +1,32 @@
+/**
+ * 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.protocol;
+
+import java.io.IOException;
+
+/**
+ * The exception that happens when you ask to create a file that already
+ * is being created, but is not closed yet.
+ */
+public class AlreadyBeingCreatedException extends IOException {
+  static final long serialVersionUID = 0x12308AD009L;
+  public AlreadyBeingCreatedException(String msg) {
+    super(msg);
+  }
+}

+ 212 - 0
src/java/org/apache/hadoop/hdfs/protocol/Block.java

@@ -0,0 +1,212 @@
+/**
+ * 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.protocol;
+
+import java.io.*;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.io.*;
+
+/**************************************************
+ * A Block is a Hadoop FS primitive, identified by a 
+ * long.
+ *
+ **************************************************/
+public class Block implements Writable, Comparable<Block> {
+  public static final String BLOCK_FILE_PREFIX = "blk_";
+  public static final String METADATA_EXTENSION = ".meta";
+  static {                                      // register a ctor
+    WritableFactories.setFactory
+      (Block.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new Block(); }
+       });
+  }
+
+  // generation stamp of blocks that pre-date the introduction of
+  // a generation stamp.
+  public static final long GRANDFATHER_GENERATION_STAMP = 0;
+
+  public static final Pattern blockFilePattern = Pattern
+      .compile(BLOCK_FILE_PREFIX + "(-??\\d++)$");
+  public static final Pattern metaFilePattern = Pattern
+      .compile(BLOCK_FILE_PREFIX + "(-??\\d++)_(\\d++)\\" + METADATA_EXTENSION
+          + "$");
+
+  public static boolean isBlockFilename(File f) {
+    String name = f.getName();
+    return blockFilePattern.matcher(name).matches();
+  }
+
+  public static long filename2id(String name) {
+    Matcher m = blockFilePattern.matcher(name);
+    return m.matches() ? Long.parseLong(m.group(1)) : 0;
+  }
+
+  public static boolean isMetaFilename(String name) {
+    return metaFilePattern.matcher(name).matches();
+  }
+
+  /**
+   * Get generation stamp from the name of the metafile name
+   */
+  public static long getGenerationStamp(String metaFile) {
+    Matcher m = metaFilePattern.matcher(metaFile);
+    return m.matches() ? Long.parseLong(m.group(2))
+        : GRANDFATHER_GENERATION_STAMP;
+  }
+
+  /**
+   * Get the blockId from the name of the metafile name
+   */
+  public static long getBlockId(String metaFile) {
+    Matcher m = metaFilePattern.matcher(metaFile);
+    return m.matches() ? Long.parseLong(m.group(1)) : 0;
+  }
+
+  private long blockId;
+  private long numBytes;
+  private long generationStamp;
+
+  public Block() {this(0, 0, 0);}
+
+  public Block(final long blkid, final long len, final long generationStamp) {
+    set(blkid, len, generationStamp);
+  }
+
+  public Block(final long blkid) {this(blkid, 0, GenerationStamp.WILDCARD_STAMP);}
+
+  public Block(Block blk) {this(blk.blockId, blk.numBytes, blk.generationStamp);}
+
+  /**
+   * Find the blockid from the given filename
+   */
+  public Block(File f, long len, long genstamp) {
+    this(filename2id(f.getName()), len, genstamp);
+  }
+
+  public void set(long blkid, long len, long genStamp) {
+    this.blockId = blkid;
+    this.numBytes = len;
+    this.generationStamp = genStamp;
+  }
+  /**
+   */
+  public long getBlockId() {
+    return blockId;
+  }
+  
+  public void setBlockId(long bid) {
+    blockId = bid;
+  }
+
+  /**
+   */
+  public String getBlockName() {
+    return BLOCK_FILE_PREFIX + String.valueOf(blockId);
+  }
+
+  /**
+   */
+  public long getNumBytes() {
+    return numBytes;
+  }
+  public void setNumBytes(long len) {
+    this.numBytes = len;
+  }
+
+  public long getGenerationStamp() {
+    return generationStamp;
+  }
+  
+  public void setGenerationStamp(long stamp) {
+    generationStamp = stamp;
+  }
+
+  /**
+   */
+  public String toString() {
+    return getBlockName() + "_" + getGenerationStamp();
+  }
+
+  /////////////////////////////////////
+  // Writable
+  /////////////////////////////////////
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(blockId);
+    out.writeLong(numBytes);
+    out.writeLong(generationStamp);
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    this.blockId = in.readLong();
+    this.numBytes = in.readLong();
+    this.generationStamp = in.readLong();
+    if (numBytes < 0) {
+      throw new IOException("Unexpected block size: " + numBytes);
+    }
+  }
+
+  /////////////////////////////////////
+  // Comparable
+  /////////////////////////////////////
+  static void validateGenerationStamp(long generationstamp) {
+    if (generationstamp == GenerationStamp.WILDCARD_STAMP) {
+      throw new IllegalStateException("generationStamp (=" + generationstamp
+          + ") == GenerationStamp.WILDCARD_STAMP");
+    }    
+  }
+
+  /** {@inheritDoc} */
+  public int compareTo(Block b) {
+    //Wildcard generationStamp is NOT ALLOWED here
+    validateGenerationStamp(this.generationStamp);
+    validateGenerationStamp(b.generationStamp);
+
+    if (blockId < b.blockId) {
+      return -1;
+    } else if (blockId == b.blockId) {
+      return GenerationStamp.compare(generationStamp, b.generationStamp);
+    } else {
+      return 1;
+    }
+  }
+
+  /** {@inheritDoc} */
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof Block)) {
+      return false;
+    }
+    final Block that = (Block)o;
+    //Wildcard generationStamp is ALLOWED here
+    return this.blockId == that.blockId
+      && GenerationStamp.equalsWithWildcard(
+          this.generationStamp, that.generationStamp);
+  }
+
+  /** {@inheritDoc} */
+  public int hashCode() {
+    //GenerationStamp is IRRELEVANT and should not be used here
+    return 37 * 17 + (int) (blockId^(blockId>>>32));
+  }
+}

+ 127 - 0
src/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java

@@ -0,0 +1,127 @@
+/**
+ * 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.protocol;
+
+/**
+ * This class provides an interface for accessing list of blocks that
+ * has been implemented as long[].
+ * This class is usefull for block report. Rather than send block reports
+ * as a Block[] we can send it as a long[].
+ *
+ */
+public class BlockListAsLongs {
+  /**
+   * A block as 3 longs
+   *   block-id and block length and generation stamp
+   */
+  private static final int LONGS_PER_BLOCK = 3;
+  
+  private static int index2BlockId(int index) {
+    return index*LONGS_PER_BLOCK;
+  }
+  private static int index2BlockLen(int index) {
+    return (index*LONGS_PER_BLOCK) + 1;
+  }
+  private static int index2BlockGenStamp(int index) {
+    return (index*LONGS_PER_BLOCK) + 2;
+  }
+  
+  private long[] blockList;
+  
+  /**
+   * Converting a block[] to a long[]
+   * @param blockArray - the input array block[]
+   * @return the output array of long[]
+   */
+  
+  public static long[] convertToArrayLongs(final Block[] blockArray) {
+    long[] blocksAsLongs = new long[blockArray.length * LONGS_PER_BLOCK];
+
+    BlockListAsLongs bl = new BlockListAsLongs(blocksAsLongs);
+    assert bl.getNumberOfBlocks() == blockArray.length;
+
+    for (int i = 0; i < blockArray.length; i++) {
+      bl.setBlock(i, blockArray[i]);
+    }
+    return blocksAsLongs;
+  }
+
+  /**
+   * Constructor
+   * @param iBlockList - BlockListALongs create from this long[] parameter
+   */
+  public BlockListAsLongs(final long[] iBlockList) {
+    if (iBlockList == null) {
+      blockList = new long[0];
+    } else {
+      if (iBlockList.length%LONGS_PER_BLOCK != 0) {
+        // must be multiple of LONGS_PER_BLOCK
+        throw new IllegalArgumentException();
+      }
+      blockList = iBlockList;
+    }
+  }
+
+  
+  /**
+   * The number of blocks
+   * @return - the number of blocks
+   */
+  public int getNumberOfBlocks() {
+    return blockList.length/LONGS_PER_BLOCK;
+  }
+  
+  
+  /**
+   * The block-id of the indexTh block
+   * @param index - the block whose block-id is desired
+   * @return the block-id
+   */
+  public long getBlockId(final int index)  {
+    return blockList[index2BlockId(index)];
+  }
+  
+  /**
+   * The block-len of the indexTh block
+   * @param index - the block whose block-len is desired
+   * @return - the block-len
+   */
+  public long getBlockLen(final int index)  {
+    return blockList[index2BlockLen(index)];
+  }
+
+  /**
+   * The generation stamp of the indexTh block
+   * @param index - the block whose block-len is desired
+   * @return - the generation stamp
+   */
+  public long getBlockGenStamp(final int index)  {
+    return blockList[index2BlockGenStamp(index)];
+  }
+  
+  /**
+   * Set the indexTh block
+   * @param index - the index of the block to set
+   * @param b - the block is set to the value of the this block
+   */
+  void setBlock(final int index, final Block b) {
+    blockList[index2BlockId(index)] = b.getBlockId();
+    blockList[index2BlockLen(index)] = b.getNumBytes();
+    blockList[index2BlockGenStamp(index)] = b.getGenerationStamp();
+  }
+}

+ 47 - 0
src/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java

@@ -0,0 +1,47 @@
+/**
+ * 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.protocol;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.ipc.VersionedProtocol;
+
+/** An client-datanode protocol for block recovery
+ */
+public interface ClientDatanodeProtocol extends VersionedProtocol {
+  public static final Log LOG = LogFactory.getLog(ClientDatanodeProtocol.class);
+
+  /**
+   * 3: add keepLength parameter.
+   */
+  public static final long versionID = 3L;
+
+  /** Start generation-stamp recovery for specified block
+   * @param block the specified block
+   * @param keepLength keep the block length
+   * @param targets the list of possible locations of specified block
+   * @return the new blockid if recovery successful and the generation stamp
+   * got updated as part of the recovery, else returns null if the block id
+   * not have any data and the block was deleted.
+   * @throws IOException
+   */
+  LocatedBlock recoverBlock(Block block, boolean keepLength,
+      DatanodeInfo[] targets) throws IOException;
+}

+ 489 - 0
src/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -0,0 +1,489 @@
+/**
+ * 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.protocol;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.hadoop.security.AccessControlException;
+
+/**********************************************************************
+ * ClientProtocol is used by user code via 
+ * {@link org.apache.hadoop.hdfs.DistributedFileSystem} class to communicate 
+ * with the NameNode.  User code can manipulate the directory namespace, 
+ * as well as open/close file streams, etc.
+ *
+ **********************************************************************/
+public interface ClientProtocol extends VersionedProtocol {
+
+  /**
+   * Compared to the previous version the following changes have been introduced:
+   * (Only the latest change is reflected.
+   * The log of historical changes can be retrieved from the svn).
+   * 44: All LocatedBlock objects contain access tokens
+   */
+  public static final long versionID = 44L;
+  
+  ///////////////////////////////////////
+  // File contents
+  ///////////////////////////////////////
+  /**
+   * Get locations of the blocks of the specified file within the specified range.
+   * DataNode locations for each block are sorted by
+   * the proximity to the client.
+   * <p>
+   * Return {@link LocatedBlocks} which contains
+   * file length, blocks and their locations.
+   * DataNode locations for each block are sorted by
+   * the distance to the client's address.
+   * <p>
+   * The client will then have to contact 
+   * one of the indicated DataNodes to obtain the actual data.
+   * 
+   * @param src file name
+   * @param offset range start offset
+   * @param length range length
+   * @return file length and array of blocks with their locations
+   * @throws IOException
+   */
+  public LocatedBlocks  getBlockLocations(String src,
+                                          long offset,
+                                          long length) throws IOException;
+
+  /**
+   * Create a new file entry in the namespace.
+   * <p>
+   * This will create an empty file specified by the source path.
+   * The path should reflect a full path originated at the root.
+   * The name-node does not have a notion of "current" directory for a client.
+   * <p>
+   * Once created, the file is visible and available for read to other clients.
+   * Although, other clients cannot {@link #delete(String, boolean)}, re-create or 
+   * {@link #rename(String, String)} it until the file is completed
+   * or explicitly as a result of lease expiration.
+   * <p>
+   * Blocks have a maximum size.  Clients that intend to
+   * create multi-block files must also use {@link #addBlock(String, String)}.
+   *
+   * @param src path of the file being created.
+   * @param masked masked permission.
+   * @param clientName name of the current client.
+   * @param overwrite indicates whether the file should be 
+   * overwritten if it already exists.
+   * @param replication block replication factor.
+   * @param blockSize maximum block size.
+   * 
+   * @throws AccessControlException if permission to create file is 
+   * denied by the system. As usually on the client side the exception will 
+   * be wrapped into {@link org.apache.hadoop.ipc.RemoteException}.
+   * @throws QuotaExceededException if the file creation violates 
+   *                                any quota restriction
+   * @throws IOException if other errors occur.
+   */
+  public void create(String src, 
+                     FsPermission masked,
+                             String clientName, 
+                             boolean overwrite, 
+                             short replication,
+                             long blockSize
+                             ) throws IOException;
+
+  /**
+   * Append to the end of the file. 
+   * @param src path of the file being created.
+   * @param clientName name of the current client.
+   * @return information about the last partial block if any.
+   * @throws AccessControlException if permission to append file is 
+   * denied by the system. As usually on the client side the exception will 
+   * be wrapped into {@link org.apache.hadoop.ipc.RemoteException}.
+   * Allows appending to an existing file if the server is
+   * configured with the parameter dfs.support.append set to true, otherwise
+   * throws an IOException.
+   * @throws IOException if other errors occur.
+   */
+  public LocatedBlock append(String src, String clientName) throws IOException;
+
+  /**
+   * Set replication for an existing file.
+   * <p>
+   * The NameNode sets replication to the new value and returns.
+   * The actual block replication is not expected to be performed during  
+   * this method call. The blocks will be populated or removed in the 
+   * background as the result of the routine block maintenance procedures.
+   * 
+   * @param src file name
+   * @param replication new replication
+   * @throws IOException
+   * @return true if successful;
+   *         false if file does not exist or is a directory
+   */
+  public boolean setReplication(String src, 
+                                short replication
+                                ) throws IOException;
+
+  /**
+   * Set permissions for an existing file/directory.
+   */
+  public void setPermission(String src, FsPermission permission
+      ) throws IOException;
+
+  /**
+   * Set owner of a path (i.e. a file or a directory).
+   * The parameters username and groupname cannot both be null.
+   * @param src
+   * @param username If it is null, the original username remains unchanged.
+   * @param groupname If it is null, the original groupname remains unchanged.
+   */
+  public void setOwner(String src, String username, String groupname
+      ) throws IOException;
+
+  /**
+   * The client can give up on a blcok by calling abandonBlock().
+   * The client can then
+   * either obtain a new block, or complete or abandon the file.
+   * Any partial writes to the block will be discarded.
+   */
+  public void abandonBlock(Block b, String src, String holder
+      ) throws IOException;
+
+  /**
+   * A client that wants to write an additional block to the 
+   * indicated filename (which must currently be open for writing)
+   * should call addBlock().  
+   *
+   * addBlock() allocates a new block and datanodes the block data
+   * should be replicated to.
+   * 
+   * @return LocatedBlock allocated block information.
+   */
+  public LocatedBlock addBlock(String src, String clientName) throws IOException;
+
+  /**
+   * The client is done writing data to the given filename, and would 
+   * like to complete it.  
+   *
+   * The function returns whether the file has been closed successfully.
+   * If the function returns false, the caller should try again.
+   *
+   * A call to complete() will not return true until all the file's
+   * blocks have been replicated the minimum number of times.  Thus,
+   * DataNode failures may cause a client to call complete() several
+   * times before succeeding.
+   */
+  public boolean complete(String src, String clientName) throws IOException;
+
+  /**
+   * The client wants to report corrupted blocks (blocks with specified
+   * locations on datanodes).
+   * @param blocks Array of located blocks to report
+   */
+  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
+
+  ///////////////////////////////////////
+  // Namespace management
+  ///////////////////////////////////////
+  /**
+   * Rename an item in the file system namespace.
+   * 
+   * @param src existing file or directory name.
+   * @param dst new name.
+   * @return true if successful, or false if the old name does not exist
+   * or if the new name already belongs to the namespace.
+   * @throws IOException if the new name is invalid.
+   * @throws QuotaExceededException if the rename would violate 
+   *                                any quota restriction
+   */
+  public boolean rename(String src, String dst) throws IOException;
+
+  /**
+   * Delete the given file or directory from the file system.
+   * <p>
+   * Any blocks belonging to the deleted files will be garbage-collected.
+   * 
+   * @param src existing name.
+   * @return true only if the existing file or directory was actually removed 
+   * from the file system. 
+   * @deprecated use {@link #delete(String, boolean)} istead.
+   */
+  @Deprecated
+  public boolean delete(String src) throws IOException;
+
+  /**
+   * Delete the given file or directory from the file system.
+   * <p>
+   * same as delete but provides a way to avoid accidentally 
+   * deleting non empty directories programmatically. 
+   * @param src existing name
+   * @param recursive if true deletes a non empty directory recursively,
+   * else throws an exception.
+   * @return true only if the existing file or directory was actually removed 
+   * from the file system. 
+   */
+  public boolean delete(String src, boolean recursive) throws IOException;
+  
+  /**
+   * Create a directory (or hierarchy of directories) with the given
+   * name and permission.
+   *
+   * @param src The path of the directory being created
+   * @param masked The masked permission of the directory being created
+   * @return True if the operation success.
+   * @throws {@link AccessControlException} if permission to create file is 
+   * denied by the system. As usually on the client side the exception will 
+   * be wraped into {@link org.apache.hadoop.ipc.RemoteException}.
+   * @throws QuotaExceededException if the operation would violate 
+   *                                any quota restriction.
+   */
+  public boolean mkdirs(String src, FsPermission masked) throws IOException;
+
+  /**
+   * Get a listing of the indicated directory
+   */
+  public FileStatus[] getListing(String src) throws IOException;
+
+  ///////////////////////////////////////
+  // System issues and management
+  ///////////////////////////////////////
+
+  /**
+   * Client programs can cause stateful changes in the NameNode
+   * that affect other clients.  A client may obtain a file and 
+   * neither abandon nor complete it.  A client might hold a series
+   * of locks that prevent other clients from proceeding.
+   * Clearly, it would be bad if a client held a bunch of locks
+   * that it never gave up.  This can happen easily if the client
+   * dies unexpectedly.
+   * <p>
+   * So, the NameNode will revoke the locks and live file-creates
+   * for clients that it thinks have died.  A client tells the
+   * NameNode that it is still alive by periodically calling
+   * renewLease().  If a certain amount of time passes since
+   * the last call to renewLease(), the NameNode assumes the
+   * client has died.
+   */
+  public void renewLease(String clientName) throws IOException;
+
+  public int GET_STATS_CAPACITY_IDX = 0;
+  public int GET_STATS_USED_IDX = 1;
+  public int GET_STATS_REMAINING_IDX = 2;
+  public int GET_STATS_UNDER_REPLICATED_IDX = 3;
+  public int GET_STATS_CORRUPT_BLOCKS_IDX = 4;
+  public int GET_STATS_MISSING_BLOCKS_IDX = 5;
+  
+  /**
+   * Get a set of statistics about the filesystem.
+   * Right now, only three values are returned.
+   * <ul>
+   * <li> [0] contains the total storage capacity of the system, in bytes.</li>
+   * <li> [1] contains the total used space of the system, in bytes.</li>
+   * <li> [2] contains the available storage of the system, in bytes.</li>
+   * <li> [3] contains number of under replicated blocks in the system.</li>
+   * <li> [4] contains number of blocks with a corrupt replica. </li>
+   * <li> [5] contains number of blocks without any good replicas left. </li>
+   * </ul>
+   * Use public constants like {@link #GET_STATS_CAPACITY_IDX} in place of 
+   * actual numbers to index into the array.
+   */
+  public long[] getStats() throws IOException;
+
+  /**
+   * Get a report on the system's current datanodes.
+   * One DatanodeInfo object is returned for each DataNode.
+   * Return live datanodes if type is LIVE; dead datanodes if type is DEAD;
+   * otherwise all datanodes if type is ALL.
+   */
+  public DatanodeInfo[] getDatanodeReport(FSConstants.DatanodeReportType type)
+  throws IOException;
+
+  /**
+   * Get the block size for the given file.
+   * @param filename The name of the file
+   * @return The number of bytes in each block
+   * @throws IOException
+   */
+  public long getPreferredBlockSize(String filename) throws IOException;
+
+  /**
+   * Enter, leave or get safe mode.
+   * <p>
+   * Safe mode is a name node state when it
+   * <ol><li>does not accept changes to name space (read-only), and</li>
+   * <li>does not replicate or delete blocks.</li></ol>
+   * 
+   * <p>
+   * Safe mode is entered automatically at name node startup.
+   * Safe mode can also be entered manually using
+   * {@link #setSafeMode(FSConstants.SafeModeAction) setSafeMode(SafeModeAction.SAFEMODE_GET)}.
+   * <p>
+   * At startup the name node accepts data node reports collecting
+   * information about block locations.
+   * In order to leave safe mode it needs to collect a configurable
+   * percentage called threshold of blocks, which satisfy the minimal 
+   * replication condition.
+   * The minimal replication condition is that each block must have at least
+   * <tt>dfs.replication.min</tt> replicas.
+   * When the threshold is reached the name node extends safe mode
+   * for a configurable amount of time
+   * to let the remaining data nodes to check in before it
+   * will start replicating missing blocks.
+   * Then the name node leaves safe mode.
+   * <p>
+   * If safe mode is turned on manually using
+   * {@link #setSafeMode(FSConstants.SafeModeAction) setSafeMode(SafeModeAction.SAFEMODE_ENTER)}
+   * then the name node stays in safe mode until it is manually turned off
+   * using {@link #setSafeMode(FSConstants.SafeModeAction) setSafeMode(SafeModeAction.SAFEMODE_LEAVE)}.
+   * Current state of the name node can be verified using
+   * {@link #setSafeMode(FSConstants.SafeModeAction) setSafeMode(SafeModeAction.SAFEMODE_GET)}
+   * <h4>Configuration parameters:</h4>
+   * <tt>dfs.safemode.threshold.pct</tt> is the threshold parameter.<br>
+   * <tt>dfs.safemode.extension</tt> is the safe mode extension parameter.<br>
+   * <tt>dfs.replication.min</tt> is the minimal replication parameter.
+   * 
+   * <h4>Special cases:</h4>
+   * The name node does not enter safe mode at startup if the threshold is 
+   * set to 0 or if the name space is empty.<br>
+   * If the threshold is set to 1 then all blocks need to have at least 
+   * minimal replication.<br>
+   * If the threshold value is greater than 1 then the name node will not be 
+   * able to turn off safe mode automatically.<br>
+   * Safe mode can always be turned off manually.
+   * 
+   * @param action  <ul> <li>0 leave safe mode;</li>
+   *                <li>1 enter safe mode;</li>
+   *                <li>2 get safe mode state.</li></ul>
+   * @return <ul><li>0 if the safe mode is OFF or</li> 
+   *         <li>1 if the safe mode is ON.</li></ul>
+   * @throws IOException
+   */
+  public boolean setSafeMode(FSConstants.SafeModeAction action) throws IOException;
+
+  /**
+   * Save namespace image.
+   * <p>
+   * Saves current namespace into storage directories and reset edits log.
+   * Requires superuser privilege and safe mode.
+   * 
+   * @throws AccessControlException if the superuser privilege is violated.
+   * @throws IOException if image creation failed.
+   */
+  public void saveNamespace() throws IOException;
+
+  /**
+   * Enable/Disable restore failed storage.
+   * <p>
+   * sets flag to enable restore of failed storage replicas
+   * 
+   * @throws AccessControlException if the superuser privilege is violated.
+   */
+  public boolean restoreFailedStorage(String arg) throws AccessControlException;
+
+  /**
+   * Tells the namenode to reread the hosts and exclude files. 
+   * @throws IOException
+   */
+  public void refreshNodes() throws IOException;
+
+  /**
+   * Finalize previous upgrade.
+   * Remove file system state saved during the upgrade.
+   * The upgrade will become irreversible.
+   * 
+   * @throws IOException
+   */
+  public void finalizeUpgrade() throws IOException;
+
+  /**
+   * Report distributed upgrade progress or force current upgrade to proceed.
+   * 
+   * @param action {@link FSConstants.UpgradeAction} to perform
+   * @return upgrade status information or null if no upgrades are in progress
+   * @throws IOException
+   */
+  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action) 
+  throws IOException;
+
+  /**
+   * Dumps namenode data structures into specified file. If file
+   * already exists, then append.
+   * @throws IOException
+   */
+  public void metaSave(String filename) throws IOException;
+
+  /**
+   * Get the file info for a specific file or directory.
+   * @param src The string representation of the path to the file
+   * @throws IOException if permission to access file is denied by the system 
+   * @return object containing information regarding the file
+   *         or null if file not found
+   */
+  public FileStatus getFileInfo(String src) throws IOException;
+
+  /**
+   * Get {@link ContentSummary} rooted at the specified directory.
+   * @param path The string representation of the path
+   */
+  public ContentSummary getContentSummary(String path) throws IOException;
+
+  /**
+   * Set the quota for a directory.
+   * @param path  The string representation of the path to the directory
+   * @param namespaceQuota Limit on the number of names in the tree rooted 
+   *                       at the directory
+   * @param diskspaceQuota Limit on disk space occupied all the files under
+   *                       this directory. 
+   * <br><br>
+   *                       
+   * The quota can have three types of values : (1) 0 or more will set 
+   * the quota to that value, (2) {@link FSConstants#QUOTA_DONT_SET}  implies 
+   * the quota will not be changed, and (3) {@link FSConstants#QUOTA_RESET} 
+   * implies the quota will be reset. Any other value is a runtime error.
+   *                        
+   * @throws FileNotFoundException if the path is a file or 
+   *                               does not exist 
+   * @throws QuotaExceededException if the directory size 
+   *                                is greater than the given quota
+   */
+  public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
+                      throws IOException;
+  
+  /**
+   * Write all metadata for this file into persistent storage.
+   * The file must be currently open for writing.
+   * @param src The string representation of the path
+   * @param client The string representation of the client
+   */
+  public void fsync(String src, String client) throws IOException;
+
+  /**
+   * Sets the modification and access time of the file to the specified time.
+   * @param src The string representation of the path
+   * @param mtime The number of milliseconds since Jan 1, 1970.
+   *              Setting mtime to -1 means that modification time should not be set
+   *              by this call.
+   * @param atime The number of milliseconds since Jan 1, 1970.
+   *              Setting atime to -1 means that access time should not be set
+   *              by this call.
+   */
+  public void setTimes(String src, long mtime, long atime) throws IOException;
+}

+ 61 - 0
src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java

@@ -0,0 +1,61 @@
+/**
+ * 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.protocol;
+
+
+/**
+ * 
+ * The Client transfers data to/from datanode using a streaming protocol.
+ *
+ */
+public interface DataTransferProtocol {
+  
+  
+  /** Version for data transfers between clients and datanodes
+   * This should change when serialization of DatanodeInfo, not just
+   * when protocol changes. It is not very obvious. 
+   */
+  /*
+   * Version 15:
+   *    Added a new status OP_STATUS_ERROR_ACCESS_TOKEN
+   *    Access token is now required on all DN operations
+   */
+  public static final int DATA_TRANSFER_VERSION = 15;
+
+  // Processed at datanode stream-handler
+  public static final byte OP_WRITE_BLOCK = (byte) 80;
+  public static final byte OP_READ_BLOCK = (byte) 81;
+  /**
+   * @deprecated As of version 15, OP_READ_METADATA is no longer supported
+   */
+  @Deprecated public static final byte OP_READ_METADATA = (byte) 82;
+  public static final byte OP_REPLACE_BLOCK = (byte) 83;
+  public static final byte OP_COPY_BLOCK = (byte) 84;
+  public static final byte OP_BLOCK_CHECKSUM = (byte) 85;
+  
+  public static final int OP_STATUS_SUCCESS = 0;  
+  public static final int OP_STATUS_ERROR = 1;  
+  public static final int OP_STATUS_ERROR_CHECKSUM = 2;  
+  public static final int OP_STATUS_ERROR_INVALID = 3;  
+  public static final int OP_STATUS_ERROR_EXISTS = 4;  
+  public static final int OP_STATUS_ERROR_ACCESS_TOKEN = 5;
+  public static final int OP_STATUS_CHECKSUM_OK = 6;
+
+
+
+}

+ 188 - 0
src/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java

@@ -0,0 +1,188 @@
+/**
+ * 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.protocol;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.DeprecatedUTF8;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * DatanodeID is composed of the data node 
+ * name (hostname:portNumber) and the data storage ID, 
+ * which it currently represents.
+ * 
+ */
+public class DatanodeID implements WritableComparable<DatanodeID> {
+  public static final DatanodeID[] EMPTY_ARRAY = {}; 
+
+  public String name;      /// hostname:portNumber
+  public String storageID; /// unique per cluster storageID
+  protected int infoPort;     /// the port where the infoserver is running
+  public int ipcPort;     /// the port where the ipc server is running
+
+  /** Equivalent to DatanodeID(""). */
+  public DatanodeID() {this("");}
+
+  /** Equivalent to DatanodeID(nodeName, "", -1, -1). */
+  public DatanodeID(String nodeName) {this(nodeName, "", -1, -1);}
+
+  /**
+   * DatanodeID copy constructor
+   * 
+   * @param from
+   */
+  public DatanodeID(DatanodeID from) {
+    this(from.getName(),
+        from.getStorageID(),
+        from.getInfoPort(),
+        from.getIpcPort());
+  }
+  
+  /**
+   * Create DatanodeID
+   * @param nodeName (hostname:portNumber) 
+   * @param storageID data storage ID
+   * @param infoPort info server port 
+   * @param ipcPort ipc server port
+   */
+  public DatanodeID(String nodeName, String storageID,
+      int infoPort, int ipcPort) {
+    this.name = nodeName;
+    this.storageID = storageID;
+    this.infoPort = infoPort;
+    this.ipcPort = ipcPort;
+  }
+  
+  /**
+   * @return hostname:portNumber.
+   */
+  public String getName() {
+    return name;
+  }
+  
+  /**
+   * @return data storage ID.
+   */
+  public String getStorageID() {
+    return this.storageID;
+  }
+
+  /**
+   * @return infoPort (the port at which the HTTP server bound to)
+   */
+  public int getInfoPort() {
+    return infoPort;
+  }
+
+  /**
+   * @return ipcPort (the port at which the IPC server bound to)
+   */
+  public int getIpcPort() {
+    return ipcPort;
+  }
+
+  /**
+   * sets the data storage ID.
+   */
+  public void setStorageID(String storageID) {
+    this.storageID = storageID;
+  }
+
+  /**
+   * @return hostname and no :portNumber.
+   */
+  public String getHost() {
+    int colon = name.indexOf(":");
+    if (colon < 0) {
+      return name;
+    } else {
+      return name.substring(0, colon);
+    }
+  }
+  
+  public int getPort() {
+    int colon = name.indexOf(":");
+    if (colon < 0) {
+      return 50010; // default port.
+    }
+    return Integer.parseInt(name.substring(colon+1));
+  }
+
+  public boolean equals(Object to) {
+    if (this == to) {
+      return true;
+    }
+    if (!(to instanceof DatanodeID)) {
+      return false;
+    }
+    return (name.equals(((DatanodeID)to).getName()) &&
+            storageID.equals(((DatanodeID)to).getStorageID()));
+  }
+  
+  public int hashCode() {
+    return name.hashCode()^ storageID.hashCode();
+  }
+  
+  public String toString() {
+    return name;
+  }
+  
+  /**
+   * Update fields when a new registration request comes in.
+   * Note that this does not update storageID.
+   */
+  public void updateRegInfo(DatanodeID nodeReg) {
+    name = nodeReg.getName();
+    infoPort = nodeReg.getInfoPort();
+    // update any more fields added in future.
+  }
+    
+  /** Comparable.
+   * Basis of compare is the String name (host:portNumber) only.
+   * @param that
+   * @return as specified by Comparable.
+   */
+  public int compareTo(DatanodeID that) {
+    return name.compareTo(that.getName());
+  }
+
+  /////////////////////////////////////////////////
+  // Writable
+  /////////////////////////////////////////////////
+  /** {@inheritDoc} */
+  public void write(DataOutput out) throws IOException {
+    DeprecatedUTF8.writeString(out, name);
+    DeprecatedUTF8.writeString(out, storageID);
+    out.writeShort(infoPort);
+  }
+
+  /** {@inheritDoc} */
+  public void readFields(DataInput in) throws IOException {
+    name = DeprecatedUTF8.readString(in);
+    storageID = DeprecatedUTF8.readString(in);
+    // the infoPort read could be negative, if the port is a large number (more
+    // than 15 bits in storage size (but less than 16 bits).
+    // So chop off the first two bytes (and hence the signed bits) before 
+    // setting the field.
+    this.infoPort = in.readShort() & 0x0000ffff;
+  }
+}

+ 351 - 0
src/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -0,0 +1,351 @@
+/**
+ * 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.protocol;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Date;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.net.Node;
+import org.apache.hadoop.net.NodeBase;
+import org.apache.hadoop.util.StringUtils;
+
+/** 
+ * DatanodeInfo represents the status of a DataNode.
+ * This object is used for communication in the
+ * Datanode Protocol and the Client Protocol.
+ */
+public class DatanodeInfo extends DatanodeID implements Node {
+  protected long capacity;
+  protected long dfsUsed;
+  protected long remaining;
+  protected long lastUpdate;
+  protected int xceiverCount;
+  protected String location = NetworkTopology.DEFAULT_RACK;
+
+  /** HostName as supplied by the datanode during registration as its 
+   * name. Namenode uses datanode IP address as the name.
+   */
+  protected String hostName = null;
+  
+  // administrative states of a datanode
+  public enum AdminStates {NORMAL, DECOMMISSION_INPROGRESS, DECOMMISSIONED; }
+  protected AdminStates adminState;
+
+
+  public DatanodeInfo() {
+    super();
+    adminState = null;
+  }
+  
+  public DatanodeInfo(DatanodeInfo from) {
+    super(from);
+    this.capacity = from.getCapacity();
+    this.dfsUsed = from.getDfsUsed();
+    this.remaining = from.getRemaining();
+    this.lastUpdate = from.getLastUpdate();
+    this.xceiverCount = from.getXceiverCount();
+    this.location = from.getNetworkLocation();
+    this.adminState = from.adminState;
+    this.hostName = from.hostName;
+  }
+
+  public DatanodeInfo(DatanodeID nodeID) {
+    super(nodeID);
+    this.capacity = 0L;
+    this.dfsUsed = 0L;
+    this.remaining = 0L;
+    this.lastUpdate = 0L;
+    this.xceiverCount = 0;
+    this.adminState = null;    
+  }
+  
+  protected DatanodeInfo(DatanodeID nodeID, String location, String hostName) {
+    this(nodeID);
+    this.location = location;
+    this.hostName = hostName;
+  }
+  
+  /** The raw capacity. */
+  public long getCapacity() { return capacity; }
+  
+  /** The used space by the data node. */
+  public long getDfsUsed() { return dfsUsed; }
+
+  /** The used space by the data node. */
+  public long getNonDfsUsed() { 
+    long nonDFSUsed = capacity - dfsUsed - remaining;
+    return nonDFSUsed < 0 ? 0 : nonDFSUsed;
+  }
+
+  /** The used space by the data node as percentage of present capacity */
+  public float getDfsUsedPercent() { 
+    if (capacity <= 0) {
+      return 100;
+    }
+
+    return ((float)dfsUsed * 100.0f)/(float)capacity; 
+  }
+
+  /** The raw free space. */
+  public long getRemaining() { return remaining; }
+
+  /** The remaining space as percentage of configured capacity. */
+  public float getRemainingPercent() { 
+    if (capacity <= 0) {
+      return 0;
+    }
+
+    return ((float)remaining * 100.0f)/(float)capacity; 
+  }
+
+  /** The time when this information was accurate. */
+  public long getLastUpdate() { return lastUpdate; }
+
+  /** number of active connections */
+  public int getXceiverCount() { return xceiverCount; }
+
+  /** Sets raw capacity. */
+  public void setCapacity(long capacity) { 
+    this.capacity = capacity; 
+  }
+
+  /** Sets raw free space. */
+  public void setRemaining(long remaining) { 
+    this.remaining = remaining; 
+  }
+
+  /** Sets time when this information was accurate. */
+  public void setLastUpdate(long lastUpdate) { 
+    this.lastUpdate = lastUpdate; 
+  }
+
+  /** Sets number of active connections */
+  public void setXceiverCount(int xceiverCount) { 
+    this.xceiverCount = xceiverCount; 
+  }
+
+  /** rack name */
+  public synchronized String getNetworkLocation() {return location;}
+    
+  /** Sets the rack name */
+  public synchronized void setNetworkLocation(String location) {
+    this.location = NodeBase.normalize(location);
+  }
+  
+  public String getHostName() {
+    return (hostName == null || hostName.length()==0) ? getHost() : hostName;
+  }
+  
+  public void setHostName(String host) {
+    hostName = host;
+  }
+  
+  /** A formatted string for reporting the status of the DataNode. */
+  public String getDatanodeReport() {
+    StringBuffer buffer = new StringBuffer();
+    long c = getCapacity();
+    long r = getRemaining();
+    long u = getDfsUsed();
+    long nonDFSUsed = getNonDfsUsed();
+    float usedPercent = getDfsUsedPercent();
+    float remainingPercent = getRemainingPercent();
+    String hostName = NetUtils.getHostNameOfIP(name);
+
+    buffer.append("Name: "+ name);
+    if(hostName != null)
+      buffer.append(" (" + hostName + ")");
+    buffer.append("\n");
+
+    if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
+      buffer.append("Rack: "+location+"\n");
+    }
+    buffer.append("Decommission Status : ");
+    if (isDecommissioned()) {
+      buffer.append("Decommissioned\n");
+    } else if (isDecommissionInProgress()) {
+      buffer.append("Decommission in progress\n");
+    } else {
+      buffer.append("Normal\n");
+    }
+    buffer.append("Configured Capacity: "+c+" ("+StringUtils.byteDesc(c)+")"+"\n");
+    buffer.append("DFS Used: "+u+" ("+StringUtils.byteDesc(u)+")"+"\n");
+    buffer.append("Non DFS Used: "+nonDFSUsed+" ("+StringUtils.byteDesc(nonDFSUsed)+")"+"\n");
+    buffer.append("DFS Remaining: " +r+ " ("+StringUtils.byteDesc(r)+")"+"\n");
+    buffer.append("DFS Used%: "+StringUtils.limitDecimalTo2(usedPercent)+"%\n");
+    buffer.append("DFS Remaining%: "+StringUtils.limitDecimalTo2(remainingPercent)+"%\n");
+    buffer.append("Last contact: "+new Date(lastUpdate)+"\n");
+    return buffer.toString();
+  }
+
+  /** A formatted string for printing the status of the DataNode. */
+  public String dumpDatanode() {
+    StringBuffer buffer = new StringBuffer();
+    long c = getCapacity();
+    long r = getRemaining();
+    long u = getDfsUsed();
+    buffer.append(name);
+    if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
+      buffer.append(" "+location);
+    }
+    if (isDecommissioned()) {
+      buffer.append(" DD");
+    } else if (isDecommissionInProgress()) {
+      buffer.append(" DP");
+    } else {
+      buffer.append(" IN");
+    }
+    buffer.append(" " + c + "(" + StringUtils.byteDesc(c)+")");
+    buffer.append(" " + u + "(" + StringUtils.byteDesc(u)+")");
+    buffer.append(" " + StringUtils.limitDecimalTo2(((1.0*u)/c)*100)+"%");
+    buffer.append(" " + r + "(" + StringUtils.byteDesc(r)+")");
+    buffer.append(" " + new Date(lastUpdate));
+    return buffer.toString();
+  }
+
+  /**
+   * Start decommissioning a node.
+   * old state.
+   */
+  public void startDecommission() {
+    adminState = AdminStates.DECOMMISSION_INPROGRESS;
+  }
+
+  /**
+   * Stop decommissioning a node.
+   * old state.
+   */
+  public void stopDecommission() {
+    adminState = null;
+  }
+
+  /**
+   * Returns true if the node is in the process of being decommissioned
+   */
+  public boolean isDecommissionInProgress() {
+    if (adminState == AdminStates.DECOMMISSION_INPROGRESS) {
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Returns true if the node has been decommissioned.
+   */
+  public boolean isDecommissioned() {
+    if (adminState == AdminStates.DECOMMISSIONED) {
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Sets the admin state to indicate that decommision is complete.
+   */
+  public void setDecommissioned() {
+    adminState = AdminStates.DECOMMISSIONED;
+  }
+
+  /**
+   * Retrieves the admin state of this node.
+   */
+  AdminStates getAdminState() {
+    if (adminState == null) {
+      return AdminStates.NORMAL;
+    }
+    return adminState;
+  }
+
+  /**
+   * Sets the admin state of this node.
+   */
+  protected void setAdminState(AdminStates newState) {
+    if (newState == AdminStates.NORMAL) {
+      adminState = null;
+    }
+    else {
+      adminState = newState;
+    }
+  }
+
+  private int level; //which level of the tree the node resides
+  private Node parent; //its parent
+
+  /** Return this node's parent */
+  public Node getParent() { return parent; }
+  public void setParent(Node parent) {this.parent = parent;}
+   
+  /** Return this node's level in the tree.
+   * E.g. the root of a tree returns 0 and its children return 1
+   */
+  public int getLevel() { return level; }
+  public void setLevel(int level) {this.level = level;}
+
+  /////////////////////////////////////////////////
+  // Writable
+  /////////////////////////////////////////////////
+  static {                                      // register a ctor
+    WritableFactories.setFactory
+      (DatanodeInfo.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new DatanodeInfo(); }
+       });
+  }
+
+  /** {@inheritDoc} */
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+
+    //TODO: move it to DatanodeID once DatanodeID is not stored in FSImage
+    out.writeShort(ipcPort);
+
+    out.writeLong(capacity);
+    out.writeLong(dfsUsed);
+    out.writeLong(remaining);
+    out.writeLong(lastUpdate);
+    out.writeInt(xceiverCount);
+    Text.writeString(out, location);
+    Text.writeString(out, hostName == null? "": hostName);
+    WritableUtils.writeEnum(out, getAdminState());
+  }
+
+  /** {@inheritDoc} */
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+
+    //TODO: move it to DatanodeID once DatanodeID is not stored in FSImage
+    this.ipcPort = in.readShort() & 0x0000ffff;
+
+    this.capacity = in.readLong();
+    this.dfsUsed = in.readLong();
+    this.remaining = in.readLong();
+    this.lastUpdate = in.readLong();
+    this.xceiverCount = in.readInt();
+    this.location = Text.readString(in);
+    this.hostName = Text.readString(in);
+    setAdminState(WritableUtils.readEnum(in, AdminStates.class));
+  }
+}

+ 86 - 0
src/java/org/apache/hadoop/hdfs/protocol/FSConstants.java

@@ -0,0 +1,86 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.conf.Configuration;
+
+/************************************
+ * Some handy constants
+ *
+ ************************************/
+public interface FSConstants {
+  public static int MIN_BLOCKS_FOR_WRITE = 5;
+
+  // Chunk the block Invalidate message
+  public static final int BLOCK_INVALIDATE_CHUNK = 100;
+
+  // Long that indicates "leave current quota unchanged"
+  public static final long QUOTA_DONT_SET = Long.MAX_VALUE;
+  public static final long QUOTA_RESET = -1L;
+  
+  //
+  // Timeouts, constants
+  //
+  public static long HEARTBEAT_INTERVAL = 3;
+  public static long BLOCKREPORT_INTERVAL = 60 * 60 * 1000;
+  public static long BLOCKREPORT_INITIAL_DELAY = 0;
+  public static final long LEASE_SOFTLIMIT_PERIOD = 60 * 1000;
+  public static final long LEASE_HARDLIMIT_PERIOD = 60 * LEASE_SOFTLIMIT_PERIOD;
+  public static final long LEASE_RECOVER_PERIOD = 10 * 1000; //in ms
+  
+  // We need to limit the length and depth of a path in the filesystem.  HADOOP-438
+  // Currently we set the maximum length to 8k characters and the maximum depth to 1k.  
+  public static int MAX_PATH_LENGTH = 8000;
+  public static int MAX_PATH_DEPTH = 1000;
+    
+  public static final int BUFFER_SIZE = new Configuration().getInt("io.file.buffer.size", 4096);
+  //Used for writing header etc.
+  public static final int SMALL_BUFFER_SIZE = Math.min(BUFFER_SIZE/2, 512);
+  //TODO mb@media-style.com: should be conf injected?
+  public static final long DEFAULT_BLOCK_SIZE = 64 * 1024 * 1024;
+  public static final int DEFAULT_DATA_SOCKET_SIZE = 128 * 1024;
+
+  public static final int SIZE_OF_INTEGER = Integer.SIZE / Byte.SIZE;
+
+  // SafeMode actions
+  public enum SafeModeAction{ SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET; }
+
+  // type of the datanode report
+  public static enum DatanodeReportType {ALL, LIVE, DEAD }
+
+  /**
+   * Distributed upgrade actions:
+   * 
+   * 1. Get upgrade status.
+   * 2. Get detailed upgrade status.
+   * 3. Proceed with the upgrade if it is stuck, no matter what the status is.
+   */
+  public static enum UpgradeAction {
+    GET_STATUS,
+    DETAILED_STATUS,
+    FORCE_PROCEED;
+  }
+
+  // Version is reflected in the dfs image and edit log files.
+  // Version is reflected in the data storage file.
+  // Versions are negative.
+  // Decrement LAYOUT_VERSION to define a new version.
+  public static final int LAYOUT_VERSION = -19;
+  // Current version: 
+  // -19: Sticky bit
+}

+ 148 - 0
src/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java

@@ -0,0 +1,148 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.security.AccessToken;
+
+import java.io.*;
+
+/****************************************************
+ * A LocatedBlock is a pair of Block, DatanodeInfo[]
+ * objects.  It tells where to find a Block.
+ * 
+ ****************************************************/
+public class LocatedBlock implements Writable {
+
+  static {                                      // register a ctor
+    WritableFactories.setFactory
+      (LocatedBlock.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new LocatedBlock(); }
+       });
+  }
+
+  private Block b;
+  private long offset;  // offset of the first byte of the block in the file
+  private DatanodeInfo[] locs;
+  // corrupt flag is true if all of the replicas of a block are corrupt.
+  // else false. If block has few corrupt replicas, they are filtered and 
+  // their locations are not part of this object
+  private boolean corrupt;
+  private AccessToken accessToken = new AccessToken();
+
+  /**
+   */
+  public LocatedBlock() {
+    this(new Block(), new DatanodeInfo[0], 0L, false);
+  }
+
+  /**
+   */
+  public LocatedBlock(Block b, DatanodeInfo[] locs) {
+    this(b, locs, -1, false); // startOffset is unknown
+  }
+
+  /**
+   */
+  public LocatedBlock(Block b, DatanodeInfo[] locs, long startOffset) {
+    this(b, locs, startOffset, false);
+  }
+
+  /**
+   */
+  public LocatedBlock(Block b, DatanodeInfo[] locs, long startOffset, 
+                      boolean corrupt) {
+    this.b = b;
+    this.offset = startOffset;
+    this.corrupt = corrupt;
+    if (locs==null) {
+      this.locs = new DatanodeInfo[0];
+    } else {
+      this.locs = locs;
+    }
+  }
+
+  public AccessToken getAccessToken() {
+    return accessToken;
+  }
+
+  public void setAccessToken(AccessToken token) {
+    this.accessToken = token;
+  }
+
+  /**
+   */
+  public Block getBlock() {
+    return b;
+  }
+
+  /**
+   */
+  public DatanodeInfo[] getLocations() {
+    return locs;
+  }
+  
+  public long getStartOffset() {
+    return offset;
+  }
+  
+  public long getBlockSize() {
+    return b.getNumBytes();
+  }
+
+  void setStartOffset(long value) {
+    this.offset = value;
+  }
+
+  void setCorrupt(boolean corrupt) {
+    this.corrupt = corrupt;
+  }
+  
+  public boolean isCorrupt() {
+    return this.corrupt;
+  }
+
+  ///////////////////////////////////////////
+  // Writable
+  ///////////////////////////////////////////
+  public void write(DataOutput out) throws IOException {
+    accessToken.write(out);
+    out.writeBoolean(corrupt);
+    out.writeLong(offset);
+    b.write(out);
+    out.writeInt(locs.length);
+    for (int i = 0; i < locs.length; i++) {
+      locs[i].write(out);
+    }
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    accessToken.readFields(in);
+    this.corrupt = in.readBoolean();
+    offset = in.readLong();
+    this.b = new Block();
+    b.readFields(in);
+    int count = in.readInt();
+    this.locs = new DatanodeInfo[count];
+    for (int i = 0; i < locs.length; i++) {
+      locs[i] = new DatanodeInfo();
+      locs[i].readFields(in);
+    }
+  }
+}

+ 187 - 0
src/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java

@@ -0,0 +1,187 @@
+/**
+ * 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.protocol;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+/**
+ * Collection of blocks with their locations and the file length.
+ */
+public class LocatedBlocks implements Writable {
+  private long fileLength;
+  private List<LocatedBlock> blocks; // array of blocks with prioritized locations
+  private boolean underConstruction;
+
+  LocatedBlocks() {
+    fileLength = 0;
+    blocks = null;
+    underConstruction = false;
+  }
+  
+  public LocatedBlocks(long flength, List<LocatedBlock> blks, boolean isUnderConstuction) {
+
+    fileLength = flength;
+    blocks = blks;
+    underConstruction = isUnderConstuction;
+  }
+  
+  /**
+   * Get located blocks.
+   */
+  public List<LocatedBlock> getLocatedBlocks() {
+    return blocks;
+  }
+  
+  /**
+   * Get located block.
+   */
+  public LocatedBlock get(int index) {
+    return blocks.get(index);
+  }
+  
+  /**
+   * Get number of located blocks.
+   */
+  public int locatedBlockCount() {
+    return blocks == null ? 0 : blocks.size();
+  }
+
+  /**
+   * 
+   */
+  public long getFileLength() {
+    return this.fileLength;
+  }
+
+  /**
+   * Return ture if file was under construction when 
+   * this LocatedBlocks was constructed, false otherwise.
+   */
+  public boolean isUnderConstruction() {
+    return underConstruction;
+  }
+  
+  /**
+   * Find block containing specified offset.
+   * 
+   * @return block if found, or null otherwise.
+   */
+  public int findBlock(long offset) {
+    // create fake block of size 1 as a key
+    LocatedBlock key = new LocatedBlock();
+    key.setStartOffset(offset);
+    key.getBlock().setNumBytes(1);
+    Comparator<LocatedBlock> comp = 
+      new Comparator<LocatedBlock>() {
+        // Returns 0 iff a is inside b or b is inside a
+        public int compare(LocatedBlock a, LocatedBlock b) {
+          long aBeg = a.getStartOffset();
+          long bBeg = b.getStartOffset();
+          long aEnd = aBeg + a.getBlockSize();
+          long bEnd = bBeg + b.getBlockSize();
+          if(aBeg <= bBeg && bEnd <= aEnd 
+              || bBeg <= aBeg && aEnd <= bEnd)
+            return 0; // one of the blocks is inside the other
+          if(aBeg < bBeg)
+            return -1; // a's left bound is to the left of the b's
+          return 1;
+        }
+      };
+    return Collections.binarySearch(blocks, key, comp);
+  }
+  
+  public void insertRange(int blockIdx, List<LocatedBlock> newBlocks) {
+    int oldIdx = blockIdx;
+    int insStart = 0, insEnd = 0;
+    for(int newIdx = 0; newIdx < newBlocks.size() && oldIdx < blocks.size(); 
+                                                        newIdx++) {
+      long newOff = newBlocks.get(newIdx).getStartOffset();
+      long oldOff = blocks.get(oldIdx).getStartOffset();
+      if(newOff < oldOff) {
+        insEnd++;
+      } else if(newOff == oldOff) {
+        // replace old cached block by the new one
+        blocks.set(oldIdx, newBlocks.get(newIdx));
+        if(insStart < insEnd) { // insert new blocks
+          blocks.addAll(oldIdx, newBlocks.subList(insStart, insEnd));
+          oldIdx += insEnd - insStart;
+        }
+        insStart = insEnd = newIdx+1;
+        oldIdx++;
+      } else {  // newOff > oldOff
+        assert false : "List of LocatedBlock must be sorted by startOffset";
+      }
+    }
+    insEnd = newBlocks.size();
+    if(insStart < insEnd) { // insert new blocks
+      blocks.addAll(oldIdx, newBlocks.subList(insStart, insEnd));
+    }
+  }
+  
+  public static int getInsertIndex(int binSearchResult) {
+    return binSearchResult >= 0 ? binSearchResult : -(binSearchResult+1);
+  }
+
+  //////////////////////////////////////////////////
+  // Writable
+  //////////////////////////////////////////////////
+  static {                                      // register a ctor
+    WritableFactories.setFactory
+      (LocatedBlocks.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new LocatedBlocks(); }
+       });
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(this.fileLength);
+    out.writeBoolean(underConstruction);
+    // write located blocks
+    int nrBlocks = locatedBlockCount();
+    out.writeInt(nrBlocks);
+    if (nrBlocks == 0) {
+      return;
+    }
+    for (LocatedBlock blk : this.blocks) {
+      blk.write(out);
+    }
+  }
+  
+  public void readFields(DataInput in) throws IOException {
+    this.fileLength = in.readLong();
+    underConstruction = in.readBoolean();
+    // read located blocks
+    int nrBlocks = in.readInt();
+    this.blocks = new ArrayList<LocatedBlock>(nrBlocks);
+    for (int idx = 0; idx < nrBlocks; idx++) {
+      LocatedBlock blk = new LocatedBlock();
+      blk.readFields(in);
+      this.blocks.add(blk);
+    }
+  }
+}

+ 67 - 0
src/java/org/apache/hadoop/hdfs/protocol/QuotaExceededException.java

@@ -0,0 +1,67 @@
+/**
+ * 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.protocol;
+
+import java.io.IOException;
+
+/** 
+ * This exception is thrown when modification to HDFS results in violation
+ * of a directory quota. A directory quota might be namespace quota (limit 
+ * on number of files and directories) or a diskspace quota (limit on space 
+ * taken by all the file under the directory tree). <br> <br>
+ * 
+ * The message for the exception specifies the directory where the quota
+ * was violated and actual quotas.
+ */
+public final class QuotaExceededException extends IOException {
+  private static final long serialVersionUID = 1L;
+  private String pathName;
+  private long nsQuota;
+  private long nsCount;
+  private long dsQuota;
+  private long diskspace;
+  
+  public QuotaExceededException(String msg) {
+    super(msg);
+  }
+  
+  public QuotaExceededException(long nsQuota, long nsCount,
+                                long dsQuota, long diskspace) {
+    this.nsQuota = nsQuota;
+    this.nsCount = nsCount;
+    this.dsQuota = dsQuota;
+    this.diskspace = diskspace;
+  }
+  
+  public void setPathName(String path) {
+    this.pathName = path;
+  }
+  
+  public String getMessage() {
+    String msg = super.getMessage();
+    if (msg == null) {
+      return "The quota" + (pathName==null?"":(" of " + pathName)) + 
+          " is exceeded: namespace quota=" + nsQuota + " file count=" + 
+          nsCount + ", diskspace quota=" + dsQuota + 
+          " diskspace=" + diskspace; 
+    } else {
+      return msg;
+    }
+  }
+}

+ 49 - 0
src/java/org/apache/hadoop/hdfs/protocol/UnregisteredNodeException.java

@@ -0,0 +1,49 @@
+/**
+ * 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.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.protocol.NodeRegistration;
+
+/**
+ * This exception is thrown when a node that has not previously 
+ * registered is trying to access the name node.
+ */
+public class UnregisteredNodeException extends IOException {
+  private static final long serialVersionUID = -5620209396945970810L;
+
+  public UnregisteredNodeException(NodeRegistration nodeReg) {
+    super("Unregistered server: " + nodeReg.toString());
+  }
+
+  /**
+   * The exception is thrown if a different data-node claims the same
+   * storage id as the existing one.
+   *  
+   * @param nodeID unregistered data-node
+   * @param storedNode data-node stored in the system with this storage id
+   */
+  public UnregisteredNodeException(DatanodeID nodeID, DatanodeInfo storedNode) {
+    super("Data node " + nodeID.getName() 
+          + " is attempting to report storage ID "
+          + nodeID.getStorageID() + ". Node " 
+          + storedNode.getName() + " is expected to serve this storage.");
+  }
+}

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

@@ -0,0 +1,1651 @@
+/**
+ * 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.balancer;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Date;
+import java.util.EnumSet;
+import java.util.Formatter;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.Util;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.security.AccessToken;
+import org.apache.hadoop.security.AccessTokenHandler;
+import org.apache.hadoop.security.ExportedAccessKeys;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/** <p>The balancer is a tool that balances disk space usage on an HDFS cluster
+ * when some datanodes become full or when new empty nodes join the cluster.
+ * The tool is deployed as an application program that can be run by the 
+ * cluster administrator on a live HDFS cluster while applications
+ * adding and deleting files.
+ * 
+ * <p>SYNOPSIS
+ * <pre>
+ * To start:
+ *      bin/start-balancer.sh [-threshold <threshold>]
+ *      Example: bin/ start-balancer.sh 
+ *                     start the balancer with a default threshold of 10%
+ *               bin/ start-balancer.sh -threshold 5
+ *                     start the balancer with a threshold of 5%
+ * To stop:
+ *      bin/ stop-balancer.sh
+ * </pre>
+ * 
+ * <p>DESCRIPTION
+ * <p>The threshold parameter is a fraction in the range of (0%, 100%) with a 
+ * default value of 10%. The threshold sets a target for whether the cluster 
+ * is balanced. A cluster is balanced if for each datanode, the utilization 
+ * of the node (ratio of used space at the node to total capacity of the node) 
+ * differs from the utilization of the (ratio of used space in the cluster 
+ * to total capacity of the cluster) by no more than the threshold value. 
+ * The smaller the threshold, the more balanced a cluster will become. 
+ * It takes more time to run the balancer for small threshold values. 
+ * Also for a very small threshold the cluster may not be able to reach the 
+ * balanced state when applications write and delete files concurrently.
+ * 
+ * <p>The tool moves blocks from highly utilized datanodes to poorly 
+ * utilized datanodes iteratively. In each iteration a datanode moves or 
+ * receives no more than the lesser of 10G bytes or the threshold fraction 
+ * of its capacity. Each iteration runs no more than 20 minutes.
+ * At the end of each iteration, the balancer obtains updated datanodes
+ * information from the namenode.
+ * 
+ * <p>A system property that limits the balancer's use of bandwidth is 
+ * defined in the default configuration file:
+ * <pre>
+ * <property>
+ *   <name>dfs.balance.bandwidthPerSec</name>
+ *   <value>1048576</value>
+ * <description>  Specifies the maximum bandwidth that each datanode 
+ * can utilize for the balancing purpose in term of the number of bytes 
+ * per second. </description>
+ * </property>
+ * </pre>
+ * 
+ * <p>This property determines the maximum speed at which a block will be 
+ * moved from one datanode to another. The default value is 1MB/s. The higher 
+ * the bandwidth, the faster a cluster can reach the balanced state, 
+ * but with greater competition with application processes. If an 
+ * administrator changes the value of this property in the configuration 
+ * file, the change is observed when HDFS is next restarted.
+ * 
+ * <p>MONITERING BALANCER PROGRESS
+ * <p>After the balancer is started, an output file name where the balancer 
+ * progress will be recorded is printed on the screen.  The administrator 
+ * can monitor the running of the balancer by reading the output file. 
+ * The output shows the balancer's status iteration by iteration. In each 
+ * iteration it prints the starting time, the iteration number, the total 
+ * number of bytes that have been moved in the previous iterations, 
+ * the total number of bytes that are left to move in order for the cluster 
+ * to be balanced, and the number of bytes that are being moved in this 
+ * iteration. Normally "Bytes Already Moved" is increasing while "Bytes Left 
+ * To Move" is decreasing.
+ * 
+ * <p>Running multiple instances of the balancer in an HDFS cluster is 
+ * prohibited by the tool.
+ * 
+ * <p>The balancer automatically exits when any of the following five 
+ * conditions is satisfied:
+ * <ol>
+ * <li>The cluster is balanced;
+ * <li>No block can be moved;
+ * <li>No block has been moved for five consecutive iterations;
+ * <li>An IOException occurs while communicating with the namenode;
+ * <li>Another balancer is running.
+ * </ol>
+ * 
+ * <p>Upon exit, a balancer returns an exit code and prints one of the 
+ * following messages to the output file in corresponding to the above exit 
+ * reasons:
+ * <ol>
+ * <li>The cluster is balanced. Exiting
+ * <li>No block can be moved. Exiting...
+ * <li>No block has been moved for 3 iterations. Exiting...
+ * <li>Received an IO exception: failure reason. Exiting...
+ * <li>Another balancer is running. Exiting...
+ * </ol>
+ * 
+ * <p>The administrator can interrupt the execution of the balancer at any 
+ * time by running the command "stop-balancer.sh" on the machine where the 
+ * balancer is running.
+ */
+
+public class Balancer implements Tool {
+  private static final Log LOG = 
+    LogFactory.getLog(Balancer.class.getName());
+  final private static long MAX_BLOCKS_SIZE_TO_FETCH = 2*1024*1024*1024L; //2GB
+
+  /** The maximum number of concurrent blocks moves for 
+   * balancing purpose at a datanode
+   */
+  public static final int MAX_NUM_CONCURRENT_MOVES = 5;
+  
+  private Configuration conf;
+
+  private double threshold = 10D;
+  private NamenodeProtocol namenode;
+  private ClientProtocol client;
+  private FileSystem fs;
+  private boolean isAccessTokenEnabled;
+  private boolean shouldRun;
+  private long keyUpdaterInterval;
+  private AccessTokenHandler accessTokenHandler;
+  private Daemon keyupdaterthread = null; // AccessKeyUpdater thread
+  private final static Random rnd = new Random();
+  
+  // all data node lists
+  private Collection<Source> overUtilizedDatanodes
+                               = new LinkedList<Source>();
+  private Collection<Source> aboveAvgUtilizedDatanodes
+                               = new LinkedList<Source>();
+  private Collection<BalancerDatanode> belowAvgUtilizedDatanodes
+                               = new LinkedList<BalancerDatanode>();
+  private Collection<BalancerDatanode> underUtilizedDatanodes
+                               = new LinkedList<BalancerDatanode>();
+  
+  private Collection<Source> sources
+                               = new HashSet<Source>();
+  private Collection<BalancerDatanode> targets
+                               = new HashSet<BalancerDatanode>();
+  
+  private Map<Block, BalancerBlock> globalBlockList
+                 = new HashMap<Block, BalancerBlock>();
+  private MovedBlocks movedBlocks = new MovedBlocks();
+  private Map<String, BalancerDatanode> datanodes
+                 = new HashMap<String, BalancerDatanode>();
+  
+  private NetworkTopology cluster = new NetworkTopology();
+  
+  private double avgUtilization = 0.0D;
+  
+  final static private int MOVER_THREAD_POOL_SIZE = 1000;
+  final private ExecutorService moverExecutor = 
+    Executors.newFixedThreadPool(MOVER_THREAD_POOL_SIZE);
+  final static private int DISPATCHER_THREAD_POOL_SIZE = 200;
+  final private ExecutorService dispatcherExecutor =
+    Executors.newFixedThreadPool(DISPATCHER_THREAD_POOL_SIZE);
+  
+  /* This class keeps track of a scheduled block move */
+  private class PendingBlockMove {
+    private BalancerBlock block;
+    private Source source;
+    private BalancerDatanode proxySource;
+    private BalancerDatanode target;
+    
+    /** constructor */
+    private PendingBlockMove() {
+    }
+    
+    /* choose a block & a proxy source for this pendingMove 
+     * whose source & target have already been chosen.
+     * 
+     * Return true if a block and its proxy are chosen; false otherwise
+     */
+    private boolean chooseBlockAndProxy() {
+      // iterate all source's blocks until find a good one    
+      for (Iterator<BalancerBlock> blocks=
+        source.getBlockIterator(); blocks.hasNext();) {
+        if (markMovedIfGoodBlock(blocks.next())) {
+          blocks.remove();
+          return true;
+        }
+      }
+      return false;
+    }
+    
+    /* Return true if the given block is good for the tentative move;
+     * If it is good, add it to the moved list to marked as "Moved".
+     * A block is good if
+     * 1. it is a good candidate; see isGoodBlockCandidate
+     * 2. can find a proxy source that's not busy for this move
+     */
+    private boolean markMovedIfGoodBlock(BalancerBlock block) {
+      synchronized(block) {
+        synchronized(movedBlocks) {
+          if (isGoodBlockCandidate(source, target, block)) {
+            this.block = block;
+            if ( chooseProxySource() ) {
+              movedBlocks.add(block);
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Decided to move block "+ block.getBlockId()
+                    +" with a length of "+StringUtils.byteDesc(block.getNumBytes())
+                    + " bytes from " + source.getName() 
+                    + " to " + target.getName()
+                    + " using proxy source " + proxySource.getName() );
+              }
+              return true;
+            }
+          }
+        }
+      }
+      return false;
+    }
+    
+    /* Now we find out source, target, and block, we need to find a proxy
+     * 
+     * @return true if a proxy is found; otherwise false
+     */
+    private boolean chooseProxySource() {
+      // check if there is replica which is on the same rack with the target
+      for (BalancerDatanode loc : block.getLocations()) {
+        if (cluster.isOnSameRack(loc.getDatanode(), target.getDatanode())) {
+          if (loc.addPendingBlock(this)) {
+            proxySource = loc;
+            return true;
+          }
+        }
+      }
+      // find out a non-busy replica
+      for (BalancerDatanode loc : block.getLocations()) {
+        if (loc.addPendingBlock(this)) {
+          proxySource = loc;
+          return true;
+        }
+      }
+      return false;
+    }
+    
+    /* Dispatch the block move task to the proxy source & wait for the response
+     */
+    private void dispatch() {
+      Socket sock = new Socket();
+      DataOutputStream out = null;
+      DataInputStream in = null;
+      try {
+        sock.connect(NetUtils.createSocketAddr(
+            target.datanode.getName()), HdfsConstants.READ_TIMEOUT);
+        sock.setKeepAlive(true);
+        out = new DataOutputStream( new BufferedOutputStream(
+            sock.getOutputStream(), FSConstants.BUFFER_SIZE));
+        sendRequest(out);
+        in = new DataInputStream( new BufferedInputStream(
+            sock.getInputStream(), FSConstants.BUFFER_SIZE));
+        receiveResponse(in);
+        bytesMoved.inc(block.getNumBytes());
+        LOG.info( "Moving block " + block.getBlock().getBlockId() +
+              " from "+ source.getName() + " to " +
+              target.getName() + " through " +
+              proxySource.getName() +
+              " is succeeded." );
+      } catch (IOException e) {
+        LOG.warn("Error moving block "+block.getBlockId()+
+            " from " + source.getName() + " to " +
+            target.getName() + " through " +
+            proxySource.getName() +
+            ": "+e.getMessage());
+      } finally {
+        IOUtils.closeStream(out);
+        IOUtils.closeStream(in);
+        IOUtils.closeSocket(sock);
+        
+        proxySource.removePendingBlock(this);
+        synchronized(target) {
+          target.removePendingBlock(this);
+        }
+
+        synchronized (this ) {
+          reset();
+        }
+        synchronized (Balancer.this) {
+          Balancer.this.notifyAll();
+        }
+      }
+    }
+    
+    /* Send a block replace request to the output stream*/
+    private void sendRequest(DataOutputStream out) throws IOException {
+      out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
+      out.writeByte(DataTransferProtocol.OP_REPLACE_BLOCK);
+      out.writeLong(block.getBlock().getBlockId());
+      out.writeLong(block.getBlock().getGenerationStamp());
+      Text.writeString(out, source.getStorageID());
+      proxySource.write(out);
+      AccessToken accessToken = AccessToken.DUMMY_TOKEN;
+      if (isAccessTokenEnabled) {
+        accessToken = accessTokenHandler.generateToken(null, block.getBlock()
+            .getBlockId(), EnumSet.of(AccessTokenHandler.AccessMode.REPLACE,
+            AccessTokenHandler.AccessMode.COPY));
+      }
+      accessToken.write(out);
+      out.flush();
+    }
+    
+    /* Receive a block copy response from the input stream */ 
+    private void receiveResponse(DataInputStream in) throws IOException {
+      short status = in.readShort();
+      if (status != DataTransferProtocol.OP_STATUS_SUCCESS) {
+        if (status == DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN)
+          throw new IOException("block move failed due to access token error");
+        throw new IOException("block move is failed");
+      }
+    }
+
+    /* reset the object */
+    private void reset() {
+      block = null;
+      source = null;
+      proxySource = null;
+      target = null;
+    }
+    
+    /* start a thread to dispatch the block move */
+    private void scheduleBlockMove() {
+      moverExecutor.execute(new Runnable() {
+        public void run() {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Starting moving "+ block.getBlockId() +
+                " from " + proxySource.getName() + " to " + target.getName());
+          }
+          dispatch();
+        }
+      });
+    }
+  }
+  
+  /* A class for keeping track of blocks in the Balancer */
+  static private class BalancerBlock {
+    private Block block; // the block
+    private List<BalancerDatanode> locations
+            = new ArrayList<BalancerDatanode>(3); // its locations
+    
+    /* Constructor */
+    private BalancerBlock(Block block) {
+      this.block = block;
+    }
+    
+    /* clean block locations */
+    private synchronized void clearLocations() {
+      locations.clear();
+    }
+    
+    /* add a location */
+    private synchronized void addLocation(BalancerDatanode datanode) {
+      if (!locations.contains(datanode)) {
+        locations.add(datanode);
+      }
+    }
+    
+    /* Return if the block is located on <code>datanode</code> */
+    private synchronized boolean isLocatedOnDatanode(
+        BalancerDatanode datanode) {
+      return locations.contains(datanode);
+    }
+    
+    /* Return its locations */
+    private synchronized List<BalancerDatanode> getLocations() {
+      return locations;
+    }
+    
+    /* Return the block */
+    private Block getBlock() {
+      return block;
+    }
+    
+    /* Return the block id */
+    private long getBlockId() {
+      return block.getBlockId();
+    }
+    
+    /* Return the length of the block */
+    private long getNumBytes() {
+      return block.getNumBytes();
+    }
+  }
+  
+  /* The class represents a desired move of bytes between two nodes 
+   * and the target.
+   * An object of this class is stored in a source node. 
+   */
+  static private class NodeTask {
+    private BalancerDatanode datanode; //target node
+    private long size;  //bytes scheduled to move
+    
+    /* constructor */
+    private NodeTask(BalancerDatanode datanode, long size) {
+      this.datanode = datanode;
+      this.size = size;
+    }
+    
+    /* Get the node */
+    private BalancerDatanode getDatanode() {
+      return datanode;
+    }
+    
+    /* Get the number of bytes that need to be moved */
+    private long getSize() {
+      return size;
+    }
+  }
+  
+  /* Return the utilization of a datanode */
+  static private double getUtilization(DatanodeInfo datanode) {
+    return ((double)datanode.getDfsUsed())/datanode.getCapacity()*100;
+  }
+  
+  /* A class that keeps track of a datanode in Balancer */
+  private static class BalancerDatanode implements Writable {
+    final private static long MAX_SIZE_TO_MOVE = 10*1024*1024*1024L; //10GB
+    protected DatanodeInfo datanode;
+    private double utilization;
+    protected long maxSizeToMove;
+    protected long scheduledSize = 0L;
+    //  blocks being moved but not confirmed yet
+    private List<PendingBlockMove> pendingBlocks = 
+      new ArrayList<PendingBlockMove>(MAX_NUM_CONCURRENT_MOVES); 
+    
+    /* Constructor 
+     * Depending on avgutil & threshold, calculate maximum bytes to move 
+     */
+    private BalancerDatanode(
+        DatanodeInfo node, double avgUtil, double threshold) {
+      datanode = node;
+      utilization = Balancer.getUtilization(node);
+        
+      if (utilization >= avgUtil+threshold
+          || utilization <= avgUtil-threshold) { 
+        maxSizeToMove = (long)(threshold*datanode.getCapacity()/100);
+      } else {
+        maxSizeToMove = 
+          (long)(Math.abs(avgUtil-utilization)*datanode.getCapacity()/100);
+      }
+      if (utilization < avgUtil ) {
+        maxSizeToMove = Math.min(datanode.getRemaining(), maxSizeToMove);
+      }
+      maxSizeToMove = Math.min(MAX_SIZE_TO_MOVE, maxSizeToMove);
+    }
+    
+    /** Get the datanode */
+    protected DatanodeInfo getDatanode() {
+      return datanode;
+    }
+    
+    /** Get the name of the datanode */
+    protected String getName() {
+      return datanode.getName();
+    }
+    
+    /* Get the storage id of the datanode */
+    protected String getStorageID() {
+      return datanode.getStorageID();
+    }
+    
+    /** Decide if still need to move more bytes */
+    protected boolean isMoveQuotaFull() {
+      return scheduledSize<maxSizeToMove;
+    }
+
+    /** Return the total number of bytes that need to be moved */
+    protected long availableSizeToMove() {
+      return maxSizeToMove-scheduledSize;
+    }
+    
+    /* increment scheduled size */
+    protected void incScheduledSize(long size) {
+      scheduledSize += size;
+    }
+    
+    /* Check if the node can schedule more blocks to move */
+    synchronized private boolean isPendingQNotFull() {
+      if ( pendingBlocks.size() < MAX_NUM_CONCURRENT_MOVES ) {
+        return true;
+      }
+      return false;
+    }
+    
+    /* Check if all the dispatched moves are done */
+    synchronized private boolean isPendingQEmpty() {
+      return pendingBlocks.isEmpty();
+    }
+    
+    /* Add a scheduled block move to the node */
+    private synchronized boolean addPendingBlock(
+        PendingBlockMove pendingBlock) {
+      if (isPendingQNotFull()) {
+        return pendingBlocks.add(pendingBlock);
+      }
+      return false;
+    }
+    
+    /* Remove a scheduled block move from the node */
+    private synchronized boolean  removePendingBlock(
+        PendingBlockMove pendingBlock) {
+      return pendingBlocks.remove(pendingBlock);
+    }
+
+    /** The following two methods support the Writable interface */
+    /** Deserialize */
+    public void readFields(DataInput in) throws IOException {
+      datanode.readFields(in);
+    }
+
+    /** Serialize */
+    public void write(DataOutput out) throws IOException {
+      datanode.write(out);
+    }
+  }
+  
+  /** A node that can be the sources of a block move */
+  private class Source extends BalancerDatanode {
+    
+    /* A thread that initiates a block move 
+     * and waits for block move to complete */
+    private class BlockMoveDispatcher implements Runnable {
+      public void run() {
+        dispatchBlocks();
+      }
+    }
+    
+    private ArrayList<NodeTask> nodeTasks = new ArrayList<NodeTask>(2);
+    private long blocksToReceive = 0L;
+    /* source blocks point to balancerBlocks in the global list because
+     * we want to keep one copy of a block in balancer and be aware that
+     * the locations are changing over time.
+     */
+    private List<BalancerBlock> srcBlockList
+            = new ArrayList<BalancerBlock>();
+    
+    /* constructor */
+    private Source(DatanodeInfo node, double avgUtil, double threshold) {
+      super(node, avgUtil, threshold);
+    }
+    
+    /** Add a node task */
+    private void addNodeTask(NodeTask task) {
+      assert (task.datanode != this) :
+        "Source and target are the same " + datanode.getName();
+      incScheduledSize(task.getSize());
+      nodeTasks.add(task);
+    }
+    
+    /* Return an iterator to this source's blocks */
+    private Iterator<BalancerBlock> getBlockIterator() {
+      return srcBlockList.iterator();
+    }
+    
+    /* fetch new blocks of this source from namenode and
+     * update this source's block list & the global block list
+     * Return the total size of the received blocks in the number of bytes.
+     */
+    private long getBlockList() throws IOException {
+      BlockWithLocations[] newBlocks = namenode.getBlocks(datanode, 
+        Math.min(MAX_BLOCKS_SIZE_TO_FETCH, blocksToReceive)).getBlocks();
+      long bytesReceived = 0;
+      for (BlockWithLocations blk : newBlocks) {
+        bytesReceived += blk.getBlock().getNumBytes();
+        BalancerBlock block;
+        synchronized(globalBlockList) {
+          block = globalBlockList.get(blk.getBlock());
+          if (block==null) {
+            block = new BalancerBlock(blk.getBlock());
+            globalBlockList.put(blk.getBlock(), block);
+          } else {
+            block.clearLocations();
+          }
+        
+          synchronized (block) {
+            // update locations
+            for ( String location : blk.getDatanodes() ) {
+              BalancerDatanode datanode = datanodes.get(location);
+              if (datanode != null) { // not an unknown datanode
+                block.addLocation(datanode);
+              }
+            }
+          }
+          if (!srcBlockList.contains(block) && isGoodBlockCandidate(block)) {
+            // filter bad candidates
+            srcBlockList.add(block);
+          }
+        }
+      }
+      return bytesReceived;
+    }
+
+    /* Decide if the given block is a good candidate to move or not */
+    private boolean isGoodBlockCandidate(BalancerBlock block) {
+      for (NodeTask nodeTask : nodeTasks) {
+        if (Balancer.this.isGoodBlockCandidate(this, nodeTask.datanode, block)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    /* Return a block that's good for the source thread to dispatch immediately
+     * The block's source, target, and proxy source are determined too.
+     * When choosing proxy and target, source & target throttling
+     * has been considered. They are chosen only when they have the capacity
+     * to support this block move.
+     * The block should be dispatched immediately after this method is returned.
+     */
+    private PendingBlockMove chooseNextBlockToMove() {
+      for ( Iterator<NodeTask> tasks=nodeTasks.iterator(); tasks.hasNext(); ) {
+        NodeTask task = tasks.next();
+        BalancerDatanode target = task.getDatanode();
+        PendingBlockMove pendingBlock = new PendingBlockMove();
+        if ( target.addPendingBlock(pendingBlock) ) { 
+          // target is not busy, so do a tentative block allocation
+          pendingBlock.source = this;
+          pendingBlock.target = target;
+          if ( pendingBlock.chooseBlockAndProxy() ) {
+            long blockSize = pendingBlock.block.getNumBytes(); 
+            scheduledSize -= blockSize;
+            task.size -= blockSize;
+            if (task.size == 0) {
+              tasks.remove();
+            }
+            return pendingBlock;
+          } else {
+            // cancel the tentative move
+            target.removePendingBlock(pendingBlock);
+          }
+        }
+      }
+      return null;
+    }
+
+    /* iterate all source's blocks to remove moved ones */    
+    private void filterMovedBlocks() {
+      for (Iterator<BalancerBlock> blocks=getBlockIterator();
+            blocks.hasNext();) {
+        if (movedBlocks.contains(blocks.next())) {
+          blocks.remove();
+        }
+      }
+    }
+    
+    private static final int SOURCE_BLOCK_LIST_MIN_SIZE=5;
+    /* Return if should fetch more blocks from namenode */
+    private boolean shouldFetchMoreBlocks() {
+      return srcBlockList.size()<SOURCE_BLOCK_LIST_MIN_SIZE &&
+                 blocksToReceive>0;
+    }
+    
+    /* This method iteratively does the following:
+     * it first selects a block to move,
+     * then sends a request to the proxy source to start the block move
+     * when the source's block list falls below a threshold, it asks
+     * the namenode for more blocks.
+     * It terminates when it has dispatch enough block move tasks or
+     * it has received enough blocks from the namenode, or 
+     * the elapsed time of the iteration has exceeded the max time limit.
+     */ 
+    private static final long MAX_ITERATION_TIME = 20*60*1000L; //20 mins
+    private void dispatchBlocks() {
+      long startTime = Util.now();
+      this.blocksToReceive = 2*scheduledSize;
+      boolean isTimeUp = false;
+      while(!isTimeUp && scheduledSize>0 &&
+          (!srcBlockList.isEmpty() || blocksToReceive>0)) {
+        PendingBlockMove pendingBlock = chooseNextBlockToMove();
+        if (pendingBlock != null) {
+          // move the block
+          pendingBlock.scheduleBlockMove();
+          continue;
+        }
+        
+        /* Since we can not schedule any block to move,
+         * filter any moved blocks from the source block list and
+         * check if we should fetch more blocks from the namenode
+         */
+        filterMovedBlocks(); // filter already moved blocks
+        if (shouldFetchMoreBlocks()) {
+          // fetch new blocks
+          try {
+            blocksToReceive -= getBlockList();
+            continue;
+          } catch (IOException e) {
+            LOG.warn(StringUtils.stringifyException(e));
+            return;
+          }
+        } 
+        
+        // check if time is up or not
+        if (Util.now()-startTime > MAX_ITERATION_TIME) {
+          isTimeUp = true;
+          continue;
+        }
+        
+        /* Now we can not schedule any block to move and there are
+         * no new blocks added to the source block list, so we wait. 
+         */
+        try {
+          synchronized(Balancer.this) {
+            Balancer.this.wait(1000);  // wait for targets/sources to be idle
+          }
+        } catch (InterruptedException ignored) {
+        }
+      }
+    }
+  }
+  
+  /** Default constructor */
+  Balancer() {
+  }
+  
+  /** Construct a balancer from the given configuration */
+  Balancer(Configuration conf) {
+    setConf(conf);
+  } 
+
+  /** Construct a balancer from the given configuration and threshold */
+  Balancer(Configuration conf, double threshold) {
+    setConf(conf);
+    this.threshold = threshold;
+  }
+
+  /**
+   * Run a balancer
+   * @param args
+   */
+  public static void main(String[] args) {
+    try {
+      System.exit( ToolRunner.run(null, new Balancer(), args) );
+    } catch (Throwable e) {
+      LOG.error(StringUtils.stringifyException(e));
+      System.exit(-1);
+    }
+
+  }
+
+  private static void printUsage() {
+    System.out.println("Usage: java Balancer");
+    System.out.println("          [-threshold <threshold>]\t" 
+        +"percentage of disk capacity");
+  }
+
+  /* parse argument to get the threshold */
+  private double parseArgs(String[] args) {
+    double threshold=0;
+    int argsLen = (args == null) ? 0 : args.length;
+    if (argsLen==0) {
+      threshold = 10;
+    } else {
+      if (argsLen != 2 || !"-threshold".equalsIgnoreCase(args[0])) {
+        printUsage();
+        throw new IllegalArgumentException(Arrays.toString(args));
+      } else {
+        try {
+          threshold = Double.parseDouble(args[1]);
+          if (threshold < 0 || threshold >100) {
+            throw new NumberFormatException();
+          }
+          LOG.info( "Using a threshold of " + threshold );
+        } catch(NumberFormatException e) {
+          System.err.println(
+              "Expect a double parameter in the range of [0, 100]: "+ args[1]);
+          printUsage();
+          throw e;
+        }
+      }
+    }
+    return threshold;
+  }
+  
+  /* Initialize balancer. It sets the value of the threshold, and 
+   * builds the communication proxies to
+   * namenode as a client and a secondary namenode and retry proxies
+   * when connection fails.
+   */
+  private void init(double threshold) throws IOException {
+    this.threshold = threshold;
+    this.namenode = createNamenode(conf);
+    this.client = DFSClient.createNamenode(conf);
+    this.fs = FileSystem.get(conf);
+    ExportedAccessKeys keys = namenode.getAccessKeys();
+    this.isAccessTokenEnabled = keys.isAccessTokenEnabled();
+    if (isAccessTokenEnabled) {
+      long accessKeyUpdateInterval = keys.getKeyUpdateInterval();
+      long accessTokenLifetime = keys.getTokenLifetime();
+      LOG.info("Access token params received from NN: keyUpdateInterval="
+          + accessKeyUpdateInterval / (60 * 1000) + " min(s), tokenLifetime="
+          + accessTokenLifetime / (60 * 1000) + " min(s)");
+      this.accessTokenHandler = new AccessTokenHandler(false,
+          accessKeyUpdateInterval, accessTokenLifetime);
+      this.accessTokenHandler.setKeys(keys);
+      /*
+       * Balancer should sync its access keys with NN more frequently than NN
+       * updates its access keys
+       */
+      this.keyUpdaterInterval = accessKeyUpdateInterval / 4;
+      LOG.info("Balancer will update its access keys every "
+          + keyUpdaterInterval / (60 * 1000) + " minute(s)");
+      this.keyupdaterthread = new Daemon(new AccessKeyUpdater());
+      this.shouldRun = true;
+      this.keyupdaterthread.start();
+    }
+  }
+  
+  /**
+   * Periodically updates access keys.
+   */
+  class AccessKeyUpdater implements Runnable {
+
+    public void run() {
+      while (shouldRun) {
+        try {
+          accessTokenHandler.setKeys(namenode.getAccessKeys());
+        } catch (Exception e) {
+          LOG.error(StringUtils.stringifyException(e));
+        }
+        try {
+          Thread.sleep(keyUpdaterInterval);
+        } catch (InterruptedException ie) {
+        }
+      }
+    }
+  }
+  
+  /* Build a NamenodeProtocol connection to the namenode and
+   * set up the retry policy */ 
+  private static NamenodeProtocol createNamenode(Configuration conf)
+    throws IOException {
+    InetSocketAddress nameNodeAddr = NameNode.getAddress(conf);
+    RetryPolicy timeoutPolicy = RetryPolicies.exponentialBackoffRetry(
+        5, 200, TimeUnit.MILLISECONDS);
+    Map<Class<? extends Exception>,RetryPolicy> exceptionToPolicyMap =
+      new HashMap<Class<? extends Exception>, RetryPolicy>();
+    RetryPolicy methodPolicy = RetryPolicies.retryByException(
+        timeoutPolicy, exceptionToPolicyMap);
+    Map<String,RetryPolicy> methodNameToPolicyMap =
+        new HashMap<String, RetryPolicy>();
+    methodNameToPolicyMap.put("getBlocks", methodPolicy);
+    methodNameToPolicyMap.put("getAccessKeys", methodPolicy);
+
+    UserGroupInformation ugi;
+    try {
+      ugi = UnixUserGroupInformation.login(conf);
+    } catch (javax.security.auth.login.LoginException e) {
+      throw new IOException(StringUtils.stringifyException(e));
+    }
+
+    return (NamenodeProtocol) RetryProxy.create(
+        NamenodeProtocol.class,
+        RPC.getProxy(NamenodeProtocol.class,
+            NamenodeProtocol.versionID,
+            nameNodeAddr,
+            ugi,
+            conf,
+            NetUtils.getDefaultSocketFactory(conf)),
+        methodNameToPolicyMap);
+  }
+  
+  /* Shuffle datanode array */
+  static private void shuffleArray(DatanodeInfo[] datanodes) {
+    for (int i=datanodes.length; i>1; i--) {
+      int randomIndex = rnd.nextInt(i);
+      DatanodeInfo tmp = datanodes[randomIndex];
+      datanodes[randomIndex] = datanodes[i-1];
+      datanodes[i-1] = tmp;
+    }
+  }
+  
+  /* get all live datanodes of a cluster and their disk usage
+   * decide the number of bytes need to be moved
+   */
+  private long initNodes() throws IOException {
+    return initNodes(client.getDatanodeReport(DatanodeReportType.LIVE));
+  }
+  
+  /* Given a data node set, build a network topology and decide
+   * over-utilized datanodes, above average utilized datanodes, 
+   * below average utilized datanodes, and underutilized datanodes. 
+   * The input data node set is shuffled before the datanodes 
+   * are put into the over-utilized datanodes, above average utilized
+   * datanodes, below average utilized datanodes, and
+   * underutilized datanodes lists. This will add some randomness
+   * to the node matching later on.
+   * 
+   * @return the total number of bytes that are 
+   *                needed to move to make the cluster balanced.
+   * @param datanodes a set of datanodes
+   */
+  private long initNodes(DatanodeInfo[] datanodes) {
+    // compute average utilization
+    long totalCapacity=0L, totalUsedSpace=0L;
+    for (DatanodeInfo datanode : datanodes) {
+      if (datanode.isDecommissioned() || datanode.isDecommissionInProgress()) {
+        continue; // ignore decommissioning or decommissioned nodes
+      }
+      totalCapacity += datanode.getCapacity();
+      totalUsedSpace += datanode.getDfsUsed();
+    }
+    this.avgUtilization = ((double)totalUsedSpace)/totalCapacity*100;
+
+    /*create network topology and all data node lists: 
+     * overloaded, above-average, below-average, and underloaded
+     * we alternates the accessing of the given datanodes array either by
+     * an increasing order or a decreasing order.
+     */  
+    long overLoadedBytes = 0L, underLoadedBytes = 0L;
+    shuffleArray(datanodes);
+    for (DatanodeInfo datanode : datanodes) {
+      if (datanode.isDecommissioned() || datanode.isDecommissionInProgress()) {
+        continue; // ignore decommissioning or decommissioned nodes
+      }
+      cluster.add(datanode);
+      BalancerDatanode datanodeS;
+      if (getUtilization(datanode) > avgUtilization) {
+        datanodeS = new Source(datanode, avgUtilization, threshold);
+        if (isAboveAvgUtilized(datanodeS)) {
+          this.aboveAvgUtilizedDatanodes.add((Source)datanodeS);
+        } else {
+          assert(isOverUtilized(datanodeS)) :
+            datanodeS.getName()+ "is not an overUtilized node";
+          this.overUtilizedDatanodes.add((Source)datanodeS);
+          overLoadedBytes += (long)((datanodeS.utilization-avgUtilization
+              -threshold)*datanodeS.datanode.getCapacity()/100.0);
+        }
+      } else {
+        datanodeS = new BalancerDatanode(datanode, avgUtilization, threshold);
+        if ( isBelowAvgUtilized(datanodeS)) {
+          this.belowAvgUtilizedDatanodes.add(datanodeS);
+        } else {
+          assert (isUnderUtilized(datanodeS)) :
+            datanodeS.getName()+ "is not an underUtilized node"; 
+          this.underUtilizedDatanodes.add(datanodeS);
+          underLoadedBytes += (long)((avgUtilization-threshold-
+              datanodeS.utilization)*datanodeS.datanode.getCapacity()/100.0);
+        }
+      }
+      this.datanodes.put(datanode.getStorageID(), datanodeS);
+    }
+
+    //logging
+    logImbalancedNodes();
+    
+    assert (this.datanodes.size() == 
+      overUtilizedDatanodes.size()+underUtilizedDatanodes.size()+
+      aboveAvgUtilizedDatanodes.size()+belowAvgUtilizedDatanodes.size())
+      : "Mismatched number of datanodes";
+    
+    // return number of bytes to be moved in order to make the cluster balanced
+    return Math.max(overLoadedBytes, underLoadedBytes);
+  }
+
+  /* log the over utilized & under utilized nodes */
+  private void logImbalancedNodes() {
+    StringBuilder msg = new StringBuilder();
+    msg.append(overUtilizedDatanodes.size());
+    msg.append(" over utilized nodes:");
+    for (Source node : overUtilizedDatanodes) {
+      msg.append( " " );
+      msg.append( node.getName() );
+    }
+    LOG.info(msg);
+    msg = new StringBuilder();
+    msg.append(underUtilizedDatanodes.size());
+    msg.append(" under utilized nodes: ");
+    for (BalancerDatanode node : underUtilizedDatanodes) {
+      msg.append( " " );
+      msg.append( node.getName() );
+    }
+    LOG.info(msg);
+  }
+  
+  /* Decide all <source, target> pairs and
+   * the number of bytes to move from a source to a target
+   * Maximum bytes to be moved per node is
+   * Min(1 Band worth of bytes,  MAX_SIZE_TO_MOVE).
+   * Return total number of bytes to move in this iteration
+   */
+  private long chooseNodes() {
+    // Match nodes on the same rack first
+    chooseNodes(true);
+    // Then match nodes on different racks
+    chooseNodes(false);
+    
+    assert (datanodes.size() == 
+      overUtilizedDatanodes.size()+underUtilizedDatanodes.size()+
+      aboveAvgUtilizedDatanodes.size()+belowAvgUtilizedDatanodes.size()+
+      sources.size()+targets.size())
+      : "Mismatched number of datanodes";
+
+    long bytesToMove = 0L;
+    for (Source src : sources) {
+      bytesToMove += src.scheduledSize;
+    }
+    return bytesToMove;
+  }
+
+  /* if onRack is true, decide all <source, target> pairs
+   * where source and target are on the same rack; Otherwise
+   * decide all <source, target> pairs where source and target are
+   * on different racks
+   */
+  private void chooseNodes(boolean onRack) {
+    /* first step: match each overUtilized datanode (source) to
+     * one or more underUtilized datanodes (targets).
+     */
+    chooseTargets(underUtilizedDatanodes.iterator(), onRack);
+    
+    /* match each remaining overutilized datanode (source) to 
+     * below average utilized datanodes (targets).
+     * Note only overutilized datanodes that haven't had that max bytes to move
+     * satisfied in step 1 are selected
+     */
+    chooseTargets(belowAvgUtilizedDatanodes.iterator(), onRack);
+
+    /* match each remaining underutilized datanode to 
+     * above average utilized datanodes.
+     * Note only underutilized datanodes that have not had that max bytes to
+     * move satisfied in step 1 are selected.
+     */
+    chooseSources(aboveAvgUtilizedDatanodes.iterator(), onRack);
+  }
+   
+  /* choose targets from the target candidate list for each over utilized
+   * source datanode. OnRackTarget determines if the chosen target 
+   * should be on the same rack as the source
+   */
+  private void chooseTargets(  
+      Iterator<BalancerDatanode> targetCandidates, boolean onRackTarget ) {
+    for (Iterator<Source> srcIterator = overUtilizedDatanodes.iterator();
+        srcIterator.hasNext();) {
+      Source source = srcIterator.next();
+      while (chooseTarget(source, targetCandidates, onRackTarget)) {
+      }
+      if (!source.isMoveQuotaFull()) {
+        srcIterator.remove();
+      }
+    }
+    return;
+  }
+  
+  /* choose sources from the source candidate list for each under utilized
+   * target datanode. onRackSource determines if the chosen source 
+   * should be on the same rack as the target
+   */
+  private void chooseSources(
+      Iterator<Source> sourceCandidates, boolean onRackSource) {
+    for (Iterator<BalancerDatanode> targetIterator = 
+      underUtilizedDatanodes.iterator(); targetIterator.hasNext();) {
+      BalancerDatanode target = targetIterator.next();
+      while (chooseSource(target, sourceCandidates, onRackSource)) {
+      }
+      if (!target.isMoveQuotaFull()) {
+        targetIterator.remove();
+      }
+    }
+    return;
+  }
+
+  /* For the given source, choose targets from the target candidate list.
+   * OnRackTarget determines if the chosen target 
+   * should be on the same rack as the source
+   */
+  private boolean chooseTarget(Source source,
+      Iterator<BalancerDatanode> targetCandidates, boolean onRackTarget) {
+    if (!source.isMoveQuotaFull()) {
+      return false;
+    }
+    boolean foundTarget = false;
+    BalancerDatanode target = null;
+    while (!foundTarget && targetCandidates.hasNext()) {
+      target = targetCandidates.next();
+      if (!target.isMoveQuotaFull()) {
+        targetCandidates.remove();
+        continue;
+      }
+      if (onRackTarget) {
+        // choose from on-rack nodes
+        if (cluster.isOnSameRack(source.datanode, target.datanode)) {
+          foundTarget = true;
+        }
+      } else {
+        // choose from off-rack nodes
+        if (!cluster.isOnSameRack(source.datanode, target.datanode)) {
+          foundTarget = true;
+        }
+      }
+    }
+    if (foundTarget) {
+      assert(target != null):"Choose a null target";
+      long size = Math.min(source.availableSizeToMove(),
+          target.availableSizeToMove());
+      NodeTask nodeTask = new NodeTask(target, size);
+      source.addNodeTask(nodeTask);
+      target.incScheduledSize(nodeTask.getSize());
+      sources.add(source);
+      targets.add(target);
+      if (!target.isMoveQuotaFull()) {
+        targetCandidates.remove();
+      }
+      LOG.info("Decided to move "+StringUtils.byteDesc(size)+" bytes from "
+          +source.datanode.getName() + " to " + target.datanode.getName());
+      return true;
+    }
+    return false;
+  }
+  
+  /* For the given target, choose sources from the source candidate list.
+   * OnRackSource determines if the chosen source 
+   * should be on the same rack as the target
+   */
+  private boolean chooseSource(BalancerDatanode target,
+      Iterator<Source> sourceCandidates, boolean onRackSource) {
+    if (!target.isMoveQuotaFull()) {
+      return false;
+    }
+    boolean foundSource = false;
+    Source source = null;
+    while (!foundSource && sourceCandidates.hasNext()) {
+      source = sourceCandidates.next();
+      if (!source.isMoveQuotaFull()) {
+        sourceCandidates.remove();
+        continue;
+      }
+      if (onRackSource) {
+        // choose from on-rack nodes
+        if ( cluster.isOnSameRack(source.getDatanode(), target.getDatanode())) {
+          foundSource = true;
+        }
+      } else {
+        // choose from off-rack nodes
+        if (!cluster.isOnSameRack(source.datanode, target.datanode)) {
+          foundSource = true;
+        }
+      }
+    }
+    if (foundSource) {
+      assert(source != null):"Choose a null source";
+      long size = Math.min(source.availableSizeToMove(),
+          target.availableSizeToMove());
+      NodeTask nodeTask = new NodeTask(target, size);
+      source.addNodeTask(nodeTask);
+      target.incScheduledSize(nodeTask.getSize());
+      sources.add(source);
+      targets.add(target);
+      if ( !source.isMoveQuotaFull()) {
+        sourceCandidates.remove();
+      }
+      LOG.info("Decided to move "+StringUtils.byteDesc(size)+" bytes from "
+          +source.datanode.getName() + " to " + target.datanode.getName());
+      return true;
+    }
+    return false;
+  }
+
+  private static class BytesMoved {
+    private long bytesMoved = 0L;;
+    private synchronized void inc( long bytes ) {
+      bytesMoved += bytes;
+    }
+
+    private long get() {
+      return bytesMoved;
+    }
+  };
+  private BytesMoved bytesMoved = new BytesMoved();
+  private int notChangedIterations = 0;
+  
+  /* Start a thread to dispatch block moves for each source. 
+   * The thread selects blocks to move & sends request to proxy source to
+   * initiate block move. The process is flow controlled. Block selection is
+   * blocked if there are too many un-confirmed block moves.
+   * Return the total number of bytes successfully moved in this iteration.
+   */
+  private long dispatchBlockMoves() throws InterruptedException {
+    long bytesLastMoved = bytesMoved.get();
+    Future<?>[] futures = new Future<?>[sources.size()];
+    int i=0;
+    for (Source source : sources) {
+      futures[i++] = dispatcherExecutor.submit(source.new BlockMoveDispatcher());
+    }
+    
+    // wait for all dispatcher threads to finish
+    for (Future<?> future : futures) {
+      try {
+        future.get();
+      } catch (ExecutionException e) {
+        LOG.warn("Dispatcher thread failed", e.getCause());
+      }
+    }
+    
+    // wait for all block moving to be done
+    waitForMoveCompletion();
+    
+    return bytesMoved.get()-bytesLastMoved;
+  }
+  
+  // The sleeping period before checking if block move is completed again
+  static private long blockMoveWaitTime = 30000L;
+  
+  /** set the sleeping period for block move completion check */
+  static void setBlockMoveWaitTime(long time) {
+    blockMoveWaitTime = time;
+  }
+  
+  /* wait for all block move confirmations 
+   * by checking each target's pendingMove queue 
+   */
+  private void waitForMoveCompletion() {
+    boolean shouldWait;
+    do {
+      shouldWait = false;
+      for (BalancerDatanode target : targets) {
+        if (!target.isPendingQEmpty()) {
+          shouldWait = true;
+        }
+      }
+      if (shouldWait) {
+        try {
+          Thread.sleep(blockMoveWaitTime);
+        } catch (InterruptedException ignored) {
+        }
+      }
+    } while (shouldWait);
+  }
+
+  /** This window makes sure to keep blocks that have been moved within 1.5 hour.
+   * Old window has blocks that are older;
+   * Current window has blocks that are more recent;
+   * Cleanup method triggers the check if blocks in the old window are
+   * more than 1.5 hour old. If yes, purge the old window and then
+   * move blocks in current window to old window.
+   */ 
+  private static class MovedBlocks {
+    private long lastCleanupTime = System.currentTimeMillis();
+    private static long winWidth = 5400*1000L; // 1.5 hour
+    final private static int CUR_WIN = 0;
+    final private static int OLD_WIN = 1;
+    final private static int NUM_WINS = 2;
+    final private List<HashMap<Block, BalancerBlock>> movedBlocks = 
+      new ArrayList<HashMap<Block, BalancerBlock>>(NUM_WINS);
+    
+    /* initialize the moved blocks collection */
+    private MovedBlocks() {
+      movedBlocks.add(new HashMap<Block,BalancerBlock>());
+      movedBlocks.add(new HashMap<Block,BalancerBlock>());
+    }
+
+    /* set the win width */
+    private void setWinWidth(Configuration conf) {
+      winWidth = conf.getLong(
+          "dfs.balancer.movedWinWidth", 5400*1000L);
+    }
+    
+    /* add a block thus marking a block to be moved */
+    synchronized private void add(BalancerBlock block) {
+      movedBlocks.get(CUR_WIN).put(block.getBlock(), block);
+    }
+
+    /* check if a block is marked as moved */
+    synchronized private boolean contains(BalancerBlock block) {
+      return contains(block.getBlock());
+    }
+
+    /* check if a block is marked as moved */
+    synchronized private boolean contains(Block block) {
+      return movedBlocks.get(CUR_WIN).containsKey(block) ||
+        movedBlocks.get(OLD_WIN).containsKey(block);
+    }
+
+    /* remove old blocks */
+    synchronized private void cleanup() {
+      long curTime = System.currentTimeMillis();
+      // check if old win is older than winWidth
+      if (lastCleanupTime + winWidth <= curTime) {
+        // purge the old window
+        movedBlocks.set(OLD_WIN, movedBlocks.get(CUR_WIN));
+        movedBlocks.set(CUR_WIN, new HashMap<Block, BalancerBlock>());
+        lastCleanupTime = curTime;
+      }
+    }
+  }
+
+  /* Decide if it is OK to move the given block from source to target
+   * A block is a good candidate if
+   * 1. the block is not in the process of being moved/has not been moved;
+   * 2. the block does not have a replica on the target;
+   * 3. doing the move does not reduce the number of racks that the block has
+   */
+  private boolean isGoodBlockCandidate(Source source, 
+      BalancerDatanode target, BalancerBlock block) {
+    // check if the block is moved or not
+    if (movedBlocks.contains(block)) {
+        return false;
+    }
+    if (block.isLocatedOnDatanode(target)) {
+      return false;
+    }
+
+    boolean goodBlock = false;
+    if (cluster.isOnSameRack(source.getDatanode(), target.getDatanode())) {
+      // good if source and target are on the same rack
+      goodBlock = true;
+    } else {
+      boolean notOnSameRack = true;
+      synchronized (block) {
+        for (BalancerDatanode loc : block.locations) {
+          if (cluster.isOnSameRack(loc.datanode, target.datanode)) {
+            notOnSameRack = false;
+            break;
+          }
+        }
+      }
+      if (notOnSameRack) {
+        // good if target is target is not on the same rack as any replica
+        goodBlock = true;
+      } else {
+        // good if source is on the same rack as on of the replicas
+        for (BalancerDatanode loc : block.locations) {
+          if (loc != source && 
+              cluster.isOnSameRack(loc.datanode, source.datanode)) {
+            goodBlock = true;
+            break;
+          }
+        }
+      }
+    }
+    return goodBlock;
+  }
+  
+  /* reset all fields in a balancer preparing for the next iteration */
+  private void resetData() {
+    this.cluster = new NetworkTopology();
+    this.overUtilizedDatanodes.clear();
+    this.aboveAvgUtilizedDatanodes.clear();
+    this.belowAvgUtilizedDatanodes.clear();
+    this.underUtilizedDatanodes.clear();
+    this.datanodes.clear();
+    this.sources.clear();
+    this.targets.clear();  
+    this.avgUtilization = 0.0D;
+    cleanGlobalBlockList();
+    this.movedBlocks.cleanup();
+  }
+  
+  /* Remove all blocks from the global block list except for the ones in the
+   * moved list.
+   */
+  private void cleanGlobalBlockList() {
+    for (Iterator<Block> globalBlockListIterator=globalBlockList.keySet().iterator();
+    globalBlockListIterator.hasNext();) {
+      Block block = globalBlockListIterator.next();
+      if(!movedBlocks.contains(block)) {
+        globalBlockListIterator.remove();
+      }
+    }
+  }
+  
+  /* Return true if the given datanode is overUtilized */
+  private boolean isOverUtilized(BalancerDatanode datanode) {
+    return datanode.utilization > (avgUtilization+threshold);
+  }
+  
+  /* Return true if the given datanode is above average utilized
+   * but not overUtilized */
+  private boolean isAboveAvgUtilized(BalancerDatanode datanode) {
+    return (datanode.utilization <= (avgUtilization+threshold))
+        && (datanode.utilization > avgUtilization);
+  }
+  
+  /* Return true if the given datanode is underUtilized */
+  private boolean isUnderUtilized(BalancerDatanode datanode) {
+    return datanode.utilization < (avgUtilization-threshold);
+  }
+
+  /* Return true if the given datanode is below average utilized 
+   * but not underUtilized */
+  private boolean isBelowAvgUtilized(BalancerDatanode datanode) {
+        return (datanode.utilization >= (avgUtilization-threshold))
+                 && (datanode.utilization < avgUtilization);
+  }
+
+  // Exit status
+  final public static int SUCCESS = 1;
+  final public static int ALREADY_RUNNING = -1;
+  final public static int NO_MOVE_BLOCK = -2;
+  final public static int NO_MOVE_PROGRESS = -3;
+  final public static int IO_EXCEPTION = -4;
+  final public static int ILLEGAL_ARGS = -5;
+  /** main method of Balancer
+   * @param args arguments to a Balancer
+   * @exception any exception occurs during datanode balancing
+   */
+  public int run(String[] args) throws Exception {
+    long startTime = Util.now();
+    OutputStream out = null;
+    try {
+      // initialize a balancer
+      init(parseArgs(args));
+      
+      /* Check if there is another balancer running.
+       * Exit if there is another one running.
+       */
+      out = checkAndMarkRunningBalancer(); 
+      if (out == null) {
+        System.out.println("Another balancer is running. Exiting...");
+        return ALREADY_RUNNING;
+      }
+
+      Formatter formatter = new Formatter(System.out);
+      System.out.println("Time Stamp               Iteration#  Bytes Already Moved  Bytes Left To Move  Bytes Being Moved");
+      int iterations = 0;
+      while (true ) {
+        /* get all live datanodes of a cluster and their disk usage
+         * decide the number of bytes need to be moved
+         */
+        long bytesLeftToMove = initNodes();
+        if (bytesLeftToMove == 0) {
+          System.out.println("The cluster is balanced. Exiting...");
+          return SUCCESS;
+        } else {
+          LOG.info( "Need to move "+ StringUtils.byteDesc(bytesLeftToMove)
+              +" bytes to make the cluster balanced." );
+        }
+        
+        /* Decide all the nodes that will participate in the block move and
+         * the number of bytes that need to be moved from one node to another
+         * in this iteration. Maximum bytes to be moved per node is
+         * Min(1 Band worth of bytes,  MAX_SIZE_TO_MOVE).
+         */
+        long bytesToMove = chooseNodes();
+        if (bytesToMove == 0) {
+          System.out.println("No block can be moved. Exiting...");
+          return NO_MOVE_BLOCK;
+        } else {
+          LOG.info( "Will move " + StringUtils.byteDesc(bytesToMove) +
+              "bytes in this iteration");
+        }
+   
+        formatter.format("%-24s %10d  %19s  %18s  %17s\n", 
+            DateFormat.getDateTimeInstance().format(new Date()),
+            iterations,
+            StringUtils.byteDesc(bytesMoved.get()),
+            StringUtils.byteDesc(bytesLeftToMove),
+            StringUtils.byteDesc(bytesToMove)
+            );
+        
+        /* For each pair of <source, target>, start a thread that repeatedly 
+         * decide a block to be moved and its proxy source, 
+         * then initiates the move until all bytes are moved or no more block
+         * available to move.
+         * Exit no byte has been moved for 5 consecutive iterations.
+         */
+        if (dispatchBlockMoves() > 0) {
+          notChangedIterations = 0;
+        } else {
+          notChangedIterations++;
+          if (notChangedIterations >= 5) {
+            System.out.println(
+                "No block has been moved for 5 iterations. Exiting...");
+            return NO_MOVE_PROGRESS;
+          }
+        }
+
+        // clean all lists
+        resetData();
+        
+        try {
+          Thread.sleep(2*conf.getLong("dfs.heartbeat.interval", 3));
+        } catch (InterruptedException ignored) {
+        }
+        
+        iterations++;
+      }
+    } catch (IllegalArgumentException ae) {
+      return ILLEGAL_ARGS;
+    } catch (IOException e) {
+      System.out.println("Received an IO exception: " + e.getMessage() +
+          " . Exiting...");
+      return IO_EXCEPTION;
+    } finally {
+      // shutdown thread pools
+      dispatcherExecutor.shutdownNow();
+      moverExecutor.shutdownNow();
+
+      shouldRun = false;
+      try {
+        if (keyupdaterthread != null) keyupdaterthread.interrupt();
+      } catch (Exception e) {
+        LOG.warn("Exception shutting down access key updater thread", e);
+      }
+      // close the output file
+      IOUtils.closeStream(out); 
+      if (fs != null) {
+        try {
+          fs.delete(BALANCER_ID_PATH, true);
+        } catch(IOException ignored) {
+        }
+      }
+      System.out.println("Balancing took " + 
+          time2Str(Util.now()-startTime));
+    }
+  }
+
+  private Path BALANCER_ID_PATH = new Path("/system/balancer.id");
+  /* The idea for making sure that there is no more than one balancer
+   * running in an HDFS is to create a file in the HDFS, writes the IP address
+   * of the machine on which the balancer is running to the file, but did not
+   * close the file until the balancer exits. 
+   * This prevents the second balancer from running because it can not
+   * creates the file while the first one is running.
+   * 
+   * This method checks if there is any running balancer and 
+   * if no, mark yes if no.
+   * Note that this is an atomic operation.
+   * 
+   * Return null if there is a running balancer; otherwise the output stream
+   * to the newly created file.
+   */
+  private OutputStream checkAndMarkRunningBalancer() throws IOException {
+    try {
+      DataOutputStream out = fs.create(BALANCER_ID_PATH);
+      out. writeBytes(InetAddress.getLocalHost().getHostName());
+      out.flush();
+      return out;
+    } catch(RemoteException e) {
+      if(AlreadyBeingCreatedException.class.getName().equals(e.getClassName())){
+        return null;
+      } else {
+        throw e;
+      }
+    }
+  }
+  
+  /* Given elaspedTime in ms, return a printable string */
+  private static String time2Str(long elapsedTime) {
+    String unit;
+    double time = elapsedTime;
+    if (elapsedTime < 1000) {
+      unit = "milliseconds";
+    } else if (elapsedTime < 60*1000) {
+      unit = "seconds";
+      time = time/1000;
+    } else if (elapsedTime < 3600*1000) {
+      unit = "minutes";
+      time = time/(60*1000);
+    } else {
+      unit = "hours";
+      time = time/(3600*1000);
+    }
+
+    return time+" "+unit;
+  }
+
+  /** return this balancer's configuration */
+  public Configuration getConf() {
+    return conf;
+  }
+
+  /** set this balancer's configuration */
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    movedBlocks.setWinWidth(conf);
+  }
+
+}

+ 114 - 0
src/java/org/apache/hadoop/hdfs/server/common/GenerationStamp.java

@@ -0,0 +1,114 @@
+/**
+ * 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.common;
+
+import java.io.*;
+import org.apache.hadoop.io.*;
+
+/****************************************************************
+ * A GenerationStamp is a Hadoop FS primitive, identified by a long.
+ ****************************************************************/
+public class GenerationStamp implements WritableComparable<GenerationStamp> {
+  public static final long WILDCARD_STAMP = 1;
+  public static final long FIRST_VALID_STAMP = 1000L;
+
+  static {                                      // register a ctor
+    WritableFactories.setFactory
+      (GenerationStamp.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new GenerationStamp(0); }
+       });
+  }
+
+  long genstamp;
+
+  /**
+   * Create a new instance, initialized to FIRST_VALID_STAMP.
+   */
+  public GenerationStamp() {this(GenerationStamp.FIRST_VALID_STAMP);}
+
+  /**
+   * Create a new instance, initialized to the specified value.
+   */
+  GenerationStamp(long stamp) {this.genstamp = stamp;}
+
+  /**
+   * Returns the current generation stamp
+   */
+  public long getStamp() {
+    return this.genstamp;
+  }
+
+  /**
+   * Sets the current generation stamp
+   */
+  public void setStamp(long stamp) {
+    this.genstamp = stamp;
+  }
+
+  /**
+   * First increments the counter and then returns the stamp 
+   */
+  public synchronized long nextStamp() {
+    this.genstamp++;
+    return this.genstamp;
+  }
+
+  /////////////////////////////////////
+  // Writable
+  /////////////////////////////////////
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(genstamp);
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    this.genstamp = in.readLong();
+    if (this.genstamp < 0) {
+      throw new IOException("Bad Generation Stamp: " + this.genstamp);
+    }
+  }
+
+  /////////////////////////////////////
+  // Comparable
+  /////////////////////////////////////
+  public static int compare(long x, long y) {
+    return x < y? -1: x == y? 0: 1;
+  }
+
+  /** {@inheritDoc} */
+  public int compareTo(GenerationStamp that) {
+    return compare(this.genstamp, that.genstamp);
+  }
+
+  /** {@inheritDoc} */
+  public boolean equals(Object o) {
+    if (!(o instanceof GenerationStamp)) {
+      return false;
+    }
+    return genstamp == ((GenerationStamp)o).genstamp;
+  }
+
+  public static boolean equalsWithWildcard(long x, long y) {
+    return x == y || x == WILDCARD_STAMP || y == WILDCARD_STAMP;  
+  }
+
+  /** {@inheritDoc} */
+  public int hashCode() {
+    return 37 * 17 + (int) (genstamp^(genstamp>>>32));
+  }
+}

+ 83 - 0
src/java/org/apache/hadoop/hdfs/server/common/HdfsConstants.java

@@ -0,0 +1,83 @@
+/**
+ * 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.common;
+
+/************************************
+ * Some handy internal HDFS constants
+ *
+ ************************************/
+
+public interface HdfsConstants {
+  /**
+   * Type of the node
+   */
+  static public enum NodeType {
+    NAME_NODE,
+    DATA_NODE;
+  }
+
+  /** Startup options */
+  static public enum StartupOption{
+    FORMAT  ("-format"),
+    REGULAR ("-regular"),
+    BACKUP  ("-backup"),
+    CHECKPOINT("-checkpoint"),
+    UPGRADE ("-upgrade"),
+    ROLLBACK("-rollback"),
+    FINALIZE("-finalize"),
+    IMPORT  ("-importCheckpoint");
+    
+    private String name = null;
+    private StartupOption(String arg) {this.name = arg;}
+    public String getName() {return name;}
+    public NamenodeRole toNodeRole() {
+      switch(this) {
+      case BACKUP: 
+        return NamenodeRole.BACKUP;
+      case CHECKPOINT: 
+        return NamenodeRole.CHECKPOINT;
+      default:
+        return NamenodeRole.ACTIVE;
+      }
+    }
+
+  }
+
+  // Timeouts for communicating with DataNode for streaming writes/reads
+  public static int READ_TIMEOUT = 60 * 1000;
+  public static int WRITE_TIMEOUT = 8 * 60 * 1000;
+  public static int WRITE_TIMEOUT_EXTENSION = 5 * 1000; //for write pipeline
+
+  /**
+   * Defines the NameNode role.
+   */
+  static public enum NamenodeRole {
+    ACTIVE    ("NameNode"),
+    BACKUP    ("Backup Node"),
+    CHECKPOINT("Checkpoint Node"),
+    STANDBY   ("Standby Node");
+
+    private String description = null;
+    private NamenodeRole(String arg) {this.description = arg;}
+  
+    public String toString() {
+      return description;
+    }
+  }
+}
+

+ 47 - 0
src/java/org/apache/hadoop/hdfs/server/common/InconsistentFSStateException.java

@@ -0,0 +1,47 @@
+/**
+ * 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.common;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * The exception is thrown when file system state is inconsistent 
+ * and is not recoverable. 
+ * 
+ */
+public class InconsistentFSStateException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public InconsistentFSStateException(File dir, String descr) {
+    super("Directory " + getFilePath(dir)
+          + " is in an inconsistent state: " + descr);
+  }
+
+  public InconsistentFSStateException(File dir, String descr, Throwable ex) {
+    this(dir, descr + "\n" + StringUtils.stringifyException(ex));
+  }
+  
+  private static String getFilePath(File dir) {
+    try {
+      return dir.getCanonicalPath();
+    } catch(IOException e) {}
+    return dir.getPath();
+  }
+}

+ 44 - 0
src/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java

@@ -0,0 +1,44 @@
+/**
+ * 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.common;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+
+/**
+ * The exception is thrown when external version does not match 
+ * current version of the application.
+ * 
+ */
+public class IncorrectVersionException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public IncorrectVersionException(int versionReported, String ofWhat) {
+    this(versionReported, ofWhat, FSConstants.LAYOUT_VERSION);
+  }
+  
+  public IncorrectVersionException(int versionReported,
+                                   String ofWhat,
+                                   int versionExpected) {
+    super("Unexpected version " 
+          + (ofWhat==null ? "" : "of " + ofWhat) + ". Reported: "
+          + versionReported + ". Expecting = " + versionExpected + ".");
+  }
+
+}

+ 774 - 0
src/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -0,0 +1,774 @@
+/**
+ * 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.common;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileLock;
+import java.nio.channels.OverlappingFileLockException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Iterator;
+import java.util.Properties;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.VersionInfo;
+
+
+
+/**
+ * Storage information file.
+ * <p>
+ * Local storage information is stored in a separate file VERSION.
+ * It contains type of the node, 
+ * the storage layout version, the namespace id, and 
+ * the fs state creation time.
+ * <p>
+ * Local storage can reside in multiple directories. 
+ * Each directory should contain the same VERSION file as the others.
+ * During startup Hadoop servers (name-node and data-nodes) read their local 
+ * storage information from them.
+ * <p>
+ * The servers hold a lock for each storage directory while they run so that 
+ * other nodes were not able to startup sharing the same storage.
+ * The locks are released when the servers stop (normally or abnormally).
+ * 
+ */
+public abstract class Storage extends StorageInfo {
+  public static final Log LOG = LogFactory.getLog(Storage.class.getName());
+
+  // Constants
+  
+  // last layout version that did not suppot upgrades
+  protected static final int LAST_PRE_UPGRADE_LAYOUT_VERSION = -3;
+  
+  // this corresponds to Hadoop-0.14.
+  public static final int LAST_UPGRADABLE_LAYOUT_VERSION = -7;
+  protected static final String LAST_UPGRADABLE_HADOOP_VERSION = "Hadoop-0.14";
+
+  /* this should be removed when LAST_UPGRADABLE_LV goes beyond -13.
+   * any upgrade code that uses this constant should also be removed. */
+  public static final int PRE_GENERATIONSTAMP_LAYOUT_VERSION = -13;
+  
+  private   static final String STORAGE_FILE_LOCK     = "in_use.lock";
+  protected static final String STORAGE_FILE_VERSION  = "VERSION";
+  public static final String STORAGE_DIR_CURRENT   = "current";
+  private   static final String STORAGE_DIR_PREVIOUS  = "previous";
+  private   static final String STORAGE_TMP_REMOVED   = "removed.tmp";
+  private   static final String STORAGE_TMP_PREVIOUS  = "previous.tmp";
+  private   static final String STORAGE_TMP_FINALIZED = "finalized.tmp";
+  private   static final String STORAGE_TMP_LAST_CKPT = "lastcheckpoint.tmp";
+  private   static final String STORAGE_PREVIOUS_CKPT = "previous.checkpoint";
+  
+  public enum StorageState {
+    NON_EXISTENT,
+    NOT_FORMATTED,
+    COMPLETE_UPGRADE,
+    RECOVER_UPGRADE,
+    COMPLETE_FINALIZE,
+    COMPLETE_ROLLBACK,
+    RECOVER_ROLLBACK,
+    COMPLETE_CHECKPOINT,
+    RECOVER_CHECKPOINT,
+    NORMAL;
+  }
+  
+  /**
+   * An interface to denote storage directory type
+   * Implementations can define a type for storage directory by implementing
+   * this interface.
+   */
+  public interface StorageDirType {
+    public StorageDirType getStorageDirType();
+    public boolean isOfType(StorageDirType type);
+  }
+  
+  private NodeType storageType;    // Type of the node using this storage 
+  protected List<StorageDirectory> storageDirs = new ArrayList<StorageDirectory>();
+  
+  private class DirIterator implements Iterator<StorageDirectory> {
+    StorageDirType dirType;
+    int prevIndex; // for remove()
+    int nextIndex; // for next()
+    
+    DirIterator(StorageDirType dirType) {
+      this.dirType = dirType;
+      this.nextIndex = 0;
+      this.prevIndex = 0;
+    }
+    
+    public boolean hasNext() {
+      if (storageDirs.isEmpty() || nextIndex >= storageDirs.size())
+        return false;
+      if (dirType != null) {
+        while (nextIndex < storageDirs.size()) {
+          if (getStorageDir(nextIndex).getStorageDirType().isOfType(dirType))
+            break;
+          nextIndex++;
+        }
+        if (nextIndex >= storageDirs.size())
+         return false;
+      }
+      return true;
+    }
+    
+    public StorageDirectory next() {
+      StorageDirectory sd = getStorageDir(nextIndex);
+      prevIndex = nextIndex;
+      nextIndex++;
+      if (dirType != null) {
+        while (nextIndex < storageDirs.size()) {
+          if (getStorageDir(nextIndex).getStorageDirType().isOfType(dirType))
+            break;
+          nextIndex++;
+        }
+      }
+      return sd;
+    }
+    
+    public void remove() {
+      nextIndex = prevIndex; // restore previous state
+      storageDirs.remove(prevIndex); // remove last returned element
+      hasNext(); // reset nextIndex to correct place
+    }
+  }
+  
+  /**
+   * Return default iterator
+   * This iterator returns all entries in storageDirs
+   */
+  public Iterator<StorageDirectory> dirIterator() {
+    return dirIterator(null);
+  }
+  
+  /**
+   * Return iterator based on Storage Directory Type
+   * This iterator selects entries in storageDirs of type dirType and returns
+   * them via the Iterator
+   */
+  public Iterator<StorageDirectory> dirIterator(StorageDirType dirType) {
+    return new DirIterator(dirType);
+  }
+  
+  /**
+   * generate storage list (debug line)
+   */
+  public String listStorageDirectories() {
+    StringBuffer buf = new StringBuffer();
+    for (StorageDirectory sd : storageDirs) {
+      buf.append(sd.getRoot() + "(" + sd.getStorageDirType() + ");");
+    }
+    return buf.toString();
+  }
+  
+  /**
+   * One of the storage directories.
+   */
+  public class StorageDirectory {
+    File              root; // root directory
+    FileLock          lock; // storage lock
+    StorageDirType dirType; // storage dir type
+    
+    public StorageDirectory(File dir) {
+      // default dirType is null
+      this(dir, null);
+    }
+    
+    public StorageDirectory(File dir, StorageDirType dirType) {
+      this.root = dir;
+      this.lock = null;
+      this.dirType = dirType;
+    }
+    
+    /**
+     * Get root directory of this storage
+     */
+    public File getRoot() {
+      return root;
+    }
+
+    /**
+     * Get storage directory type
+     */
+    public StorageDirType getStorageDirType() {
+      return dirType;
+    }
+    
+    /**
+     * Read version file.
+     * 
+     * @throws IOException if file cannot be read or contains inconsistent data
+     */
+    public void read() throws IOException {
+      read(getVersionFile());
+    }
+    
+    public void read(File from) throws IOException {
+      RandomAccessFile file = new RandomAccessFile(from, "rws");
+      FileInputStream in = null;
+      try {
+        in = new FileInputStream(file.getFD());
+        file.seek(0);
+        Properties props = new Properties();
+        props.load(in);
+        getFields(props, this);
+      } finally {
+        if (in != null) {
+          in.close();
+        }
+        file.close();
+      }
+    }
+
+    /**
+     * Write version file.
+     * 
+     * @throws IOException
+     */
+    public void write() throws IOException {
+      corruptPreUpgradeStorage(root);
+      write(getVersionFile());
+    }
+
+    public void write(File to) throws IOException {
+      Properties props = new Properties();
+      setFields(props, this);
+      RandomAccessFile file = new RandomAccessFile(to, "rws");
+      FileOutputStream out = null;
+      try {
+        file.seek(0);
+        out = new FileOutputStream(file.getFD());
+        /*
+         * If server is interrupted before this line, 
+         * the version file will remain unchanged.
+         */
+        props.store(out, null);
+        /*
+         * Now the new fields are flushed to the head of the file, but file 
+         * length can still be larger then required and therefore the file can 
+         * contain whole or corrupted fields from its old contents in the end.
+         * If server is interrupted here and restarted later these extra fields
+         * either should not effect server behavior or should be handled
+         * by the server correctly.
+         */
+        file.setLength(out.getChannel().position());
+      } finally {
+        if (out != null) {
+          out.close();
+        }
+        file.close();
+      }
+    }
+
+    /**
+     * Clear and re-create storage directory.
+     * <p>
+     * Removes contents of the current directory and creates an empty directory.
+     * 
+     * This does not fully format storage directory. 
+     * It cannot write the version file since it should be written last after  
+     * all other storage type dependent files are written.
+     * Derived storage is responsible for setting specific storage values and
+     * writing the version file to disk.
+     * 
+     * @throws IOException
+     */
+    public void clearDirectory() throws IOException {
+      File curDir = this.getCurrentDir();
+      if (curDir.exists())
+        if (!(FileUtil.fullyDelete(curDir)))
+          throw new IOException("Cannot remove current directory: " + curDir);
+      if (!curDir.mkdirs())
+        throw new IOException("Cannot create directory " + curDir);
+    }
+
+    public File getCurrentDir() {
+      return new File(root, STORAGE_DIR_CURRENT);
+    }
+    public File getVersionFile() {
+      return new File(new File(root, STORAGE_DIR_CURRENT), STORAGE_FILE_VERSION);
+    }
+    public File getPreviousVersionFile() {
+      return new File(new File(root, STORAGE_DIR_PREVIOUS), STORAGE_FILE_VERSION);
+    }
+    public File getPreviousDir() {
+      return new File(root, STORAGE_DIR_PREVIOUS);
+    }
+    public File getPreviousTmp() {
+      return new File(root, STORAGE_TMP_PREVIOUS);
+    }
+    public File getRemovedTmp() {
+      return new File(root, STORAGE_TMP_REMOVED);
+    }
+    public File getFinalizedTmp() {
+      return new File(root, STORAGE_TMP_FINALIZED);
+    }
+    public File getLastCheckpointTmp() {
+      return new File(root, STORAGE_TMP_LAST_CKPT);
+    }
+    public File getPreviousCheckpoint() {
+      return new File(root, STORAGE_PREVIOUS_CKPT);
+    }
+
+    /**
+     * Check consistency of the storage directory
+     * 
+     * @param startOpt a startup option.
+     *  
+     * @return state {@link StorageState} of the storage directory 
+     * @throws {@link InconsistentFSStateException} if directory state is not 
+     * consistent and cannot be recovered 
+     */
+    public StorageState analyzeStorage(StartupOption startOpt) throws IOException {
+      assert root != null : "root is null";
+      String rootPath = root.getCanonicalPath();
+      try { // check that storage exists
+        if (!root.exists()) {
+          // storage directory does not exist
+          if (startOpt != StartupOption.FORMAT) {
+            LOG.info("Storage directory " + rootPath + " does not exist.");
+            return StorageState.NON_EXISTENT;
+          }
+          LOG.info(rootPath + " does not exist. Creating ...");
+          if (!root.mkdirs())
+            throw new IOException("Cannot create directory " + rootPath);
+        }
+        // or is inaccessible
+        if (!root.isDirectory()) {
+          LOG.info(rootPath + "is not a directory.");
+          return StorageState.NON_EXISTENT;
+        }
+        if (!root.canWrite()) {
+          LOG.info("Cannot access storage directory " + rootPath);
+          return StorageState.NON_EXISTENT;
+        }
+      } catch(SecurityException ex) {
+        LOG.info("Cannot access storage directory " + rootPath, ex);
+        return StorageState.NON_EXISTENT;
+      }
+
+      this.lock(); // lock storage if it exists
+
+      if (startOpt == HdfsConstants.StartupOption.FORMAT)
+        return StorageState.NOT_FORMATTED;
+      if (startOpt != HdfsConstants.StartupOption.IMPORT) {
+        //make sure no conversion is required
+        checkConversionNeeded(this);
+      }
+
+      // check whether current directory is valid
+      File versionFile = getVersionFile();
+      boolean hasCurrent = versionFile.exists();
+
+      // check which directories exist
+      boolean hasPrevious = getPreviousDir().exists();
+      boolean hasPreviousTmp = getPreviousTmp().exists();
+      boolean hasRemovedTmp = getRemovedTmp().exists();
+      boolean hasFinalizedTmp = getFinalizedTmp().exists();
+      boolean hasCheckpointTmp = getLastCheckpointTmp().exists();
+
+      if (!(hasPreviousTmp || hasRemovedTmp
+          || hasFinalizedTmp || hasCheckpointTmp)) {
+        // no temp dirs - no recovery
+        if (hasCurrent)
+          return StorageState.NORMAL;
+        if (hasPrevious)
+          throw new InconsistentFSStateException(root,
+                              "version file in current directory is missing.");
+        return StorageState.NOT_FORMATTED;
+      }
+
+      if ((hasPreviousTmp?1:0) + (hasRemovedTmp?1:0)
+          + (hasFinalizedTmp?1:0) + (hasCheckpointTmp?1:0) > 1)
+        // more than one temp dirs
+        throw new InconsistentFSStateException(root,
+                                               "too many temporary directories.");
+
+      // # of temp dirs == 1 should either recover or complete a transition
+      if (hasCheckpointTmp) {
+        return hasCurrent ? StorageState.COMPLETE_CHECKPOINT
+                          : StorageState.RECOVER_CHECKPOINT;
+      }
+
+      if (hasFinalizedTmp) {
+        if (hasPrevious)
+          throw new InconsistentFSStateException(root,
+                                                 STORAGE_DIR_PREVIOUS + " and " + STORAGE_TMP_FINALIZED
+                                                 + "cannot exist together.");
+        return StorageState.COMPLETE_FINALIZE;
+      }
+
+      if (hasPreviousTmp) {
+        if (hasPrevious)
+          throw new InconsistentFSStateException(root,
+                                                 STORAGE_DIR_PREVIOUS + " and " + STORAGE_TMP_PREVIOUS
+                                                 + " cannot exist together.");
+        if (hasCurrent)
+          return StorageState.COMPLETE_UPGRADE;
+        return StorageState.RECOVER_UPGRADE;
+      }
+      
+      assert hasRemovedTmp : "hasRemovedTmp must be true";
+      if (!(hasCurrent ^ hasPrevious))
+        throw new InconsistentFSStateException(root,
+                                               "one and only one directory " + STORAGE_DIR_CURRENT 
+                                               + " or " + STORAGE_DIR_PREVIOUS 
+                                               + " must be present when " + STORAGE_TMP_REMOVED
+                                               + " exists.");
+      if (hasCurrent)
+        return StorageState.COMPLETE_ROLLBACK;
+      return StorageState.RECOVER_ROLLBACK;
+    }
+
+    /**
+     * Complete or recover storage state from previously failed transition.
+     * 
+     * @param curState specifies what/how the state should be recovered
+     * @throws IOException
+     */
+    public void doRecover(StorageState curState) throws IOException {
+      File curDir = getCurrentDir();
+      String rootPath = root.getCanonicalPath();
+      switch(curState) {
+      case COMPLETE_UPGRADE:  // mv previous.tmp -> previous
+        LOG.info("Completing previous upgrade for storage directory " 
+                 + rootPath + ".");
+        rename(getPreviousTmp(), getPreviousDir());
+        return;
+      case RECOVER_UPGRADE:   // mv previous.tmp -> current
+        LOG.info("Recovering storage directory " + rootPath
+                 + " from previous upgrade.");
+        if (curDir.exists())
+          deleteDir(curDir);
+        rename(getPreviousTmp(), curDir);
+        return;
+      case COMPLETE_ROLLBACK: // rm removed.tmp
+        LOG.info("Completing previous rollback for storage directory "
+                 + rootPath + ".");
+        deleteDir(getRemovedTmp());
+        return;
+      case RECOVER_ROLLBACK:  // mv removed.tmp -> current
+        LOG.info("Recovering storage directory " + rootPath
+                 + " from previous rollback.");
+        rename(getRemovedTmp(), curDir);
+        return;
+      case COMPLETE_FINALIZE: // rm finalized.tmp
+        LOG.info("Completing previous finalize for storage directory "
+                 + rootPath + ".");
+        deleteDir(getFinalizedTmp());
+        return;
+      case COMPLETE_CHECKPOINT: // mv lastcheckpoint.tmp -> previous.checkpoint
+        LOG.info("Completing previous checkpoint for storage directory " 
+                 + rootPath + ".");
+        File prevCkptDir = getPreviousCheckpoint();
+        if (prevCkptDir.exists())
+          deleteDir(prevCkptDir);
+        rename(getLastCheckpointTmp(), prevCkptDir);
+        return;
+      case RECOVER_CHECKPOINT:  // mv lastcheckpoint.tmp -> current
+        LOG.info("Recovering storage directory " + rootPath
+                 + " from failed checkpoint.");
+        if (curDir.exists())
+          deleteDir(curDir);
+        rename(getLastCheckpointTmp(), curDir);
+        return;
+      default:
+        throw new IOException("Unexpected FS state: " + curState);
+      }
+    }
+
+    /**
+     * Lock storage to provide exclusive access.
+     * 
+     * <p> Locking is not supported by all file systems.
+     * E.g., NFS does not consistently support exclusive locks.
+     * 
+     * <p> If locking is supported we guarantee exculsive access to the
+     * storage directory. Otherwise, no guarantee is given.
+     * 
+     * @throws IOException if locking fails
+     */
+    public void lock() throws IOException {
+      this.lock = tryLock();
+      if (lock == null) {
+        String msg = "Cannot lock storage " + this.root 
+          + ". The directory is already locked.";
+        LOG.info(msg);
+        throw new IOException(msg);
+      }
+    }
+
+    /**
+     * Attempts to acquire an exclusive lock on the storage.
+     * 
+     * @return A lock object representing the newly-acquired lock or
+     * <code>null</code> if storage is already locked.
+     * @throws IOException if locking fails.
+     */
+    FileLock tryLock() throws IOException {
+      File lockF = new File(root, STORAGE_FILE_LOCK);
+      lockF.deleteOnExit();
+      RandomAccessFile file = new RandomAccessFile(lockF, "rws");
+      FileLock res = null;
+      try {
+        res = file.getChannel().tryLock();
+      } catch(OverlappingFileLockException oe) {
+        file.close();
+        return null;
+      } catch(IOException e) {
+        LOG.info(StringUtils.stringifyException(e));
+        file.close();
+        throw e;
+      }
+      return res;
+    }
+
+    /**
+     * Unlock storage.
+     * 
+     * @throws IOException
+     */
+    public void unlock() throws IOException {
+      if (this.lock == null)
+        return;
+      this.lock.release();
+      lock.channel().close();
+      lock = null;
+    }
+  }
+
+  /**
+   * Create empty storage info of the specified type
+   */
+  protected Storage(NodeType type) {
+    super();
+    this.storageType = type;
+  }
+  
+  protected Storage(NodeType type, int nsID, long cT) {
+    super(FSConstants.LAYOUT_VERSION, nsID, cT);
+    this.storageType = type;
+  }
+  
+  protected Storage(NodeType type, StorageInfo storageInfo) {
+    super(storageInfo);
+    this.storageType = type;
+  }
+  
+  public int getNumStorageDirs() {
+    return storageDirs.size();
+  }
+  
+  public StorageDirectory getStorageDir(int idx) {
+    return storageDirs.get(idx);
+  }
+  
+  protected void addStorageDir(StorageDirectory sd) {
+    storageDirs.add(sd);
+  }
+  
+  public abstract boolean isConversionNeeded(StorageDirectory sd) throws IOException;
+
+  /*
+   * Coversion is no longer supported. So this should throw exception if
+   * conversion is needed.
+   */
+  private void checkConversionNeeded(StorageDirectory sd) throws IOException {
+    if (isConversionNeeded(sd)) {
+      //throw an exception
+      checkVersionUpgradable(0);
+    }
+  }
+
+  /**
+   * Checks if the upgrade from the given old version is supported. If
+   * no upgrade is supported, it throws IncorrectVersionException.
+   * 
+   * @param oldVersion
+   */
+  protected static void checkVersionUpgradable(int oldVersion) 
+                                     throws IOException {
+    if (oldVersion > LAST_UPGRADABLE_LAYOUT_VERSION) {
+      String msg = "*********** Upgrade is not supported from this older" +
+                   " version of storage to the current version." + 
+                   " Please upgrade to " + LAST_UPGRADABLE_HADOOP_VERSION +
+                   " or a later version and then upgrade to current" +
+                   " version. Old layout version is " + 
+                   (oldVersion == 0 ? "'too old'" : (""+oldVersion)) +
+                   " and latest layout version this software version can" +
+                   " upgrade from is " + LAST_UPGRADABLE_LAYOUT_VERSION +
+                   ". ************";
+      LOG.error(msg);
+      throw new IOException(msg); 
+    }
+    
+  }
+  
+  /**
+   * Get common storage fields.
+   * Should be overloaded if additional fields need to be get.
+   * 
+   * @param props
+   * @throws IOException
+   */
+  protected void getFields(Properties props, 
+                           StorageDirectory sd 
+                           ) throws IOException {
+    String sv, st, sid, sct;
+    sv = props.getProperty("layoutVersion");
+    st = props.getProperty("storageType");
+    sid = props.getProperty("namespaceID");
+    sct = props.getProperty("cTime");
+    if (sv == null || st == null || sid == null || sct == null)
+      throw new InconsistentFSStateException(sd.root,
+                                             "file " + STORAGE_FILE_VERSION + " is invalid.");
+    int rv = Integer.parseInt(sv);
+    NodeType rt = NodeType.valueOf(st);
+    int rid = Integer.parseInt(sid);
+    long rct = Long.parseLong(sct);
+    if (!storageType.equals(rt) ||
+        !((namespaceID == 0) || (rid == 0) || namespaceID == rid))
+      throw new InconsistentFSStateException(sd.root,
+                                             "is incompatible with others.");
+    if (rv < FSConstants.LAYOUT_VERSION) // future version
+      throw new IncorrectVersionException(rv, "storage directory " 
+                                          + sd.root.getCanonicalPath());
+    layoutVersion = rv;
+    storageType = rt;
+    namespaceID = rid;
+    cTime = rct;
+  }
+  
+  /**
+   * Set common storage fields.
+   * Should be overloaded if additional fields need to be set.
+   * 
+   * @param props
+   * @throws IOException
+   */
+  protected void setFields(Properties props, 
+                           StorageDirectory sd 
+                           ) throws IOException {
+    props.setProperty("layoutVersion", String.valueOf(layoutVersion));
+    props.setProperty("storageType", storageType.toString());
+    props.setProperty("namespaceID", String.valueOf(namespaceID));
+    props.setProperty("cTime", String.valueOf(cTime));
+  }
+
+  public static void rename(File from, File to) throws IOException {
+    if (!from.renameTo(to))
+      throw new IOException("Failed to rename " 
+                            + from.getCanonicalPath() + " to " + to.getCanonicalPath());
+  }
+
+  protected static void deleteDir(File dir) throws IOException {
+    if (!FileUtil.fullyDelete(dir))
+      throw new IOException("Failed to delete " + dir.getCanonicalPath());
+  }
+  
+  /**
+   * Write all data storage files.
+   * @throws IOException
+   */
+  public void writeAll() throws IOException {
+    this.layoutVersion = FSConstants.LAYOUT_VERSION;
+    for (Iterator<StorageDirectory> it = storageDirs.iterator(); it.hasNext();) {
+      it.next().write();
+    }
+  }
+
+  /**
+   * Unlock all storage directories.
+   * @throws IOException
+   */
+  public void unlockAll() throws IOException {
+    for (Iterator<StorageDirectory> it = storageDirs.iterator(); it.hasNext();) {
+      it.next().unlock();
+    }
+  }
+
+  /**
+   * Check whether underlying file system supports file locking.
+   * 
+   * @return <code>true</code> if exclusive locks are supported or
+   *         <code>false</code> otherwise.
+   * @throws IOException
+   * @see StorageDirectory#lock()
+   */
+  public boolean isLockSupported(int idx) throws IOException {
+    StorageDirectory sd = storageDirs.get(idx);
+    FileLock firstLock = null;
+    FileLock secondLock = null;
+    try {
+      firstLock = sd.lock;
+      if(firstLock == null) {
+        firstLock = sd.tryLock();
+        if(firstLock == null)
+          return true;
+      }
+      secondLock = sd.tryLock();
+      if(secondLock == null)
+        return true;
+    } finally {
+      if(firstLock != null && firstLock != sd.lock) {
+        firstLock.release();
+        firstLock.channel().close();
+      }
+      if(secondLock != null) {
+        secondLock.release();
+        secondLock.channel().close();
+      }
+    }
+    return false;
+  }
+
+  public static String getBuildVersion() {
+    return VersionInfo.getRevision();
+  }
+
+  public static String getRegistrationID(StorageInfo storage) {
+    return "NS-" + Integer.toString(storage.getNamespaceID())
+      + "-" + Integer.toString(storage.getLayoutVersion())
+      + "-" + Long.toString(storage.getCTime());
+  }
+
+  // Pre-upgrade version compatibility
+  protected abstract void corruptPreUpgradeStorage(File rootDir) throws IOException;
+
+  protected void writeCorruptedData(RandomAccessFile file) throws IOException {
+    final String messageForPreUpgradeVersion =
+      "\nThis file is INTENTIONALLY CORRUPTED so that versions\n"
+      + "of Hadoop prior to 0.13 (which are incompatible\n"
+      + "with this directory layout) will fail to start.\n";
+  
+    file.seek(0);
+    file.writeInt(FSConstants.LAYOUT_VERSION);
+    org.apache.hadoop.io.DeprecatedUTF8.writeString(file, "");
+    file.writeBytes(messageForPreUpgradeVersion);
+    file.getFD().sync();
+  }
+}

+ 72 - 0
src/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java

@@ -0,0 +1,72 @@
+package org.apache.hadoop.hdfs.server.common;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+
+
+/**
+ * Common class for storage information.
+ * 
+ * TODO namespaceID should be long and computed as hash(address + port)
+ */
+public class StorageInfo implements Writable {
+  public int   layoutVersion;   // layout version of the storage data
+  public int   namespaceID;     // id of the file system
+  public long  cTime;           // creation time of the file system state
+  
+  public StorageInfo () {
+    this(0, 0, 0L);
+  }
+  
+  public StorageInfo(int layoutV, int nsID, long cT) {
+    layoutVersion = layoutV;
+    namespaceID = nsID;
+    cTime = cT;
+  }
+  
+  public StorageInfo(StorageInfo from) {
+    setStorageInfo(from);
+  }
+
+  /**
+   * Layout version of the storage data.
+   */
+  public int    getLayoutVersion(){ return layoutVersion; }
+
+  /**
+   * Namespace id of the file system.<p>
+   * Assigned to the file system at formatting and never changes after that.
+   * Shared by all file system components.
+   */
+  public int    getNamespaceID()  { return namespaceID; }
+
+  /**
+   * Creation time of the file system state.<p>
+   * Modified during upgrades.
+   */
+  public long   getCTime()        { return cTime; }
+
+  public void   setStorageInfo(StorageInfo from) {
+    layoutVersion = from.layoutVersion;
+    namespaceID = from.namespaceID;
+    cTime = from.cTime;
+  }
+
+  /////////////////////////////////////////////////
+  // Writable
+  /////////////////////////////////////////////////
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(getLayoutVersion());
+    out.writeInt(getNamespaceID());
+    out.writeLong(getCTime());
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    layoutVersion = in.readInt();
+    namespaceID = in.readInt();
+    cTime = in.readLong();
+  }
+}

+ 89 - 0
src/java/org/apache/hadoop/hdfs/server/common/UpgradeManager.java

@@ -0,0 +1,89 @@
+/**
+ * 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.common;
+
+import java.io.IOException;
+import java.util.SortedSet;
+
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+
+/**
+ * Generic upgrade manager.
+ * 
+ * {@link #broadcastCommand} is the command that should be 
+ *
+ */
+public abstract class UpgradeManager {
+  protected SortedSet<Upgradeable> currentUpgrades = null;
+  protected boolean upgradeState = false; // true if upgrade is in progress
+  protected int upgradeVersion = 0;
+  protected UpgradeCommand broadcastCommand = null;
+
+  public synchronized UpgradeCommand getBroadcastCommand() {
+    return this.broadcastCommand;
+  }
+
+  public synchronized boolean getUpgradeState() {
+    return this.upgradeState;
+  }
+
+  public synchronized int getUpgradeVersion(){
+    return this.upgradeVersion;
+  }
+
+  public synchronized void setUpgradeState(boolean uState, int uVersion) {
+    this.upgradeState = uState;
+    this.upgradeVersion = uVersion;
+  }
+
+  public SortedSet<Upgradeable> getDistributedUpgrades() throws IOException {
+    return UpgradeObjectCollection.getDistributedUpgrades(
+                                            getUpgradeVersion(), getType());
+  }
+
+  public synchronized short getUpgradeStatus() {
+    if(currentUpgrades == null)
+      return 100;
+    return currentUpgrades.first().getUpgradeStatus();
+  }
+
+  public synchronized boolean initializeUpgrade() throws IOException {
+    currentUpgrades = getDistributedUpgrades();
+    if(currentUpgrades == null) {
+      // set new upgrade state
+      setUpgradeState(false, FSConstants.LAYOUT_VERSION);
+      return false;
+    }
+    Upgradeable curUO = currentUpgrades.first();
+    // set and write new upgrade state into disk
+    setUpgradeState(true, curUO.getVersion());
+    return true;
+  }
+
+  public synchronized boolean isUpgradeCompleted() {
+    if (currentUpgrades == null) {
+      return true;
+    }
+    return false;
+  }
+
+  public abstract HdfsConstants.NodeType getType();
+  public abstract boolean startUpgrade() throws IOException;
+  public abstract void completeUpgrade() throws IOException;
+}

+ 66 - 0
src/java/org/apache/hadoop/hdfs/server/common/UpgradeObject.java

@@ -0,0 +1,66 @@
+/**
+ * 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.common;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.common.UpgradeObjectCollection.UOSignature;
+
+/**
+ * Abstract upgrade object.
+ * 
+ * Contains default implementation of common methods of {@link Upgradeable}
+ * interface.
+ */
+public abstract class UpgradeObject implements Upgradeable {
+  protected short status;
+  
+  public short getUpgradeStatus() {
+    return status;
+  }
+
+  public String getDescription() {
+    return "Upgrade object for " + getType() + " layout version " + getVersion();
+  }
+
+  public UpgradeStatusReport getUpgradeStatusReport(boolean details) 
+                                                    throws IOException {
+    return new UpgradeStatusReport(getVersion(), getUpgradeStatus(), false);
+  }
+
+  public int compareTo(Upgradeable o) {
+    if(this.getVersion() != o.getVersion())
+      return (getVersion() > o.getVersion() ? -1 : 1);
+    int res = this.getType().toString().compareTo(o.getType().toString());
+    if(res != 0)
+      return res;
+    return getClass().getCanonicalName().compareTo(
+                    o.getClass().getCanonicalName());
+  }
+
+  public boolean equals(Object o) {
+    if (!(o instanceof UpgradeObject)) {
+      return false;
+    }
+    return this.compareTo((UpgradeObject)o) == 0;
+  }
+
+  public int hashCode() {
+    return new UOSignature(this).hashCode(); 
+  }
+}

+ 130 - 0
src/java/org/apache/hadoop/hdfs/server/common/UpgradeObjectCollection.java

@@ -0,0 +1,130 @@
+/**
+ * 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.common;
+
+import java.io.IOException;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Collection of upgrade objects.
+ *
+ * Upgrade objects should be registered here before they can be used. 
+ */
+public class UpgradeObjectCollection {
+  static {
+    initialize();
+    // Registered distributed upgrade objects here
+    // registerUpgrade(new UpgradeObject());
+  }
+
+  static class UOSignature implements Comparable<UOSignature> {
+    int version;
+    HdfsConstants.NodeType type;
+    String className;
+
+    UOSignature(Upgradeable uo) {
+      this.version = uo.getVersion();
+      this.type = uo.getType();
+      this.className = uo.getClass().getCanonicalName();
+    }
+
+    int getVersion() {
+      return version;
+    }
+
+    HdfsConstants.NodeType getType() {
+      return type;
+    }
+
+    String getClassName() {
+      return className;
+    }
+
+    Upgradeable instantiate() throws IOException {
+      try {
+        return (Upgradeable)Class.forName(getClassName()).newInstance();
+      } catch(ClassNotFoundException e) {
+        throw new IOException(StringUtils.stringifyException(e));
+      } catch(InstantiationException e) {
+        throw new IOException(StringUtils.stringifyException(e));
+      } catch(IllegalAccessException e) {
+        throw new IOException(StringUtils.stringifyException(e));
+      }
+    }
+
+    public int compareTo(UOSignature o) {
+      if(this.version != o.version)
+        return (version < o.version ? -1 : 1);
+      int res = this.getType().toString().compareTo(o.getType().toString());
+      if(res != 0)
+        return res;
+      return className.compareTo(o.className);
+    }
+
+    public boolean equals(Object o) {
+        if (!(o instanceof UOSignature)) {
+          return false;
+        }
+        return this.compareTo((UOSignature)o) == 0;
+      }
+
+      public int hashCode() {
+        return version ^ ((type==null)?0:type.hashCode()) 
+                       ^ ((className==null)?0:className.hashCode());
+      }
+  }
+
+  /**
+   * Static collection of upgrade objects sorted by version.
+   * Layout versions are negative therefore newer versions will go first.
+   */
+  static SortedSet<UOSignature> upgradeTable;
+
+  static final void initialize() {
+    upgradeTable = new TreeSet<UOSignature>();
+  }
+
+  static void registerUpgrade(Upgradeable uo) {
+    // Registered distributed upgrade objects here
+    upgradeTable.add(new UOSignature(uo));
+  }
+
+  public static SortedSet<Upgradeable> getDistributedUpgrades(int versionFrom, 
+                                                       HdfsConstants.NodeType type
+                                                       ) throws IOException {
+    assert FSConstants.LAYOUT_VERSION <= versionFrom : "Incorrect version " 
+      + versionFrom + ". Expected to be <= " + FSConstants.LAYOUT_VERSION;
+    SortedSet<Upgradeable> upgradeObjects = new TreeSet<Upgradeable>();
+    for(UOSignature sig : upgradeTable) {
+      if(sig.getVersion() < FSConstants.LAYOUT_VERSION)
+        continue;
+      if(sig.getVersion() > versionFrom)
+        break;
+      if(sig.getType() != type )
+        continue;
+      upgradeObjects.add(sig.instantiate());
+    }
+    if(upgradeObjects.size() == 0)
+      return null;
+    return upgradeObjects;
+  }
+}

+ 124 - 0
src/java/org/apache/hadoop/hdfs/server/common/UpgradeStatusReport.java

@@ -0,0 +1,124 @@
+/**
+ * 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.common;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+/**
+ * Base upgrade upgradeStatus class.
+ * Overload this class if specific status fields need to be reported.
+ * 
+ * Describes status of current upgrade.
+ */
+public class UpgradeStatusReport implements Writable {
+  protected int version;
+  protected short upgradeStatus;
+  protected boolean finalized;
+
+  public UpgradeStatusReport() {
+    this.version = 0;
+    this.upgradeStatus = 0;
+    this.finalized = false;
+  }
+
+  public UpgradeStatusReport(int version, short status, boolean isFinalized) {
+    this.version = version;
+    this.upgradeStatus = status;
+    this.finalized = isFinalized;
+  }
+
+  /**
+   * Get the layout version of the currently running upgrade.
+   * @return layout version
+   */
+  public int getVersion() {
+    return this.version;
+  }
+
+  /**
+   * Get upgrade upgradeStatus as a percentage of the total upgrade done.
+   * 
+   * @see Upgradeable#getUpgradeStatus() 
+   */ 
+  public short getUpgradeStatus() {
+    return upgradeStatus;
+  }
+
+  /**
+   * Is current upgrade finalized.
+   * @return true if finalized or false otherwise.
+   */
+  public boolean isFinalized() {
+    return this.finalized;
+  }
+
+  /**
+   * Get upgradeStatus data as a text for reporting.
+   * Should be overloaded for a particular upgrade specific upgradeStatus data.
+   * 
+   * @param details true if upgradeStatus details need to be included, 
+   *                false otherwise
+   * @return text
+   */
+  public String getStatusText(boolean details) {
+    return "Upgrade for version " + getVersion() 
+            + (upgradeStatus<100 ? 
+              " is in progress. Status = " + upgradeStatus + "%" : 
+              " has been completed."
+              + "\nUpgrade is " + (finalized ? "" : "not ")
+              + "finalized.");
+  }
+
+  /**
+   * Print basic upgradeStatus details.
+   */
+  public String toString() {
+    return getStatusText(false);
+  }
+
+  /////////////////////////////////////////////////
+  // Writable
+  /////////////////////////////////////////////////
+  static {                                      // register a ctor
+    WritableFactories.setFactory
+      (UpgradeStatusReport.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new UpgradeStatusReport(); }
+       });
+  }
+
+  /**
+   */
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(this.version);
+    out.writeShort(this.upgradeStatus);
+  }
+
+  /**
+   */
+  public void readFields(DataInput in) throws IOException {
+    this.version = in.readInt();
+    this.upgradeStatus = in.readShort();
+  }
+}

+ 98 - 0
src/java/org/apache/hadoop/hdfs/server/common/Upgradeable.java

@@ -0,0 +1,98 @@
+/**
+ * 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.common;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+
+/**
+ * Common interface for distributed upgrade objects.
+ * 
+ * Each upgrade object corresponds to a layout version,
+ * which is the latest version that should be upgraded using this object.
+ * That is all components whose layout version is greater or equal to the
+ * one returned by {@link #getVersion()} must be upgraded with this object.
+ */
+public interface Upgradeable extends Comparable<Upgradeable> {
+  /**
+   * Get the layout version of the upgrade object.
+   * @return layout version
+   */
+  int getVersion();
+
+  /**
+   * Get the type of the software component, which this object is upgrading.
+   * @return type
+   */
+  HdfsConstants.NodeType getType();
+
+  /**
+   * Description of the upgrade object for displaying.
+   * @return description
+   */
+  String getDescription();
+
+  /**
+   * Upgrade status determines a percentage of the work done out of the total 
+   * amount required by the upgrade.
+   * 
+   * 100% means that the upgrade is completed.
+   * Any value < 100 means it is not complete.
+   * 
+   * The return value should provide at least 2 values, e.g. 0 and 100.
+   * @return integer value in the range [0, 100].
+   */
+  short getUpgradeStatus();
+
+  /**
+   * Prepare for the upgrade.
+   * E.g. initialize upgrade data structures and set status to 0.
+   * 
+   * Returns an upgrade command that is used for broadcasting to other cluster
+   * components. 
+   * E.g. name-node informs data-nodes that they must perform a distributed upgrade.
+   * 
+   * @return an UpgradeCommand for broadcasting.
+   * @throws IOException
+   */
+  UpgradeCommand startUpgrade() throws IOException;
+
+  /**
+   * Complete upgrade.
+   * E.g. cleanup upgrade data structures or write metadata to disk.
+   * 
+   * Returns an upgrade command that is used for broadcasting to other cluster
+   * components. 
+   * E.g. data-nodes inform the name-node that they completed the upgrade
+   * while other data-nodes are still upgrading.
+   * 
+   * @throws IOException
+   */
+  UpgradeCommand completeUpgrade() throws IOException;
+
+  /**
+   * Get status report for the upgrade.
+   * 
+   * @param details true if upgradeStatus details need to be included, 
+   *                false otherwise
+   * @return {@link UpgradeStatusReport}
+   * @throws IOException
+   */
+  UpgradeStatusReport getUpgradeStatusReport(boolean details) throws IOException;
+}

+ 28 - 0
src/java/org/apache/hadoop/hdfs/server/common/Util.java

@@ -0,0 +1,28 @@
+/**
+ * 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.common;
+
+public final class Util {
+  /**
+   * Current system time.
+   * @return current time in msec.
+   */
+  public static long now() {
+    return System.currentTimeMillis();
+  }
+}

+ 38 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/BlockAlreadyExistsException.java

@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.IOException;
+
+/**
+ * Exception indicating that the target block already exists 
+ * and is not set to be recovered/overwritten.  
+ */
+class BlockAlreadyExistsException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public BlockAlreadyExistsException() {
+    super();
+  }
+
+  public BlockAlreadyExistsException(String msg) {
+    super(msg);
+  }
+}

+ 130 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java

@@ -0,0 +1,130 @@
+/**
+ * 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.datanode;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.DataChecksum;
+
+
+/**
+ * BlockMetadataHeader manages metadata for data blocks on Datanodes.
+ * This is not related to the Block related functionality in Namenode.
+ * The biggest part of data block metadata is CRC for the block.
+ */
+class BlockMetadataHeader {
+
+  static final short METADATA_VERSION = FSDataset.METADATA_VERSION;
+  
+  /**
+   * Header includes everything except the checksum(s) themselves.
+   * Version is two bytes. Following it is the DataChecksum
+   * that occupies 5 bytes. 
+   */
+  private short version;
+  private DataChecksum checksum = null;
+    
+  BlockMetadataHeader(short version, DataChecksum checksum) {
+    this.checksum = checksum;
+    this.version = version;
+  }
+    
+  short getVersion() {
+    return version;
+  }
+
+  DataChecksum getChecksum() {
+    return checksum;
+  }
+
+ 
+  /**
+   * This reads all the fields till the beginning of checksum.
+   * @param in 
+   * @return Metadata Header
+   * @throws IOException
+   */
+  static BlockMetadataHeader readHeader(DataInputStream in) throws IOException {
+    return readHeader(in.readShort(), in);
+  }
+  
+  /**
+   * Reads header at the top of metadata file and returns the header.
+   * 
+   * @param dataset
+   * @param block
+   * @return
+   * @throws IOException
+   */
+  static BlockMetadataHeader readHeader(File file) throws IOException {
+    DataInputStream in = null;
+    try {
+      in = new DataInputStream(new BufferedInputStream(
+                               new FileInputStream(file)));
+      return readHeader(in);
+    } finally {
+      IOUtils.closeStream(in);
+    }
+  }
+  
+  // Version is already read.
+  private static BlockMetadataHeader readHeader(short version, DataInputStream in) 
+                                   throws IOException {
+    DataChecksum checksum = DataChecksum.newDataChecksum(in);
+    return new BlockMetadataHeader(version, checksum);
+  }
+  
+  /**
+   * This writes all the fields till the beginning of checksum.
+   * @param out DataOutputStream
+   * @param header 
+   * @return 
+   * @throws IOException
+   */
+  private static void writeHeader(DataOutputStream out, 
+                                  BlockMetadataHeader header) 
+                                  throws IOException {
+    out.writeShort(header.getVersion());
+    header.getChecksum().writeHeader(out);
+  }
+  
+  /**
+   * Writes all the fields till the beginning of checksum.
+   * @param out
+   * @param checksum
+   * @throws IOException
+   */
+  static void writeHeader(DataOutputStream out, DataChecksum checksum)
+                         throws IOException {
+    writeHeader(out, new BlockMetadataHeader(METADATA_VERSION, checksum));
+  }
+
+  /**
+   * Returns the size of the header
+   */
+  static int getHeaderSize() {
+    return Short.SIZE/Byte.SIZE + DataChecksum.getChecksumHeaderSize();
+  }
+}
+

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

@@ -0,0 +1,1014 @@
+/**
+ * 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.datanode;
+
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.LinkedList;
+import java.util.zip.CRC32;
+import java.util.zip.Checksum;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.fs.FSInputChecker;
+import org.apache.hadoop.fs.FSOutputSummer;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
+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.io.IOUtils;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.StringUtils;
+import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
+
+/** A class that receives a block and writes to its own disk, meanwhile
+ * may copies it to another site. If a throttler is provided,
+ * streaming throttling is also supported.
+ **/
+class BlockReceiver implements java.io.Closeable, FSConstants {
+  public static final Log LOG = DataNode.LOG;
+  static final Log ClientTraceLog = DataNode.ClientTraceLog;
+  
+  private Block block; // the block to receive
+  protected boolean finalized;
+  private DataInputStream in = null; // from where data are read
+  private DataChecksum checksum; // from where chunks of a block can be read
+  private OutputStream out = null; // to block file at local disk
+  private DataOutputStream checksumOut = null; // to crc file at local disk
+  private int bytesPerChecksum;
+  private int checksumSize;
+  private ByteBuffer buf; // contains one full packet.
+  private int bufRead; //amount of valid data in the buf
+  private int maxPacketReadLen;
+  protected long offsetInBlock;
+  protected final String inAddr;
+  protected final String myAddr;
+  private String mirrorAddr;
+  private DataOutputStream mirrorOut;
+  private Daemon responder = null;
+  private BlockTransferThrottler throttler;
+  private FSDataset.BlockWriteStreams streams;
+  private boolean isRecovery = false;
+  private String clientName;
+  DatanodeInfo srcDataNode = null;
+  private Checksum partialCrc = null;
+  private DataNode datanode = null;
+
+  BlockReceiver(Block block, DataInputStream in, String inAddr,
+                String myAddr, boolean isRecovery, String clientName, 
+                DatanodeInfo srcDataNode, DataNode datanode) throws IOException {
+    try{
+      this.block = block;
+      this.in = in;
+      this.inAddr = inAddr;
+      this.myAddr = myAddr;
+      this.isRecovery = isRecovery;
+      this.clientName = clientName;
+      this.offsetInBlock = 0;
+      this.srcDataNode = srcDataNode;
+      this.datanode = datanode;
+      this.checksum = DataChecksum.newDataChecksum(in);
+      this.bytesPerChecksum = checksum.getBytesPerChecksum();
+      this.checksumSize = checksum.getChecksumSize();
+      //
+      // Open local disk out
+      //
+      streams = datanode.data.writeToBlock(block, isRecovery);
+      this.finalized = datanode.data.isValidBlock(block);
+      if (streams != null) {
+        this.out = streams.dataOut;
+        this.checksumOut = new DataOutputStream(new BufferedOutputStream(
+                                                  streams.checksumOut, 
+                                                  SMALL_BUFFER_SIZE));
+        // If this block is for appends, then remove it from periodic
+        // validation.
+        if (datanode.blockScanner != null && isRecovery) {
+          datanode.blockScanner.deleteBlock(block);
+        }
+      }
+    } catch (BlockAlreadyExistsException bae) {
+      throw bae;
+    } catch(IOException ioe) {
+      IOUtils.closeStream(this);
+      cleanupBlock();
+      
+      // check if there is a disk error
+      IOException cause = FSDataset.getCauseIfDiskError(ioe);
+      if (cause != null) { // possible disk error
+        ioe = cause;
+        datanode.checkDiskError(ioe); // may throw an exception here
+      }
+      
+      throw ioe;
+    }
+  }
+
+  /**
+   * close files.
+   */
+  public void close() throws IOException {
+
+    IOException ioe = null;
+    // close checksum file
+    try {
+      if (checksumOut != null) {
+        checksumOut.flush();
+        checksumOut.close();
+        checksumOut = null;
+      }
+    } catch(IOException e) {
+      ioe = e;
+    }
+    // close block file
+    try {
+      if (out != null) {
+        out.flush();
+        out.close();
+        out = null;
+      }
+    } catch (IOException e) {
+      ioe = e;
+    }
+    // disk check
+    if(ioe != null) {
+      datanode.checkDiskError(ioe);
+      throw ioe;
+    }
+  }
+
+  /**
+   * Flush block data and metadata files to disk.
+   * @throws IOException
+   */
+  void flush() throws IOException {
+    if (checksumOut != null) {
+      checksumOut.flush();
+    }
+    if (out != null) {
+      out.flush();
+    }
+  }
+
+  /**
+   * While writing to mirrorOut, failure to write to mirror should not
+   * affect this datanode unless a client is writing the block.
+   */
+  private void handleMirrorOutError(IOException ioe) throws IOException {
+    LOG.info(datanode.dnRegistration + ":Exception writing block " +
+             block + " to mirror " + mirrorAddr + "\n" +
+             StringUtils.stringifyException(ioe));
+    mirrorOut = null;
+    //
+    // If stream-copy fails, continue
+    // writing to disk for replication requests. For client
+    // writes, return error so that the client can do error
+    // recovery.
+    //
+    if (clientName.length() > 0) {
+      throw ioe;
+    }
+  }
+  
+  /**
+   * Verify multiple CRC chunks. 
+   */
+  private void verifyChunks( byte[] dataBuf, int dataOff, int len, 
+                             byte[] checksumBuf, int checksumOff ) 
+                             throws IOException {
+    while (len > 0) {
+      int chunkLen = Math.min(len, bytesPerChecksum);
+      
+      checksum.update(dataBuf, dataOff, chunkLen);
+
+      if (!checksum.compare(checksumBuf, checksumOff)) {
+        if (srcDataNode != null) {
+          try {
+            LOG.info("report corrupt block " + block + " from datanode " +
+                      srcDataNode + " to namenode");
+            LocatedBlock lb = new LocatedBlock(block, 
+                                            new DatanodeInfo[] {srcDataNode});
+            datanode.namenode.reportBadBlocks(new LocatedBlock[] {lb});
+          } catch (IOException e) {
+            LOG.warn("Failed to report bad block " + block + 
+                      " from datanode " + srcDataNode + " to namenode");
+          }
+        }
+        throw new IOException("Unexpected checksum mismatch " + 
+                              "while writing " + block + " from " + inAddr);
+      }
+
+      checksum.reset();
+      dataOff += chunkLen;
+      checksumOff += checksumSize;
+      len -= chunkLen;
+    }
+  }
+
+  /**
+   * Makes sure buf.position() is zero without modifying buf.remaining().
+   * It moves the data if position needs to be changed.
+   */
+  private void shiftBufData() {
+    if (bufRead != buf.limit()) {
+      throw new IllegalStateException("bufRead should be same as " +
+                                      "buf.limit()");
+    }
+    
+    //shift the remaining data on buf to the front
+    if (buf.position() > 0) {
+      int dataLeft = buf.remaining();
+      if (dataLeft > 0) {
+        byte[] b = buf.array();
+        System.arraycopy(b, buf.position(), b, 0, dataLeft);
+      }
+      buf.position(0);
+      bufRead = dataLeft;
+      buf.limit(bufRead);
+    }
+  }
+  
+  /**
+   * reads upto toRead byte to buf at buf.limit() and increments the limit.
+   * throws an IOException if read does not succeed.
+   */
+  private int readToBuf(int toRead) throws IOException {
+    if (toRead < 0) {
+      toRead = (maxPacketReadLen > 0 ? maxPacketReadLen : buf.capacity())
+               - buf.limit();
+    }
+    
+    int nRead = in.read(buf.array(), buf.limit(), toRead);
+    
+    if (nRead < 0) {
+      throw new EOFException("while trying to read " + toRead + " bytes");
+    }
+    bufRead = buf.limit() + nRead;
+    buf.limit(bufRead);
+    return nRead;
+  }
+  
+  
+  /**
+   * Reads (at least) one packet and returns the packet length.
+   * buf.position() points to the start of the packet and 
+   * buf.limit() point to the end of the packet. There could 
+   * be more data from next packet in buf.<br><br>
+   * 
+   * It tries to read a full packet with single read call.
+   * Consecutive packets are usually of the same length.
+   */
+  private int readNextPacket() throws IOException {
+    /* This dances around buf a little bit, mainly to read 
+     * full packet with single read and to accept arbitarary size  
+     * for next packet at the same time.
+     */
+    if (buf == null) {
+      /* initialize buffer to the best guess size:
+       * 'chunksPerPacket' calculation here should match the same 
+       * calculation in DFSClient to make the guess accurate.
+       */
+      int chunkSize = bytesPerChecksum + checksumSize;
+      int chunksPerPacket = (datanode.writePacketSize - DataNode.PKT_HEADER_LEN - 
+                             SIZE_OF_INTEGER + chunkSize - 1)/chunkSize;
+      buf = ByteBuffer.allocate(DataNode.PKT_HEADER_LEN + SIZE_OF_INTEGER +
+                                Math.max(chunksPerPacket, 1) * chunkSize);
+      buf.limit(0);
+    }
+    
+    // See if there is data left in the buffer :
+    if (bufRead > buf.limit()) {
+      buf.limit(bufRead);
+    }
+    
+    while (buf.remaining() < SIZE_OF_INTEGER) {
+      if (buf.position() > 0) {
+        shiftBufData();
+      }
+      readToBuf(-1);
+    }
+    
+    /* We mostly have the full packet or at least enough for an int
+     */
+    buf.mark();
+    int payloadLen = buf.getInt();
+    buf.reset();
+    
+    if (payloadLen == 0) {
+      //end of stream!
+      buf.limit(buf.position() + SIZE_OF_INTEGER);
+      return 0;
+    }
+    
+    // check corrupt values for pktLen, 100MB upper limit should be ok?
+    if (payloadLen < 0 || payloadLen > (100*1024*1024)) {
+      throw new IOException("Incorrect value for packet payload : " +
+                            payloadLen);
+    }
+    
+    int pktSize = payloadLen + DataNode.PKT_HEADER_LEN;
+    
+    if (buf.remaining() < pktSize) {
+      //we need to read more data
+      int toRead = pktSize - buf.remaining();
+      
+      // first make sure buf has enough space.        
+      int spaceLeft = buf.capacity() - buf.limit();
+      if (toRead > spaceLeft && buf.position() > 0) {
+        shiftBufData();
+        spaceLeft = buf.capacity() - buf.limit();
+      }
+      if (toRead > spaceLeft) {
+        byte oldBuf[] = buf.array();
+        int toCopy = buf.limit();
+        buf = ByteBuffer.allocate(toCopy + toRead);
+        System.arraycopy(oldBuf, 0, buf.array(), 0, toCopy);
+        buf.limit(toCopy);
+      }
+      
+      //now read:
+      while (toRead > 0) {
+        toRead -= readToBuf(toRead);
+      }
+    }
+    
+    if (buf.remaining() > pktSize) {
+      buf.limit(buf.position() + pktSize);
+    }
+    
+    if (pktSize > maxPacketReadLen) {
+      maxPacketReadLen = pktSize;
+    }
+    
+    return payloadLen;
+  }
+  
+  /** 
+   * Receives and processes a packet. It can contain many chunks.
+   * returns size of the packet.
+   */
+  private int receivePacket() throws IOException {
+    
+    int payloadLen = readNextPacket();
+    
+    if (payloadLen <= 0) {
+      return payloadLen;
+    }
+    
+    buf.mark();
+    //read the header
+    buf.getInt(); // packet length
+    offsetInBlock = buf.getLong(); // get offset of packet in block
+    long seqno = buf.getLong();    // get seqno
+    boolean lastPacketInBlock = (buf.get() != 0);
+    
+    int endOfHeader = buf.position();
+    buf.reset();
+    
+    if (LOG.isDebugEnabled()){
+      LOG.debug("Receiving one packet for block " + block +
+                " of length " + payloadLen +
+                " seqno " + seqno +
+                " offsetInBlock " + offsetInBlock +
+                " lastPacketInBlock " + lastPacketInBlock);
+    }
+    
+    setBlockPosition(offsetInBlock);
+    
+    //First write the packet to the mirror:
+    if (mirrorOut != null) {
+      try {
+        mirrorOut.write(buf.array(), buf.position(), buf.remaining());
+        mirrorOut.flush();
+      } catch (IOException e) {
+        handleMirrorOutError(e);
+      }
+    }
+
+    buf.position(endOfHeader);        
+    int len = buf.getInt();
+    
+    if (len < 0) {
+      throw new IOException("Got wrong length during writeBlock(" + block + 
+                            ") from " + inAddr + " at offset " + 
+                            offsetInBlock + ": " + len); 
+    } 
+
+    if (len == 0) {
+      LOG.debug("Receiving empty packet for block " + block);
+    } else {
+      offsetInBlock += len;
+
+      int checksumLen = ((len + bytesPerChecksum - 1)/bytesPerChecksum)*
+                                                            checksumSize;
+
+      if ( buf.remaining() != (checksumLen + len)) {
+        throw new IOException("Data remaining in packet does not match " +
+                              "sum of checksumLen and dataLen");
+      }
+      int checksumOff = buf.position();
+      int dataOff = checksumOff + checksumLen;
+      byte pktBuf[] = buf.array();
+
+      buf.position(buf.limit()); // move to the end of the data.
+
+      /* skip verifying checksum iff this is not the last one in the 
+       * pipeline and clientName is non-null. i.e. Checksum is verified
+       * on all the datanodes when the data is being written by a 
+       * datanode rather than a client. Whe client is writing the data, 
+       * protocol includes acks and only the last datanode needs to verify 
+       * checksum.
+       */
+      if (mirrorOut == null || clientName.length() == 0) {
+        verifyChunks(pktBuf, dataOff, len, pktBuf, checksumOff);
+      }
+
+      try {
+        if (!finalized) {
+          //finally write to the disk :
+          out.write(pktBuf, dataOff, len);
+
+          // If this is a partial chunk, then verify that this is the only
+          // chunk in the packet. Calculate new crc for this chunk.
+          if (partialCrc != null) {
+            if (len > bytesPerChecksum) {
+              throw new IOException("Got wrong length during writeBlock(" + 
+                                    block + ") from " + inAddr + " " +
+                                    "A packet can have only one partial chunk."+
+                                    " len = " + len + 
+                                    " bytesPerChecksum " + bytesPerChecksum);
+            }
+            partialCrc.update(pktBuf, dataOff, len);
+            byte[] buf = FSOutputSummer.convertToByteStream(partialCrc, checksumSize);
+            checksumOut.write(buf);
+            LOG.debug("Writing out partial crc for data len " + len);
+            partialCrc = null;
+          } else {
+            checksumOut.write(pktBuf, checksumOff, checksumLen);
+          }
+          datanode.myMetrics.bytesWritten.inc(len);
+        }
+      } catch (IOException iex) {
+        datanode.checkDiskError(iex);
+        throw iex;
+      }
+    }
+
+    /// flush entire packet before sending ack
+    flush();
+
+    // put in queue for pending acks
+    if (responder != null) {
+      ((PacketResponder)responder.getRunnable()).enqueue(seqno,
+                                      lastPacketInBlock); 
+    }
+    
+    if (throttler != null) { // throttle I/O
+      throttler.throttle(payloadLen);
+    }
+    
+    return payloadLen;
+  }
+
+  void writeChecksumHeader(DataOutputStream mirrorOut) throws IOException {
+    checksum.writeHeader(mirrorOut);
+  }
+ 
+
+  void receiveBlock(
+      DataOutputStream mirrOut, // output to next datanode
+      DataInputStream mirrIn,   // input from next datanode
+      DataOutputStream replyOut,  // output to previous datanode
+      String mirrAddr, BlockTransferThrottler throttlerArg,
+      int numTargets) throws IOException {
+
+      mirrorOut = mirrOut;
+      mirrorAddr = mirrAddr;
+      throttler = throttlerArg;
+
+    try {
+      // write data chunk header
+      if (!finalized) {
+        BlockMetadataHeader.writeHeader(checksumOut, checksum);
+      }
+      if (clientName.length() > 0) {
+        responder = new Daemon(datanode.threadGroup, 
+                               new PacketResponder(this, block, mirrIn, 
+                                                   replyOut, numTargets));
+        responder.start(); // start thread to processes reponses
+      }
+
+      /* 
+       * Receive until packet length is zero.
+       */
+      while (receivePacket() > 0) {}
+
+      // flush the mirror out
+      if (mirrorOut != null) {
+        try {
+          mirrorOut.writeInt(0); // mark the end of the block
+          mirrorOut.flush();
+        } catch (IOException e) {
+          handleMirrorOutError(e);
+        }
+      }
+
+      // wait for all outstanding packet responses. And then
+      // indicate responder to gracefully shutdown.
+      if (responder != null) {
+        ((PacketResponder)responder.getRunnable()).close();
+      }
+
+      // if this write is for a replication request (and not
+      // from a client), then finalize block. For client-writes, 
+      // the block is finalized in the PacketResponder.
+      if (clientName.length() == 0) {
+        // close the block/crc files
+        close();
+
+        // Finalize the block. Does this fsync()?
+        block.setNumBytes(offsetInBlock);
+        datanode.data.finalizeBlock(block);
+        datanode.myMetrics.blocksWritten.inc();
+      }
+
+    } catch (IOException ioe) {
+      LOG.info("Exception in receiveBlock for block " + block + 
+               " " + ioe);
+      IOUtils.closeStream(this);
+      if (responder != null) {
+        responder.interrupt();
+      }
+      cleanupBlock();
+      throw ioe;
+    } finally {
+      if (responder != null) {
+        try {
+          responder.join();
+        } catch (InterruptedException e) {
+          throw new IOException("Interrupted receiveBlock");
+        }
+        responder = null;
+      }
+    }
+  }
+
+  /** Cleanup a partial block 
+   * if this write is for a replication request (and not from a client)
+   */
+  private void cleanupBlock() throws IOException {
+    if (clientName.length() == 0) { // not client write
+      datanode.data.unfinalizeBlock(block);
+    }
+  }
+
+  /**
+   * Sets the file pointer in the local block file to the specified value.
+   */
+  private void setBlockPosition(long offsetInBlock) throws IOException {
+    if (finalized) {
+      if (!isRecovery) {
+        throw new IOException("Write to offset " + offsetInBlock +
+                              " of block " + block +
+                              " that is already finalized.");
+      }
+      if (offsetInBlock > datanode.data.getLength(block)) {
+        throw new IOException("Write to offset " + offsetInBlock +
+                              " of block " + block +
+                              " that is already finalized and is of size " +
+                              datanode.data.getLength(block));
+      }
+      return;
+    }
+
+    if (datanode.data.getChannelPosition(block, streams) == offsetInBlock) {
+      return;                   // nothing to do 
+    }
+    long offsetInChecksum = BlockMetadataHeader.getHeaderSize() +
+                            offsetInBlock / bytesPerChecksum * checksumSize;
+    if (out != null) {
+     out.flush();
+    }
+    if (checksumOut != null) {
+      checksumOut.flush();
+    }
+
+    // If this is a partial chunk, then read in pre-existing checksum
+    if (offsetInBlock % bytesPerChecksum != 0) {
+      LOG.info("setBlockPosition trying to set position to " +
+               offsetInBlock +
+               " for block " + block +
+               " which is not a multiple of bytesPerChecksum " +
+               bytesPerChecksum);
+      computePartialChunkCrc(offsetInBlock, offsetInChecksum, bytesPerChecksum);
+    }
+
+    LOG.info("Changing block file offset of block " + block + " from " + 
+        datanode.data.getChannelPosition(block, streams) +
+             " to " + offsetInBlock +
+             " meta file offset to " + offsetInChecksum);
+
+    // set the position of the block file
+    datanode.data.setChannelPosition(block, streams, offsetInBlock, offsetInChecksum);
+  }
+
+  /**
+   * reads in the partial crc chunk and computes checksum
+   * of pre-existing data in partial chunk.
+   */
+  private void computePartialChunkCrc(long blkoff, long ckoff, 
+                                      int bytesPerChecksum) throws IOException {
+
+    // find offset of the beginning of partial chunk.
+    //
+    int sizePartialChunk = (int) (blkoff % bytesPerChecksum);
+    int checksumSize = checksum.getChecksumSize();
+    blkoff = blkoff - sizePartialChunk;
+    LOG.info("computePartialChunkCrc sizePartialChunk " + 
+              sizePartialChunk +
+              " block " + block +
+              " offset in block " + blkoff +
+              " offset in metafile " + ckoff);
+
+    // create an input stream from the block file
+    // and read in partial crc chunk into temporary buffer
+    //
+    byte[] buf = new byte[sizePartialChunk];
+    byte[] crcbuf = new byte[checksumSize];
+    FSDataset.BlockInputStreams instr = null;
+    try { 
+      instr = datanode.data.getTmpInputStreams(block, blkoff, ckoff);
+      IOUtils.readFully(instr.dataIn, buf, 0, sizePartialChunk);
+
+      // open meta file and read in crc value computer earlier
+      IOUtils.readFully(instr.checksumIn, crcbuf, 0, crcbuf.length);
+    } finally {
+      IOUtils.closeStream(instr);
+    }
+
+    // compute crc of partial chunk from data read in the block file.
+    partialCrc = new CRC32();
+    partialCrc.update(buf, 0, sizePartialChunk);
+    LOG.info("Read in partial CRC chunk from disk for block " + block);
+
+    // paranoia! verify that the pre-computed crc matches what we
+    // recalculated just now
+    if (partialCrc.getValue() != FSInputChecker.checksum2long(crcbuf)) {
+      String msg = "Partial CRC " + partialCrc.getValue() +
+                   " does not match value computed the " +
+                   " last time file was closed " +
+                   FSInputChecker.checksum2long(crcbuf);
+      throw new IOException(msg);
+    }
+    //LOG.debug("Partial CRC matches 0x" + 
+    //            Long.toHexString(partialCrc.getValue()));
+  }
+  
+  
+  /**
+   * Processed responses from downstream datanodes in the pipeline
+   * and sends back replies to the originator.
+   */
+  class PacketResponder implements Runnable, FSConstants {   
+
+    //packet waiting for ack
+    private LinkedList<Packet> ackQueue = new LinkedList<Packet>(); 
+    private volatile boolean running = true;
+    private Block block;
+    DataInputStream mirrorIn;   // input from downstream datanode
+    DataOutputStream replyOut;  // output to upstream datanode
+    private int numTargets;     // number of downstream datanodes including myself
+    private BlockReceiver receiver; // The owner of this responder.
+
+    public String toString() {
+      return "PacketResponder " + numTargets + " for Block " + this.block;
+    }
+
+    PacketResponder(BlockReceiver receiver, Block b, DataInputStream in, 
+                    DataOutputStream out, int numTargets) {
+      this.receiver = receiver;
+      this.block = b;
+      mirrorIn = in;
+      replyOut = out;
+      this.numTargets = numTargets;
+    }
+
+    /**
+     * enqueue the seqno that is still be to acked by the downstream datanode.
+     * @param seqno
+     * @param lastPacketInBlock
+     */
+    synchronized void enqueue(long seqno, boolean lastPacketInBlock) {
+      if (running) {
+        LOG.debug("PacketResponder " + numTargets + " adding seqno " + seqno +
+                  " to ack queue.");
+        ackQueue.addLast(new Packet(seqno, lastPacketInBlock));
+        notifyAll();
+      }
+    }
+
+    /**
+     * wait for all pending packets to be acked. Then shutdown thread.
+     */
+    synchronized void close() {
+      while (running && ackQueue.size() != 0 && datanode.shouldRun) {
+        try {
+          wait();
+        } catch (InterruptedException e) {
+          running = false;
+        }
+      }
+      LOG.debug("PacketResponder " + numTargets +
+               " for block " + block + " Closing down.");
+      running = false;
+      notifyAll();
+    }
+
+    private synchronized void lastDataNodeRun() {
+      long lastHeartbeat = System.currentTimeMillis();
+      boolean lastPacket = false;
+      final long startTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
+
+      while (running && datanode.shouldRun && !lastPacket) {
+        long now = System.currentTimeMillis();
+        try {
+
+            // wait for a packet to be sent to downstream datanode
+            while (running && datanode.shouldRun && ackQueue.size() == 0) {
+              long idle = now - lastHeartbeat;
+              long timeout = (datanode.socketTimeout/2) - idle;
+              if (timeout <= 0) {
+                timeout = 1000;
+              }
+              try {
+                wait(timeout);
+              } catch (InterruptedException e) {
+                if (running) {
+                  LOG.info("PacketResponder " + numTargets +
+                           " for block " + block + " Interrupted.");
+                  running = false;
+                }
+                break;
+              }
+          
+              // send a heartbeat if it is time.
+              now = System.currentTimeMillis();
+              if (now - lastHeartbeat > datanode.socketTimeout/2) {
+                replyOut.writeLong(-1); // send heartbeat
+                replyOut.flush();
+                lastHeartbeat = now;
+              }
+            }
+
+            if (!running || !datanode.shouldRun) {
+              break;
+            }
+            Packet pkt = ackQueue.removeFirst();
+            long expected = pkt.seqno;
+            notifyAll();
+            LOG.debug("PacketResponder " + numTargets +
+                      " for block " + block + 
+                      " acking for packet " + expected);
+
+            // If this is the last packet in block, then close block
+            // file and finalize the block before responding success
+            if (pkt.lastPacketInBlock) {
+              if (!receiver.finalized) {
+                receiver.close();
+                final long endTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
+                block.setNumBytes(receiver.offsetInBlock);
+                datanode.data.finalizeBlock(block);
+                datanode.myMetrics.blocksWritten.inc();
+                datanode.notifyNamenodeReceivedBlock(block, 
+                    DataNode.EMPTY_DEL_HINT);
+                if (ClientTraceLog.isInfoEnabled() &&
+                    receiver.clientName.length() > 0) {
+                  long offset = 0;
+                  ClientTraceLog.info(String.format(DN_CLIENTTRACE_FORMAT,
+                        receiver.inAddr, receiver.myAddr, block.getNumBytes(),
+                        "HDFS_WRITE", receiver.clientName, offset,
+                        datanode.dnRegistration.getStorageID(), block, endTime-startTime));
+                } else {
+                  LOG.info("Received block " + block + 
+                           " of size " + block.getNumBytes() + 
+                           " from " + receiver.inAddr);
+                }
+              }
+              lastPacket = true;
+            }
+
+            replyOut.writeLong(expected);
+            replyOut.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS);
+            replyOut.flush();
+        } catch (Exception e) {
+          if (running) {
+            LOG.info("PacketResponder " + block + " " + numTargets + 
+                     " Exception " + StringUtils.stringifyException(e));
+            running = false;
+          }
+        }
+      }
+      LOG.info("PacketResponder " + numTargets + 
+               " for block " + block + " terminating");
+    }
+
+    /**
+     * Thread to process incoming acks.
+     * @see java.lang.Runnable#run()
+     */
+    public void run() {
+
+      // If this is the last datanode in pipeline, then handle differently
+      if (numTargets == 0) {
+        lastDataNodeRun();
+        return;
+      }
+
+      boolean lastPacketInBlock = false;
+      final long startTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
+      while (running && datanode.shouldRun && !lastPacketInBlock) {
+
+        try {
+            short op = DataTransferProtocol.OP_STATUS_SUCCESS;
+            boolean didRead = false;
+            long expected = -2;
+            try { 
+              // read seqno from downstream datanode
+              long seqno = mirrorIn.readLong();
+              didRead = true;
+              if (seqno == -1) {
+                replyOut.writeLong(-1); // send keepalive
+                replyOut.flush();
+                LOG.debug("PacketResponder " + numTargets + " got -1");
+                continue;
+              } else if (seqno == -2) {
+                LOG.debug("PacketResponder " + numTargets + " got -2");
+              } else {
+                LOG.debug("PacketResponder " + numTargets + " got seqno = " + 
+                    seqno);
+                Packet pkt = null;
+                synchronized (this) {
+                  while (running && datanode.shouldRun && ackQueue.size() == 0) {
+                    if (LOG.isDebugEnabled()) {
+                      LOG.debug("PacketResponder " + numTargets + 
+                                " seqno = " + seqno +
+                                " for block " + block +
+                                " waiting for local datanode to finish write.");
+                    }
+                    wait();
+                  }
+                  pkt = ackQueue.removeFirst();
+                  expected = pkt.seqno;
+                  notifyAll();
+                  LOG.debug("PacketResponder " + numTargets + " seqno = " + seqno);
+                  if (seqno != expected) {
+                    throw new IOException("PacketResponder " + numTargets +
+                                          " for block " + block +
+                                          " expected seqno:" + expected +
+                                          " received:" + seqno);
+                  }
+                  lastPacketInBlock = pkt.lastPacketInBlock;
+                }
+              }
+            } catch (Throwable e) {
+              if (running) {
+                LOG.info("PacketResponder " + block + " " + numTargets + 
+                         " Exception " + StringUtils.stringifyException(e));
+                running = false;
+              }
+            }
+
+            if (Thread.interrupted()) {
+              /* The receiver thread cancelled this thread. 
+               * We could also check any other status updates from the 
+               * receiver thread (e.g. if it is ok to write to replyOut). 
+               * It is prudent to not send any more status back to the client
+               * because this datanode has a problem. The upstream datanode
+               * will detect a timout on heartbeats and will declare that
+               * this datanode is bad, and rightly so.
+               */
+              LOG.info("PacketResponder " + block +  " " + numTargets +
+                       " : Thread is interrupted.");
+              running = false;
+              continue;
+            }
+            
+            if (!didRead) {
+              op = DataTransferProtocol.OP_STATUS_ERROR;
+            }
+            
+            // If this is the last packet in block, then close block
+            // file and finalize the block before responding success
+            if (lastPacketInBlock && !receiver.finalized) {
+              receiver.close();
+              final long endTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
+              block.setNumBytes(receiver.offsetInBlock);
+              datanode.data.finalizeBlock(block);
+              datanode.myMetrics.blocksWritten.inc();
+              datanode.notifyNamenodeReceivedBlock(block, 
+                  DataNode.EMPTY_DEL_HINT);
+              if (ClientTraceLog.isInfoEnabled() &&
+                  receiver.clientName.length() > 0) {
+                long offset = 0;
+                ClientTraceLog.info(String.format(DN_CLIENTTRACE_FORMAT,
+                      receiver.inAddr, receiver.myAddr, block.getNumBytes(),
+                      "HDFS_WRITE", receiver.clientName, offset,
+                      datanode.dnRegistration.getStorageID(), block, endTime-startTime));
+              } else {
+                LOG.info("Received block " + block + 
+                         " of size " + block.getNumBytes() + 
+                         " from " + receiver.inAddr);
+              }
+            }
+
+            // send my status back to upstream datanode
+            replyOut.writeLong(expected); // send seqno upstream
+            replyOut.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS);
+
+            LOG.debug("PacketResponder " + numTargets + 
+                      " for block " + block +
+                      " responded my status " +
+                      " for seqno " + expected);
+
+            // forward responses from downstream datanodes.
+            for (int i = 0; i < numTargets && datanode.shouldRun; i++) {
+              try {
+                if (op == DataTransferProtocol.OP_STATUS_SUCCESS) {
+                  op = mirrorIn.readShort();
+                  if (op != DataTransferProtocol.OP_STATUS_SUCCESS) {
+                    LOG.debug("PacketResponder for block " + block +
+                              ": error code received from downstream " +
+                              " datanode[" + i + "] " + op);
+                  }
+                }
+              } catch (Throwable e) {
+                op = DataTransferProtocol.OP_STATUS_ERROR;
+              }
+              replyOut.writeShort(op);
+            }
+            replyOut.flush();
+            LOG.debug("PacketResponder " + block + " " + numTargets + 
+                      " responded other status " + " for seqno " + expected);
+
+            // If we were unable to read the seqno from downstream, then stop.
+            if (expected == -2) {
+              running = false;
+            }
+            // If we forwarded an error response from a downstream datanode
+            // and we are acting on behalf of a client, then we quit. The 
+            // client will drive the recovery mechanism.
+            if (op == DataTransferProtocol.OP_STATUS_ERROR && receiver.clientName.length() > 0) {
+              running = false;
+            }
+        } catch (IOException e) {
+          if (running) {
+            LOG.info("PacketResponder " + block + " " + numTargets + 
+                     " Exception " + StringUtils.stringifyException(e));
+            running = false;
+          }
+        } catch (RuntimeException e) {
+          if (running) {
+            LOG.info("PacketResponder " + block + " " + numTargets + 
+                     " Exception " + StringUtils.stringifyException(e));
+            running = false;
+          }
+        }
+      }
+      LOG.info("PacketResponder " + numTargets + 
+               " for block " + block + " terminating");
+    }
+  }
+  
+  /**
+   * This information is cached by the Datanode in the ackQueue.
+   */
+  static private class Packet {
+    long seqno;
+    boolean lastPacketInBlock;
+
+    Packet(long seqno, boolean lastPacketInBlock) {
+      this.seqno = seqno;
+      this.lastPacketInBlock = lastPacketInBlock;
+    }
+  }
+}

+ 430 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java

@@ -0,0 +1,430 @@
+/**
+ * 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.datanode;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.SocketException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.Arrays;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.net.SocketOutputStream;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Reads a block from the disk and sends it to a recipient.
+ */
+class BlockSender implements java.io.Closeable, FSConstants {
+  public static final Log LOG = DataNode.LOG;
+  static final Log ClientTraceLog = DataNode.ClientTraceLog;
+  
+  private Block block; // the block to read from
+  private InputStream blockIn; // data stream
+  private long blockInPosition = -1; // updated while using transferTo().
+  private DataInputStream checksumIn; // checksum datastream
+  private DataChecksum checksum; // checksum stream
+  private long offset; // starting position to read
+  private long endOffset; // ending position
+  private long blockLength;
+  private int bytesPerChecksum; // chunk size
+  private int checksumSize; // checksum size
+  private boolean corruptChecksumOk; // if need to verify checksum
+  private boolean chunkOffsetOK; // if need to send chunk offset
+  private long seqno; // sequence number of packet
+
+  private boolean transferToAllowed = true;
+  private boolean blockReadFully; //set when the whole block is read
+  private boolean verifyChecksum; //if true, check is verified while reading
+  private BlockTransferThrottler throttler;
+  private final String clientTraceFmt; // format of client trace log message
+
+  /**
+   * Minimum buffer used while sending data to clients. Used only if
+   * transferTo() is enabled. 64KB is not that large. It could be larger, but
+   * not sure if there will be much more improvement.
+   */
+  private static final int MIN_BUFFER_WITH_TRANSFERTO = 64*1024;
+
+  
+  BlockSender(Block block, long startOffset, long length,
+              boolean corruptChecksumOk, boolean chunkOffsetOK,
+              boolean verifyChecksum, DataNode datanode) throws IOException {
+    this(block, startOffset, length, corruptChecksumOk, chunkOffsetOK,
+         verifyChecksum, datanode, null);
+  }
+
+  BlockSender(Block block, long startOffset, long length,
+              boolean corruptChecksumOk, boolean chunkOffsetOK,
+              boolean verifyChecksum, DataNode datanode, String clientTraceFmt)
+      throws IOException {
+    try {
+      this.block = block;
+      this.chunkOffsetOK = chunkOffsetOK;
+      this.corruptChecksumOk = corruptChecksumOk;
+      this.verifyChecksum = verifyChecksum;
+      this.blockLength = datanode.data.getLength(block);
+      this.transferToAllowed = datanode.transferToAllowed;
+      this.clientTraceFmt = clientTraceFmt;
+
+      if ( !corruptChecksumOk || datanode.data.metaFileExists(block) ) {
+        checksumIn = new DataInputStream(
+                new BufferedInputStream(datanode.data.getMetaDataInputStream(block),
+                                        BUFFER_SIZE));
+
+        // read and handle the common header here. For now just a version
+       BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
+       short version = header.getVersion();
+
+        if (version != FSDataset.METADATA_VERSION) {
+          LOG.warn("Wrong version (" + version + ") for metadata file for "
+              + block + " ignoring ...");
+        }
+        checksum = header.getChecksum();
+      } else {
+        LOG.warn("Could not find metadata file for " + block);
+        // This only decides the buffer size. Use BUFFER_SIZE?
+        checksum = DataChecksum.newDataChecksum(DataChecksum.CHECKSUM_NULL,
+            16 * 1024);
+      }
+
+      /* If bytesPerChecksum is very large, then the metadata file
+       * is mostly corrupted. For now just truncate bytesPerchecksum to
+       * blockLength.
+       */        
+      bytesPerChecksum = checksum.getBytesPerChecksum();
+      if (bytesPerChecksum > 10*1024*1024 && bytesPerChecksum > blockLength){
+        checksum = DataChecksum.newDataChecksum(checksum.getChecksumType(),
+                                   Math.max((int)blockLength, 10*1024*1024));
+        bytesPerChecksum = checksum.getBytesPerChecksum();        
+      }
+      checksumSize = checksum.getChecksumSize();
+
+      if (length < 0) {
+        length = blockLength;
+      }
+
+      endOffset = blockLength;
+      if (startOffset < 0 || startOffset > endOffset
+          || (length + startOffset) > endOffset) {
+        String msg = " Offset " + startOffset + " and length " + length
+        + " don't match block " + block + " ( blockLen " + endOffset + " )";
+        LOG.warn(datanode.dnRegistration + ":sendBlock() : " + msg);
+        throw new IOException(msg);
+      }
+
+      
+      offset = (startOffset - (startOffset % bytesPerChecksum));
+      if (length >= 0) {
+        // Make sure endOffset points to end of a checksumed chunk.
+        long tmpLen = startOffset + length;
+        if (tmpLen % bytesPerChecksum != 0) {
+          tmpLen += (bytesPerChecksum - tmpLen % bytesPerChecksum);
+        }
+        if (tmpLen < endOffset) {
+          endOffset = tmpLen;
+        }
+      }
+
+      // seek to the right offsets
+      if (offset > 0) {
+        long checksumSkip = (offset / bytesPerChecksum) * checksumSize;
+        // note blockInStream is  seeked when created below
+        if (checksumSkip > 0) {
+          // Should we use seek() for checksum file as well?
+          IOUtils.skipFully(checksumIn, checksumSkip);
+        }
+      }
+      seqno = 0;
+
+      blockIn = datanode.data.getBlockInputStream(block, offset); // seek to offset
+    } catch (IOException ioe) {
+      IOUtils.closeStream(this);
+      IOUtils.closeStream(blockIn);
+      throw ioe;
+    }
+  }
+
+  /**
+   * close opened files.
+   */
+  public void close() throws IOException {
+    IOException ioe = null;
+    // close checksum file
+    if(checksumIn!=null) {
+      try {
+        checksumIn.close();
+      } catch (IOException e) {
+        ioe = e;
+      }
+      checksumIn = null;
+    }
+    // close data file
+    if(blockIn!=null) {
+      try {
+        blockIn.close();
+      } catch (IOException e) {
+        ioe = e;
+      }
+      blockIn = null;
+    }
+    // throw IOException if there is any
+    if(ioe!= null) {
+      throw ioe;
+    }
+  }
+
+  /**
+   * Converts an IOExcpetion (not subclasses) to SocketException.
+   * This is typically done to indicate to upper layers that the error 
+   * was a socket error rather than often more serious exceptions like 
+   * disk errors.
+   */
+  private static IOException ioeToSocketException(IOException ioe) {
+    if (ioe.getClass().equals(IOException.class)) {
+      // "se" could be a new class in stead of SocketException.
+      IOException se = new SocketException("Original Exception : " + ioe);
+      se.initCause(ioe);
+      /* Change the stacktrace so that original trace is not truncated
+       * when printed.*/ 
+      se.setStackTrace(ioe.getStackTrace());
+      return se;
+    }
+    // otherwise just return the same exception.
+    return ioe;
+  }
+
+  /**
+   * Sends upto maxChunks chunks of data.
+   * 
+   * When blockInPosition is >= 0, assumes 'out' is a 
+   * {@link SocketOutputStream} and tries 
+   * {@link SocketOutputStream#transferToFully(FileChannel, long, int)} to
+   * send data (and updates blockInPosition).
+   */
+  private int sendChunks(ByteBuffer pkt, int maxChunks, OutputStream out) 
+                         throws IOException {
+    // Sends multiple chunks in one packet with a single write().
+
+    int len = Math.min((int) (endOffset - offset),
+                       bytesPerChecksum*maxChunks);
+    if (len == 0) {
+      return 0;
+    }
+
+    int numChunks = (len + bytesPerChecksum - 1)/bytesPerChecksum;
+    int packetLen = len + numChunks*checksumSize + 4;
+    pkt.clear();
+    
+    // write packet header
+    pkt.putInt(packetLen);
+    pkt.putLong(offset);
+    pkt.putLong(seqno);
+    pkt.put((byte)((offset + len >= endOffset) ? 1 : 0));
+               //why no ByteBuf.putBoolean()?
+    pkt.putInt(len);
+    
+    int checksumOff = pkt.position();
+    int checksumLen = numChunks * checksumSize;
+    byte[] buf = pkt.array();
+    
+    if (checksumSize > 0 && checksumIn != null) {
+      try {
+        checksumIn.readFully(buf, checksumOff, checksumLen);
+      } catch (IOException e) {
+        LOG.warn(" Could not read or failed to veirfy checksum for data" +
+                 " at offset " + offset + " for block " + block + " got : "
+                 + StringUtils.stringifyException(e));
+        IOUtils.closeStream(checksumIn);
+        checksumIn = null;
+        if (corruptChecksumOk) {
+          if (checksumOff < checksumLen) {
+            // Just fill the array with zeros.
+            Arrays.fill(buf, checksumOff, checksumLen, (byte) 0);
+          }
+        } else {
+          throw e;
+        }
+      }
+    }
+    
+    int dataOff = checksumOff + checksumLen;
+    
+    if (blockInPosition < 0) {
+      //normal transfer
+      IOUtils.readFully(blockIn, buf, dataOff, len);
+
+      if (verifyChecksum) {
+        int dOff = dataOff;
+        int cOff = checksumOff;
+        int dLeft = len;
+
+        for (int i=0; i<numChunks; i++) {
+          checksum.reset();
+          int dLen = Math.min(dLeft, bytesPerChecksum);
+          checksum.update(buf, dOff, dLen);
+          if (!checksum.compare(buf, cOff)) {
+            throw new ChecksumException("Checksum failed at " + 
+                                        (offset + len - dLeft), len);
+          }
+          dLeft -= dLen;
+          dOff += dLen;
+          cOff += checksumSize;
+        }
+      }
+      //writing is done below (mainly to handle IOException)
+    }
+    
+    try {
+      if (blockInPosition >= 0) {
+        //use transferTo(). Checks on out and blockIn are already done. 
+
+        SocketOutputStream sockOut = (SocketOutputStream)out;
+        //first write the packet
+        sockOut.write(buf, 0, dataOff);
+        // no need to flush. since we know out is not a buffered stream. 
+
+        sockOut.transferToFully(((FileInputStream)blockIn).getChannel(), 
+                                blockInPosition, len);
+
+        blockInPosition += len;
+      } else {
+        // normal transfer
+        out.write(buf, 0, dataOff + len);
+      }
+      
+    } catch (IOException e) {
+      /* exception while writing to the client (well, with transferTo(),
+       * it could also be while reading from the local file).
+       */
+      throw ioeToSocketException(e);
+    }
+
+    if (throttler != null) { // rebalancing so throttle
+      throttler.throttle(packetLen);
+    }
+
+    return len;
+  }
+
+  /**
+   * sendBlock() is used to read block and its metadata and stream the data to
+   * either a client or to another datanode. 
+   * 
+   * @param out  stream to which the block is written to
+   * @param baseStream optional. if non-null, <code>out</code> is assumed to 
+   *        be a wrapper over this stream. This enables optimizations for
+   *        sending the data, e.g. 
+   *        {@link SocketOutputStream#transferToFully(FileChannel, 
+   *        long, int)}.
+   * @param throttler for sending data.
+   * @return total bytes reads, including crc.
+   */
+  long sendBlock(DataOutputStream out, OutputStream baseStream, 
+                 BlockTransferThrottler throttler) throws IOException {
+    if( out == null ) {
+      throw new IOException( "out stream is null" );
+    }
+    this.throttler = throttler;
+
+    long initialOffset = offset;
+    long totalRead = 0;
+    OutputStream streamForSendChunks = out;
+    
+    final long startTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
+    try {
+      try {
+        checksum.writeHeader(out);
+        if ( chunkOffsetOK ) {
+          out.writeLong( offset );
+        }
+        out.flush();
+      } catch (IOException e) { //socket error
+        throw ioeToSocketException(e);
+      }
+      
+      int maxChunksPerPacket;
+      int pktSize = DataNode.PKT_HEADER_LEN + SIZE_OF_INTEGER;
+      
+      if (transferToAllowed && !verifyChecksum && 
+          baseStream instanceof SocketOutputStream && 
+          blockIn instanceof FileInputStream) {
+        
+        FileChannel fileChannel = ((FileInputStream)blockIn).getChannel();
+        
+        // blockInPosition also indicates sendChunks() uses transferTo.
+        blockInPosition = fileChannel.position();
+        streamForSendChunks = baseStream;
+        
+        // assure a mininum buffer size.
+        maxChunksPerPacket = (Math.max(BUFFER_SIZE, 
+                                       MIN_BUFFER_WITH_TRANSFERTO)
+                              + bytesPerChecksum - 1)/bytesPerChecksum;
+        
+        // allocate smaller buffer while using transferTo(). 
+        pktSize += checksumSize * maxChunksPerPacket;
+      } else {
+        maxChunksPerPacket = Math.max(1,
+                 (BUFFER_SIZE + bytesPerChecksum - 1)/bytesPerChecksum);
+        pktSize += (bytesPerChecksum + checksumSize) * maxChunksPerPacket;
+      }
+
+      ByteBuffer pktBuf = ByteBuffer.allocate(pktSize);
+
+      while (endOffset > offset) {
+        long len = sendChunks(pktBuf, maxChunksPerPacket, 
+                              streamForSendChunks);
+        offset += len;
+        totalRead += len + ((len + bytesPerChecksum - 1)/bytesPerChecksum*
+                            checksumSize);
+        seqno++;
+      }
+      try {
+        out.writeInt(0); // mark the end of block        
+        out.flush();
+      } catch (IOException e) { //socket error
+        throw ioeToSocketException(e);
+      }
+    } finally {
+      if (clientTraceFmt != null) {
+        final long endTime = System.nanoTime();
+        ClientTraceLog.info(String.format(clientTraceFmt, totalRead, initialOffset, endTime - startTime));
+      }
+      close();
+    }
+
+    blockReadFully = (initialOffset == 0 && offset >= blockLength);
+
+    return totalRead;
+  }
+  
+  boolean isBlockReadFully() {
+    return blockReadFully;
+  }
+}

+ 111 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/BlockTransferThrottler.java

@@ -0,0 +1,111 @@
+/**
+ * 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.datanode;
+
+/** 
+ * a class to throttle the block transfers.
+ * This class is thread safe. It can be shared by multiple threads.
+ * The parameter bandwidthPerSec specifies the total bandwidth shared by
+ * threads.
+ */
+class BlockTransferThrottler {
+  private long period;          // period over which bw is imposed
+  private long periodExtension; // Max period over which bw accumulates.
+  private long bytesPerPeriod; // total number of bytes can be sent in each period
+  private long curPeriodStart; // current period starting time
+  private long curReserve;     // remaining bytes can be sent in the period
+  private long bytesAlreadyUsed;
+
+  /** Constructor 
+   * @param bandwidthPerSec bandwidth allowed in bytes per second. 
+   */
+  BlockTransferThrottler(long bandwidthPerSec) {
+    this(500, bandwidthPerSec);  // by default throttling period is 500ms 
+  }
+
+  /**
+   * Constructor
+   * @param period in milliseconds. Bandwidth is enforced over this
+   *        period.
+   * @param bandwidthPerSec bandwidth allowed in bytes per second. 
+   */
+  BlockTransferThrottler(long period, long bandwidthPerSec) {
+    this.curPeriodStart = System.currentTimeMillis();
+    this.period = period;
+    this.curReserve = this.bytesPerPeriod = bandwidthPerSec*period/1000;
+    this.periodExtension = period*3;
+  }
+
+  /**
+   * @return current throttle bandwidth in bytes per second.
+   */
+  synchronized long getBandwidth() {
+    return bytesPerPeriod*1000/period;
+  }
+  
+  /**
+   * Sets throttle bandwidth. This takes affect latest by the end of current
+   * period.
+   * 
+   * @param bytesPerSecond 
+   */
+  synchronized void setBandwidth(long bytesPerSecond) {
+    if ( bytesPerSecond <= 0 ) {
+      throw new IllegalArgumentException("" + bytesPerSecond);
+    }
+    bytesPerPeriod = bytesPerSecond*period/1000;
+  }
+  
+  /** Given the numOfBytes sent/received since last time throttle was called,
+   * make the current thread sleep if I/O rate is too fast
+   * compared to the given bandwidth.
+   *
+   * @param numOfBytes
+   *     number of bytes sent/received since last time throttle was called
+   */
+  synchronized void throttle(long numOfBytes) {
+    if ( numOfBytes <= 0 ) {
+      return;
+    }
+
+    curReserve -= numOfBytes;
+    bytesAlreadyUsed += numOfBytes;
+
+    while (curReserve <= 0) {
+      long now = System.currentTimeMillis();
+      long curPeriodEnd = curPeriodStart + period;
+
+      if ( now < curPeriodEnd ) {
+        // Wait for next period so that curReserve can be increased.
+        try {
+          wait( curPeriodEnd - now );
+        } catch (InterruptedException ignored) {}
+      } else if ( now <  (curPeriodStart + periodExtension)) {
+        curPeriodStart = curPeriodEnd;
+        curReserve += bytesPerPeriod;
+      } else {
+        // discard the prev period. Throttler might not have
+        // been used for a long time.
+        curPeriodStart = now;
+        curReserve = bytesPerPeriod - bytesAlreadyUsed;
+      }
+    }
+
+    bytesAlreadyUsed -= numOfBytes;
+  }
+}

+ 985 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java

@@ -0,0 +1,985 @@
+/**
+ * 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.datanode;
+
+import java.io.BufferedReader;
+import java.io.Closeable;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.TreeSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+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.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Performs two types of scanning:
+ * <li> Gets block files from the data directories and reconciles the
+ * difference between the blocks on the disk and in memory in
+ * {@link FSDataset}</li>
+ * <li> Scans the data directories for block files and verifies that
+ * the files are not corrupt</li>
+ * This keeps track of blocks and their last verification times.
+ * Currently it does not modify the metadata for block.
+ */
+
+class DataBlockScanner implements Runnable {
+  
+  public static final Log LOG = LogFactory.getLog(DataBlockScanner.class);
+  
+  private static final int MAX_SCAN_RATE = 8 * 1024 * 1024; // 8MB per sec
+  private static final int MIN_SCAN_RATE = 1 * 1024 * 1024; // 1MB per sec
+  
+  static final long DEFAULT_SCAN_PERIOD_HOURS = 21*24L; // three weeks
+  private static final long ONE_DAY = 24*3600*1000L;
+  
+  static final DateFormat dateFormat = 
+                    new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
+  
+  static final String verificationLogFile = "dncp_block_verification.log";
+  static final int verficationLogLimit = 5; // * numBlocks.
+
+  private long scanPeriod = DEFAULT_SCAN_PERIOD_HOURS * 3600 * 1000;
+  DataNode datanode;
+  FSDataset dataset;
+  
+  // sorted set
+  TreeSet<BlockScanInfo> blockInfoSet;
+  HashMap<Block, BlockScanInfo> blockMap;
+  
+  long totalScans = 0;
+  long totalVerifications = 0; // includes remote verification by clients.
+  long totalScanErrors = 0;
+  long totalTransientErrors = 0;
+  
+  long currentPeriodStart = System.currentTimeMillis();
+  long bytesLeft = 0; // Bytes to scan in this period
+  long totalBytesToScan = 0;
+  
+  private LogFileHandler verificationLog;
+  
+  Random random = new Random();
+  
+  BlockTransferThrottler throttler = null;
+  
+  // Reconciles blocks on disk to blocks in memory
+  DirectoryScanner dirScanner;
+
+  private static enum ScanType {
+    REMOTE_READ,           // Verified when a block read by a client etc
+    VERIFICATION_SCAN,     // scanned as part of periodic verfication
+    NONE,
+  }
+  
+  static class BlockScanInfo implements Comparable<BlockScanInfo> {
+    Block block;
+    long lastScanTime = 0;
+    long lastLogTime = 0;
+    ScanType lastScanType = ScanType.NONE; 
+    boolean lastScanOk = true;
+    
+    BlockScanInfo(Block block) {
+      this.block = block;
+    }
+    
+    public int hashCode() {
+      return block.hashCode();
+    }
+    
+    public boolean equals(Object other) {
+      return other instanceof BlockScanInfo &&
+             compareTo((BlockScanInfo)other) == 0;
+    }
+    
+    long getLastScanTime() {
+      return ( lastScanType == ScanType.NONE) ? 0 : lastScanTime;
+    }
+    
+    public int compareTo(BlockScanInfo other) {
+      long t1 = lastScanTime;
+      long t2 = other.lastScanTime;
+      return ( t1 < t2 ) ? -1 : 
+                          (( t1 > t2 ) ? 1 : block.compareTo(other.block)); 
+    }
+  }
+  
+  DataBlockScanner(DataNode datanode, FSDataset dataset, Configuration conf) {
+    this.datanode = datanode;
+    this.dataset = dataset;
+    scanPeriod = conf.getInt("dfs.datanode.scan.period.hours", 0);
+    if ( scanPeriod <= 0 ) {
+      scanPeriod = DEFAULT_SCAN_PERIOD_HOURS;
+    }
+    scanPeriod *= 3600 * 1000;
+    // initialized when the scanner thread is started.
+
+    dirScanner = new DirectoryScanner(dataset, conf);
+  }
+  
+  private synchronized boolean isInitiliazed() {
+    return throttler != null;
+  }
+  
+  private void updateBytesToScan(long len, long lastScanTime) {
+    // len could be negative when a block is deleted.
+    totalBytesToScan += len;
+    if ( lastScanTime < currentPeriodStart ) {
+      bytesLeft += len;
+    }
+    // Should we change throttler bandwidth every time bytesLeft changes?
+    // not really required.
+  }
+  
+  private synchronized void addBlockInfo(BlockScanInfo info) {
+    boolean added = blockInfoSet.add(info);
+    blockMap.put(info.block, info);
+    
+    if ( added ) {
+      LogFileHandler log = verificationLog;
+      if (log != null) {
+        log.setMaxNumLines(blockMap.size() * verficationLogLimit);
+      }
+      updateBytesToScan(info.block.getNumBytes(), info.lastScanTime);
+    }
+  }
+  
+  private synchronized void delBlockInfo(BlockScanInfo info) {
+    boolean exists = blockInfoSet.remove(info);
+    blockMap.remove(info.block);
+    if ( exists ) {
+      LogFileHandler log = verificationLog;
+      if (log != null) {
+        log.setMaxNumLines(blockMap.size() * verficationLogLimit);
+      }
+      updateBytesToScan(-info.block.getNumBytes(), info.lastScanTime);
+    }
+  }
+  
+  /** Update blockMap by the given LogEntry */
+  private synchronized void updateBlockInfo(LogEntry e) {
+    BlockScanInfo info = blockMap.get(new Block(e.blockId, 0, e.genStamp));
+    
+    if(info != null && e.verificationTime > 0 && 
+        info.lastScanTime < e.verificationTime) {
+      delBlockInfo(info);
+      info.lastScanTime = e.verificationTime;
+      info.lastScanType = ScanType.VERIFICATION_SCAN;
+      addBlockInfo(info);
+    }
+  }
+
+  private void init() {
+    
+    // get the list of blocks and arrange them in random order
+    Block arr[] = dataset.getBlockReport();
+    Collections.shuffle(Arrays.asList(arr));
+    
+    blockInfoSet = new TreeSet<BlockScanInfo>();
+    blockMap = new HashMap<Block, BlockScanInfo>();
+    
+    long scanTime = -1;
+    for (Block block : arr) {
+      BlockScanInfo info = new BlockScanInfo( block );
+      info.lastScanTime = scanTime--; 
+      //still keep 'info.lastScanType' to NONE.
+      addBlockInfo(info);
+    }
+
+    /* Pick the first directory that has any existing scanner log.
+     * otherwise, pick the first directory.
+     */
+    File dir = null;
+    FSDataset.FSVolume[] volumes = dataset.volumes.volumes;
+    for(FSDataset.FSVolume vol : volumes) {
+      if (LogFileHandler.isFilePresent(vol.getDir(), verificationLogFile)) {
+        dir = vol.getDir();
+        break;
+      }
+    }
+    if (dir == null) {
+      dir = volumes[0].getDir();
+    }
+    
+    try {
+      // max lines will be updated later during initialization.
+      verificationLog = new LogFileHandler(dir, verificationLogFile, 100);
+    } catch (IOException e) {
+      LOG.warn("Could not open verfication log. " +
+               "Verification times are not stored.");
+    }
+    
+    synchronized (this) {
+      throttler = new BlockTransferThrottler(200, MAX_SCAN_RATE);
+    }
+  }
+
+  private synchronized long getNewBlockScanTime() {
+    /* If there are a lot of blocks, this returns a random time with in 
+     * the scan period. Otherwise something sooner.
+     */
+    long period = Math.min(scanPeriod, 
+                           Math.max(blockMap.size(),1) * 600 * 1000L);
+    return System.currentTimeMillis() - scanPeriod + 
+           random.nextInt((int)period);    
+  }
+
+  /** Adds block to list of blocks */
+  synchronized void addBlock(Block block) {
+    if (!isInitiliazed()) {
+      return;
+    }
+    
+    BlockScanInfo info = blockMap.get(block);
+    if ( info != null ) {
+      LOG.warn("Adding an already existing block " + block);
+      delBlockInfo(info);
+    }
+    
+    info = new BlockScanInfo(block);    
+    info.lastScanTime = getNewBlockScanTime();
+    
+    addBlockInfo(info);
+    adjustThrottler();
+  }
+  
+  /** Deletes the block from internal structures */
+  synchronized void deleteBlock(Block block) {
+    if (!isInitiliazed()) {
+      return;
+    }
+    BlockScanInfo info = blockMap.get(block);
+    if ( info != null ) {
+      delBlockInfo(info);
+    }
+  }
+
+  /** @return the last scan time */
+  synchronized long getLastScanTime(Block block) {
+    if (!isInitiliazed()) {
+      return 0;
+    }
+    BlockScanInfo info = blockMap.get(block);
+    return info == null? 0: info.lastScanTime;
+  }
+
+  /** Deletes blocks from internal structures */
+  void deleteBlocks(Block[] blocks) {
+    for ( Block b : blocks ) {
+      deleteBlock(b);
+    }
+  }
+  
+  void verifiedByClient(Block block) {
+    updateScanStatus(block, ScanType.REMOTE_READ, true);
+  }
+  
+  private synchronized void updateScanStatus(Block block, 
+                                             ScanType type,
+                                             boolean scanOk) {
+    BlockScanInfo info = blockMap.get(block);
+    
+    if ( info != null ) {
+      delBlockInfo(info);
+    } else {
+      // It might already be removed. Thats ok, it will be caught next time.
+      info = new BlockScanInfo(block);
+    }
+    
+    long now = System.currentTimeMillis();
+    info.lastScanType = type;
+    info.lastScanTime = now;
+    info.lastScanOk = scanOk;
+    addBlockInfo(info);
+    
+    if (type == ScanType.REMOTE_READ) {
+      totalVerifications++;
+    }
+        
+    // Don't update meta data too often in case of REMOTE_READ
+    // of if the verification failed.
+    long diff = now - info.lastLogTime;
+    if (!scanOk || (type == ScanType.REMOTE_READ &&
+                    diff < scanPeriod/3 && diff < ONE_DAY)) {
+      return;
+    }
+    
+    info.lastLogTime = now;
+    LogFileHandler log = verificationLog;
+    if (log != null) {
+      log.appendLine(LogEntry.newEnry(block, now));
+    }
+  }
+  
+  private void handleScanFailure(Block block) {
+    
+    LOG.info("Reporting bad block " + block + " to namenode.");
+    
+    try {
+      DatanodeInfo[] dnArr = { new DatanodeInfo(datanode.dnRegistration) };
+      LocatedBlock[] blocks = { new LocatedBlock(block, dnArr) }; 
+      datanode.namenode.reportBadBlocks(blocks);
+    } catch (IOException e){
+      /* One common reason is that NameNode could be in safe mode.
+       * Should we keep on retrying in that case?
+       */
+      LOG.warn("Failed to report bad block " + block + " to namenode : " +
+               " Exception : " + StringUtils.stringifyException(e));
+    }
+  }
+  
+  static private class LogEntry {
+    long blockId = -1;
+    long verificationTime = -1;
+    long genStamp = Block.GRANDFATHER_GENERATION_STAMP;
+    
+    /**
+     * The format consists of single line with multiple entries. each 
+     * entry is in the form : name="value".
+     * This simple text and easily extendable and easily parseable with a
+     * regex.
+     */
+    private static Pattern entryPattern = 
+      Pattern.compile("\\G\\s*([^=\\p{Space}]+)=\"(.*?)\"\\s*");
+    
+    static String newEnry(Block block, long time) {
+      return "date=\"" + dateFormat.format(new Date(time)) + "\"\t " +
+             "time=\"" + time + "\"\t " +
+             "genstamp=\"" + block.getGenerationStamp() + "\"\t " +
+             "id=\"" + block.getBlockId() +"\"";
+    }
+    
+    static LogEntry parseEntry(String line) {
+      LogEntry entry = new LogEntry();
+      
+      Matcher matcher = entryPattern.matcher(line);
+      while (matcher.find()) {
+        String name = matcher.group(1);
+        String value = matcher.group(2);
+        
+        try {
+          if (name.equals("id")) {
+            entry.blockId = Long.valueOf(value);
+          } else if (name.equals("time")) {
+            entry.verificationTime = Long.valueOf(value);
+          } else if (name.equals("genstamp")) {
+            entry.genStamp = Long.valueOf(value);
+          }
+        } catch(NumberFormatException nfe) {
+          LOG.warn("Cannot parse line: " + line, nfe);
+          return null;
+        }
+      }
+      
+      return entry;
+    }
+  }
+  
+  private synchronized void adjustThrottler() {
+    long timeLeft = currentPeriodStart+scanPeriod - System.currentTimeMillis();
+    long bw = Math.max(bytesLeft*1000/timeLeft, MIN_SCAN_RATE);
+    throttler.setBandwidth(Math.min(bw, MAX_SCAN_RATE));
+  }
+  
+  private void verifyBlock(Block block) {
+    
+    BlockSender blockSender = null;
+
+    /* In case of failure, attempt to read second time to reduce
+     * transient errors. How do we flush block data from kernel 
+     * buffers before the second read? 
+     */
+    for (int i=0; i<2; i++) {
+      boolean second = (i > 0);
+      
+      try {
+        adjustThrottler();
+        
+        blockSender = new BlockSender(block, 0, -1, false, 
+                                               false, true, datanode);
+
+        DataOutputStream out = 
+                new DataOutputStream(new IOUtils.NullOutputStream());
+        
+        blockSender.sendBlock(out, null, throttler);
+
+        LOG.info((second ? "Second " : "") +
+                 "Verification succeeded for " + block);
+        
+        if ( second ) {
+          totalTransientErrors++;
+        }
+        
+        updateScanStatus(block, ScanType.VERIFICATION_SCAN, true);
+
+        return;
+      } catch (IOException e) {
+
+        totalScanErrors++;
+        updateScanStatus(block, ScanType.VERIFICATION_SCAN, false);
+
+        // If the block does not exists anymore, then its not an error
+        if ( dataset.getFile(block) == null ) {
+          LOG.info("Verification failed for " + block + ". Its ok since " +
+          "it not in datanode dataset anymore.");
+          deleteBlock(block);
+          return;
+        }
+
+        LOG.warn((second ? "Second " : "First ") + 
+                 "Verification failed for " + block + ". Exception : " +
+                 StringUtils.stringifyException(e));
+        
+        if (second) {
+          datanode.getMetrics().blockVerificationFailures.inc(); 
+          handleScanFailure(block);
+          return;
+        } 
+      } finally {
+        IOUtils.closeStream(blockSender);
+        datanode.getMetrics().blocksVerified.inc();
+        totalScans++;
+        totalVerifications++;
+      }
+    }
+  }
+  
+  private synchronized long getEarliestScanTime() {
+    if ( blockInfoSet.size() > 0 ) {
+      return blockInfoSet.first().lastScanTime;
+    }
+    return Long.MAX_VALUE; 
+  }
+  
+  // Picks one block and verifies it
+  private void verifyFirstBlock() {
+    Block block = null;
+    synchronized (this) {
+      if ( blockInfoSet.size() > 0 ) {
+        block = blockInfoSet.first().block;
+      }
+    }
+    
+    if ( block != null ) {
+      verifyBlock(block);
+    }
+  }
+  
+  /** returns false if the process was interrupted
+   * because the thread is marked to exit.
+   */
+  private boolean assignInitialVerificationTimes() {
+    int numBlocks = 1;
+    LogFileHandler log = null;
+    synchronized (this) {
+      log = verificationLog;
+      numBlocks = Math.max(blockMap.size(), 1);
+    }
+    
+    //First udpates the last verification times from the log file.
+    LogFileHandler.Reader logReader = null;
+    try {
+      if (log != null) {
+        logReader = log.new Reader(false);
+      }
+    } catch (IOException e) {
+      LOG.warn("Could not read previous verification times : " +
+               StringUtils.stringifyException(e));
+    }
+    
+    if (log != null) {
+      log.updateCurNumLines();
+    }
+    
+    try {
+    // update verification times from the verificationLog.
+    while (logReader != null && logReader.hasNext()) {
+      if (!datanode.shouldRun || Thread.interrupted()) {
+        return false;
+      }
+      LogEntry entry = LogEntry.parseEntry(logReader.next());
+      if (entry != null) {
+        updateBlockInfo(entry);
+      }
+    }
+    } finally {
+      IOUtils.closeStream(logReader);
+    }
+    
+    /* Initially spread the block reads over half of 
+     * MIN_SCAN_PERIOD so that we don't keep scanning the 
+     * blocks too quickly when restarted.
+     */
+    long verifyInterval = (long) (Math.min( scanPeriod/2.0/numBlocks,
+                                            10*60*1000 ));
+    long lastScanTime = System.currentTimeMillis() - scanPeriod;
+    
+    /* Before this loop, entries in blockInfoSet that are not
+     * updated above have lastScanTime of <= 0 . Loop until first entry has
+     * lastModificationTime > 0.
+     */    
+    synchronized (this) {
+      if (blockInfoSet.size() > 0 ) {
+        BlockScanInfo info;
+        while ((info =  blockInfoSet.first()).lastScanTime < 0) {
+          delBlockInfo(info);        
+          info.lastScanTime = lastScanTime;
+          lastScanTime += verifyInterval;
+          addBlockInfo(info);
+        }
+      }
+    }
+    
+    return true;
+  }
+  
+  private synchronized void startNewPeriod() {
+    LOG.info("Starting a new period : work left in prev period : " +
+             String.format("%.2f%%", (bytesLeft * 100.0)/totalBytesToScan));
+    // reset the byte counts :
+    bytesLeft = totalBytesToScan;
+    currentPeriodStart = System.currentTimeMillis();
+  }
+  
+  public void run() {
+    try {
+      
+      init();
+      
+      //Read last verification times
+      if (!assignInitialVerificationTimes()) {
+        return;
+      }
+      
+      adjustThrottler();
+      
+      while (datanode.shouldRun && !Thread.interrupted()) {
+        long now = System.currentTimeMillis();
+        synchronized (this) {
+          if ( now >= (currentPeriodStart + scanPeriod)) {
+            startNewPeriod();
+          }
+        }
+        if (dirScanner.newScanPeriod(now)) {
+          dirScanner.reconcile();
+          now = System.currentTimeMillis();
+        }
+        if ( (now - getEarliestScanTime()) >= scanPeriod ) {
+          verifyFirstBlock();
+        } else {
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException ignored) {}
+        }
+      }
+    } catch (RuntimeException e) {
+      LOG.warn("RuntimeException during DataBlockScanner.run() : " +
+               StringUtils.stringifyException(e));
+      throw e;
+    } finally {
+      shutdown();
+      LOG.info("Exiting DataBlockScanner thread.");
+    }
+  }
+  
+  synchronized void shutdown() {
+    LogFileHandler log = verificationLog;
+    verificationLog = null;
+    if (log != null) {
+      log.close();
+    }
+  }
+  
+  synchronized void printBlockReport(StringBuilder buffer, 
+                                     boolean summaryOnly) {
+    long oneHour = 3600*1000;
+    long oneDay = 24*oneHour;
+    long oneWeek = 7*oneDay;
+    long fourWeeks = 4*oneWeek;
+    
+    int inOneHour = 0;
+    int inOneDay = 0;
+    int inOneWeek = 0;
+    int inFourWeeks = 0;
+    int inScanPeriod = 0;
+    int neverScanned = 0;
+    
+    int total = blockInfoSet.size();
+    
+    long now = System.currentTimeMillis();
+    
+    Date date = new Date();
+    
+    for(Iterator<BlockScanInfo> it = blockInfoSet.iterator(); it.hasNext();) {
+      BlockScanInfo info = it.next();
+      
+      long scanTime = info.getLastScanTime();
+      long diff = now - scanTime;
+      
+      if (diff <= oneHour) inOneHour++;
+      if (diff <= oneDay) inOneDay++;
+      if (diff <= oneWeek) inOneWeek++;
+      if (diff <= fourWeeks) inFourWeeks++;
+      if (diff <= scanPeriod) inScanPeriod++;      
+      if (scanTime <= 0) neverScanned++;
+      
+      if (!summaryOnly) {
+        date.setTime(scanTime);
+        String scanType = 
+          (info.lastScanType == ScanType.REMOTE_READ) ? "remote" : 
+            ((info.lastScanType == ScanType.VERIFICATION_SCAN) ? "local" :
+              "none");
+        buffer.append(String.format("%-26s : status : %-6s type : %-6s" +
+        		                        " scan time : " +
+                                    "%-15d %s\n", info.block, 
+                                    (info.lastScanOk ? "ok" : "failed"),
+                                    scanType, scanTime,
+                                    (scanTime <= 0) ? "not yet verified" : 
+                                      dateFormat.format(date)));
+      }
+    }
+    
+    double pctPeriodLeft = (scanPeriod + currentPeriodStart - now)
+                           *100.0/scanPeriod;
+    double pctProgress = (totalBytesToScan == 0) ? 100 :
+                         (totalBytesToScan-bytesLeft)*10000.0/totalBytesToScan/
+                         (100-pctPeriodLeft+1e-10);
+    
+    buffer.append(String.format("\nTotal Blocks                 : %6d" +
+                                "\nVerified in last hour        : %6d" +
+                                "\nVerified in last day         : %6d" +
+                                "\nVerified in last week        : %6d" +
+                                "\nVerified in last four weeks  : %6d" +
+                                "\nVerified in SCAN_PERIOD      : %6d" +
+                                "\nNot yet verified             : %6d" +
+                                "\nVerified since restart       : %6d" +
+                                "\nScans since restart          : %6d" +
+                                "\nScan errors since restart    : %6d" +
+                                "\nTransient scan errors        : %6d" +
+                                "\nCurrent scan rate limit KBps : %6d" +
+                                "\nProgress this period         : %6.0f%%" +
+                                "\nTime left in cur period      : %6.2f%%" +
+                                "\n", 
+                                total, inOneHour, inOneDay, inOneWeek,
+                                inFourWeeks, inScanPeriod, neverScanned,
+                                totalVerifications, totalScans, 
+                                totalScanErrors, totalTransientErrors, 
+                                Math.round(throttler.getBandwidth()/1024.0),
+                                pctProgress, pctPeriodLeft));
+  }
+  
+  /**
+   * This class takes care of log file used to store the last verification
+   * times of the blocks. It rolls the current file when it is too big etc.
+   * If there is an error while writing, it stops updating with an error
+   * message.
+   */
+  private static class LogFileHandler {
+    
+    private static final String curFileSuffix = ".curr";
+    private static final String prevFileSuffix = ".prev";
+    
+    // Don't roll files more often than this
+    private static final long minRollingPeriod = 6 * 3600 * 1000L; // 6 hours
+    private static final long minWarnPeriod = minRollingPeriod;
+    private static final int minLineLimit = 1000;
+    
+    
+    static boolean isFilePresent(File dir, String filePrefix) {
+      return new File(dir, filePrefix + curFileSuffix).exists() ||
+             new File(dir, filePrefix + prevFileSuffix).exists();
+    }
+    private File curFile;
+    private File prevFile;
+    
+    private int maxNumLines = -1; // not very hard limit on number of lines.
+    private int curNumLines = -1;
+    
+    long lastWarningTime = 0;
+    
+    private PrintStream out;
+    
+    int numReaders = 0;
+        
+    /**
+     * Opens the log file for appending.
+     * Note that rolling will happen only after "updateLineCount()" is 
+     * called. This is so that line count could be updated in a separate
+     * thread without delaying start up.
+     * 
+     * @param dir where the logs files are located.
+     * @param filePrefix prefix of the file.
+     * @param maxNumLines max lines in a file (its a soft limit).
+     * @throws IOException
+     */
+    LogFileHandler(File dir, String filePrefix, int maxNumLines) 
+                                                throws IOException {
+      curFile = new File(dir, filePrefix + curFileSuffix);
+      prevFile = new File(dir, filePrefix + prevFileSuffix);
+      openCurFile();
+      curNumLines = -1;
+      setMaxNumLines(maxNumLines);
+    }
+    
+    // setting takes affect when next entry is added.
+    synchronized void setMaxNumLines(int maxNumLines) {
+      this.maxNumLines = Math.max(maxNumLines, minLineLimit);
+    }
+    
+    /**
+     * Append "\n" + line.
+     * If the log file need to be rolled, it will done after 
+     * appending the text.
+     * This does not throw IOException when there is an error while 
+     * appending. Currently does not throw an error even if rolling 
+     * fails (may be it should?).
+     * return true if append was successful.
+     */
+    synchronized boolean appendLine(String line) {
+      out.println();
+      out.print(line);
+      curNumLines += (curNumLines < 0) ? -1 : 1;
+      try {
+        rollIfRequired();
+      } catch (IOException e) {
+        warn("Rolling failed for " + curFile + " : " + e.getMessage());
+        return false;
+      }
+      return true;
+    }
+    
+    //warns only once in a while
+    synchronized private void warn(String msg) {
+      long now = System.currentTimeMillis();
+      if ((now - lastWarningTime) >= minWarnPeriod) {
+        lastWarningTime = now;
+        LOG.warn(msg);
+      }
+    }
+    
+    private synchronized void openCurFile() throws FileNotFoundException {
+      close();
+      out = new PrintStream(new FileOutputStream(curFile, true));
+    }
+    
+    //This reads the current file and updates the count.
+    void updateCurNumLines() {
+      int count = 0;
+      Reader it = null;
+      try {
+        for(it = new Reader(true); it.hasNext(); count++) {
+          it.next();
+        }
+      } catch (IOException e) {
+        
+      } finally {
+        synchronized (this) {
+          curNumLines = count;
+        }
+        IOUtils.closeStream(it);
+      }
+    }
+    
+    private void rollIfRequired() throws IOException {
+      if (curNumLines < maxNumLines || numReaders > 0) {
+        return;
+      }
+      
+      long now = System.currentTimeMillis();
+      if (now < minRollingPeriod) {
+        return;
+      }
+      
+      if (!prevFile.delete() && prevFile.exists()) {
+        throw new IOException("Could not delete " + prevFile);
+      }
+      
+      close();
+
+      if (!curFile.renameTo(prevFile)) {
+        openCurFile();
+        throw new IOException("Could not rename " + curFile + 
+                              " to " + prevFile);
+      }
+      
+      openCurFile();
+      updateCurNumLines();
+    }
+    
+    synchronized void close() {
+      if (out != null) {
+        out.close();
+        out = null;
+      }
+    }
+    
+    /**
+     * This is used to read the lines in order.
+     * If the data is not read completely (i.e, untill hasNext() returns
+     * false), it needs to be explicitly 
+     */
+    private class Reader implements Iterator<String>, Closeable {
+      
+      BufferedReader reader;
+      File file;
+      String line;
+      boolean closed = false;
+      
+      private Reader(boolean skipPrevFile) throws IOException {
+        synchronized (LogFileHandler.this) {
+          numReaders++; 
+        }
+        reader = null;
+        file = (skipPrevFile) ? curFile : prevFile;
+        readNext();        
+      }
+      
+      private boolean openFile() throws IOException {
+
+        for(int i=0; i<2; i++) {
+          if (reader != null || i > 0) {
+            // move to next file
+            file = (file == prevFile) ? curFile : null;
+          }
+          if (file == null) {
+            return false;
+          }
+          if (file.exists()) {
+            break;
+          }
+        }
+        
+        if (reader != null ) {
+          reader.close();
+          reader = null;
+        }
+        
+        reader = new BufferedReader(new FileReader(file));
+        return true;
+      }
+      
+      // read next line if possible.
+      private void readNext() throws IOException {
+        line = null;
+        try {
+          if (reader != null && (line = reader.readLine()) != null) {
+            return;
+          }
+          if (line == null) {
+            // move to the next file.
+            if (openFile()) {
+              readNext();
+            }
+          }
+        } finally {
+          if (!hasNext()) {
+            close();
+          }
+        }
+      }
+      
+      public boolean hasNext() {
+        return line != null;
+      }
+
+      public String next() {
+        String curLine = line;
+        try {
+          readNext();
+        } catch (IOException e) {
+          LOG.info("Could not reade next line in LogHandler : " +
+                   StringUtils.stringifyException(e));
+        }
+        return curLine;
+      }
+
+      public void remove() {
+        throw new RuntimeException("remove() is not supported.");
+      }
+
+      public void close() throws IOException {
+        if (!closed) {
+          try {
+            if (reader != null) {
+              reader.close();
+            }
+          } finally {
+            file = null;
+            reader = null;
+            closed = true;
+            synchronized (LogFileHandler.this) {
+              numReaders--;
+              assert(numReaders >= 0);
+            }
+          }
+        }
+      }
+    }    
+  }
+  
+  public static class Servlet extends HttpServlet {
+    private static final long serialVersionUID = 1L;
+
+    public void doGet(HttpServletRequest request, 
+                      HttpServletResponse response) throws IOException {
+      
+      response.setContentType("text/plain");
+      
+      DataBlockScanner blockScanner = (DataBlockScanner)  
+          getServletContext().getAttribute("datanode.blockScanner");
+      
+      boolean summary = (request.getParameter("listblocks") == null);
+      
+      StringBuilder buffer = new StringBuilder(8*1024);
+      if (blockScanner == null) {
+        buffer.append("Periodic block scanner is not running. " +
+                      "Please check the datanode log if this is unexpected.");
+      } else if (blockScanner.isInitiliazed()) {
+        blockScanner.printBlockReport(buffer, summary);
+      } else {
+        buffer.append("Periodic block scanner is not yet initialized. " +
+                      "Please check back again after some time.");
+      }
+      response.getWriter().write(buffer.toString()); // extra copy!
+    }
+  }
+}

+ 1683 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -0,0 +1,1683 @@
+/**
+ * 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.datanode;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+import java.security.NoSuchAlgorithmException;
+import java.security.SecureRandom;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hdfs.HDFSPolicyProvider;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+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.protocol.UnregisteredNodeException;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.StreamFile;
+import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockMetaDataInfo;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
+import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
+import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.DNS;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessToken;
+import org.apache.hadoop.security.AccessTokenHandler;
+import org.apache.hadoop.security.ExportedAccessKeys;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.authorize.ConfiguredPolicy;
+import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.DiskChecker;
+import org.apache.hadoop.util.ServicePlugin;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+
+/**********************************************************
+ * DataNode is a class (and program) that stores a set of
+ * blocks for a DFS deployment.  A single deployment can
+ * have one or many DataNodes.  Each DataNode communicates
+ * regularly with a single NameNode.  It also communicates
+ * with client code and other DataNodes from time to time.
+ *
+ * DataNodes store a series of named blocks.  The DataNode
+ * allows client code to read these blocks, or to write new
+ * block data.  The DataNode may also, in response to instructions
+ * from its NameNode, delete blocks or copy blocks to/from other
+ * DataNodes.
+ *
+ * The DataNode maintains just one critical table:
+ *   block-> stream of bytes (of BLOCK_SIZE or less)
+ *
+ * This info is stored on a local disk.  The DataNode
+ * reports the table's contents to the NameNode upon startup
+ * and every so often afterwards.
+ *
+ * DataNodes spend their lives in an endless loop of asking
+ * the NameNode for something to do.  A NameNode cannot connect
+ * to a DataNode directly; a NameNode simply returns values from
+ * functions invoked by a DataNode.
+ *
+ * DataNodes maintain an open server socket so that client code 
+ * or other DataNodes can read/write data.  The host/port for
+ * this server is reported to the NameNode, which then sends that
+ * information to clients or other DataNodes that might be interested.
+ *
+ **********************************************************/
+public class DataNode extends Configured 
+    implements InterDatanodeProtocol, ClientDatanodeProtocol, FSConstants, Runnable {
+  public static final Log LOG = LogFactory.getLog(DataNode.class);
+  
+  static{
+    Configuration.addDefaultResource("hdfs-default.xml");
+    Configuration.addDefaultResource("hdfs-site.xml");
+  }
+
+  public static final String DN_CLIENTTRACE_FORMAT =
+        "src: %s" +      // src IP
+        ", dest: %s" +   // dst IP
+        ", bytes: %s" +  // byte count
+        ", op: %s" +     // operation
+        ", cliID: %s" +  // DFSClient id
+        ", offset: %s" + // offset
+        ", srvID: %s" +  // DatanodeRegistration
+        ", blockid: %s" + // block id
+        ", duration: %s";  // duration time
+        
+  static final Log ClientTraceLog =
+    LogFactory.getLog(DataNode.class.getName() + ".clienttrace");
+
+  /**
+   * Use {@link NetUtils#createSocketAddr(String)} instead.
+   */
+  @Deprecated
+  public static InetSocketAddress createSocketAddr(String target
+                                                   ) throws IOException {
+    return NetUtils.createSocketAddr(target);
+  }
+  
+  public DatanodeProtocol namenode = null;
+  public FSDatasetInterface data = null;
+  public DatanodeRegistration dnRegistration = null;
+
+  volatile boolean shouldRun = true;
+  private LinkedList<Block> receivedBlockList = new LinkedList<Block>();
+  /** list of blocks being recovered */
+  private final Map<Block, Block> ongoingRecovery = new HashMap<Block, Block>();
+  private LinkedList<String> delHints = new LinkedList<String>();
+  public final static String EMPTY_DEL_HINT = "";
+  AtomicInteger xmitsInProgress = new AtomicInteger();
+  Daemon dataXceiverServer = null;
+  ThreadGroup threadGroup = null;
+  long blockReportInterval;
+  //disallow the sending of BR before instructed to do so
+  long lastBlockReport = 0;
+  boolean resetBlockReportTime = true;
+  long initialBlockReportDelay = BLOCKREPORT_INITIAL_DELAY * 1000L;
+  long lastHeartbeat = 0;
+  long heartBeatInterval;
+  private DataStorage storage = null;
+  private HttpServer infoServer = null;
+  DataNodeMetrics myMetrics;
+  private InetSocketAddress nameNodeAddr;
+  private InetSocketAddress selfAddr;
+  private static DataNode datanodeObject = null;
+  private Thread dataNodeThread = null;
+  String machineName;
+  private static String dnThreadName;
+  int socketTimeout;
+  int socketWriteTimeout = 0;  
+  boolean transferToAllowed = true;
+  int writePacketSize = 0;
+  boolean isAccessTokenEnabled;
+  AccessTokenHandler accessTokenHandler;
+  boolean isAccessTokenInitialized = false;
+  
+  public DataBlockScanner blockScanner = null;
+  public Daemon blockScannerThread = null;
+  
+  /** Activated plug-ins. */
+  private List<ServicePlugin> plugins;
+  
+  private static final Random R = new Random();
+  
+  // For InterDataNodeProtocol
+  public Server ipcServer;
+
+  /**
+   * Current system time.
+   * @return current time in msec.
+   */
+  static long now() {
+    return System.currentTimeMillis();
+  }
+
+  /**
+   * Create the DataNode given a configuration and an array of dataDirs.
+   * 'dataDirs' is where the blocks are stored.
+   */
+  DataNode(Configuration conf, 
+           AbstractList<File> dataDirs) throws IOException {
+    super(conf);
+    DataNode.setDataNode(this);
+    try {
+      startDataNode(conf, dataDirs);
+    } catch (IOException ie) {
+      shutdown();
+      throw ie;
+    }
+  }
+    
+  
+  /**
+   * This method starts the data node with the specified conf.
+   * 
+   * @param conf - the configuration
+   *  if conf's CONFIG_PROPERTY_SIMULATED property is set
+   *  then a simulated storage based data node is created.
+   * 
+   * @param dataDirs - only for a non-simulated storage data node
+   * @throws IOException
+   */
+  void startDataNode(Configuration conf, 
+                     AbstractList<File> dataDirs
+                     ) throws IOException {
+    // use configured nameserver & interface to get local hostname
+    if (conf.get("slave.host.name") != null) {
+      machineName = conf.get("slave.host.name");   
+    }
+    if (machineName == null) {
+      machineName = DNS.getDefaultHost(
+                                     conf.get("dfs.datanode.dns.interface","default"),
+                                     conf.get("dfs.datanode.dns.nameserver","default"));
+    }
+    this.nameNodeAddr = NameNode.getAddress(conf);
+    
+    this.socketTimeout =  conf.getInt("dfs.socket.timeout",
+                                      HdfsConstants.READ_TIMEOUT);
+    this.socketWriteTimeout = conf.getInt("dfs.datanode.socket.write.timeout",
+                                          HdfsConstants.WRITE_TIMEOUT);
+    /* Based on results on different platforms, we might need set the default 
+     * to false on some of them. */
+    this.transferToAllowed = conf.getBoolean("dfs.datanode.transferTo.allowed", 
+                                             true);
+    this.writePacketSize = conf.getInt("dfs.write.packet.size", 64*1024);
+    InetSocketAddress socAddr = NetUtils.createSocketAddr(
+        conf.get("dfs.datanode.address", "0.0.0.0:50010"));
+    int tmpPort = socAddr.getPort();
+    storage = new DataStorage();
+    // construct registration
+    this.dnRegistration = new DatanodeRegistration(machineName + ":" + tmpPort);
+
+    // connect to name node
+    this.namenode = (DatanodeProtocol) 
+      RPC.waitForProxy(DatanodeProtocol.class,
+                       DatanodeProtocol.versionID,
+                       nameNodeAddr, 
+                       conf);
+    // get version and id info from the name-node
+    NamespaceInfo nsInfo = handshake();
+    StartupOption startOpt = getStartupOption(conf);
+    assert startOpt != null : "Startup option must be set.";
+    
+    boolean simulatedFSDataset = 
+        conf.getBoolean("dfs.datanode.simulateddatastorage", false);
+    if (simulatedFSDataset) {
+        setNewStorageID(dnRegistration);
+        dnRegistration.storageInfo.layoutVersion = FSConstants.LAYOUT_VERSION;
+        dnRegistration.storageInfo.namespaceID = nsInfo.namespaceID;
+        // it would have been better to pass storage as a parameter to
+        // constructor below - need to augment ReflectionUtils used below.
+        conf.set("StorageId", dnRegistration.getStorageID());
+        try {
+          //Equivalent of following (can't do because Simulated is in test dir)
+          //  this.data = new SimulatedFSDataset(conf);
+          this.data = (FSDatasetInterface) ReflectionUtils.newInstance(
+              Class.forName("org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset"), conf);
+        } catch (ClassNotFoundException e) {
+          throw new IOException(StringUtils.stringifyException(e));
+        }
+    } else { // real storage
+      // read storage info, lock data dirs and transition fs state if necessary
+      storage.recoverTransitionRead(nsInfo, dataDirs, startOpt);
+      // adjust
+      this.dnRegistration.setStorageInfo(storage);
+      // initialize data node internal structure
+      this.data = new FSDataset(storage, conf);
+    }
+
+      
+    // find free port
+    ServerSocket ss = (socketWriteTimeout > 0) ? 
+          ServerSocketChannel.open().socket() : new ServerSocket();
+    Server.bind(ss, socAddr, 0);
+    ss.setReceiveBufferSize(DEFAULT_DATA_SOCKET_SIZE); 
+    // adjust machine name with the actual port
+    tmpPort = ss.getLocalPort();
+    selfAddr = new InetSocketAddress(ss.getInetAddress().getHostAddress(),
+                                     tmpPort);
+    this.dnRegistration.setName(machineName + ":" + tmpPort);
+    LOG.info("Opened info server at " + tmpPort);
+      
+    this.threadGroup = new ThreadGroup("dataXceiverServer");
+    this.dataXceiverServer = new Daemon(threadGroup, 
+        new DataXceiverServer(ss, conf, this));
+    this.threadGroup.setDaemon(true); // auto destroy when empty
+
+    this.blockReportInterval =
+      conf.getLong("dfs.blockreport.intervalMsec", BLOCKREPORT_INTERVAL);
+    this.initialBlockReportDelay = conf.getLong("dfs.blockreport.initialDelay",
+                                            BLOCKREPORT_INITIAL_DELAY)* 1000L; 
+    if (this.initialBlockReportDelay >= blockReportInterval) {
+      this.initialBlockReportDelay = 0;
+      LOG.info("dfs.blockreport.initialDelay is greater than " +
+        "dfs.blockreport.intervalMsec." + " Setting initial delay to 0 msec:");
+    }
+    this.heartBeatInterval = conf.getLong("dfs.heartbeat.interval", HEARTBEAT_INTERVAL) * 1000L;
+
+    //initialize periodic block scanner
+    String reason = null;
+    if (conf.getInt("dfs.datanode.scan.period.hours", 0) < 0) {
+      reason = "verification is turned off by configuration";
+    } else if ( !(data instanceof FSDataset) ) {
+      reason = "verifcation is supported only with FSDataset";
+    } 
+    if ( reason == null ) {
+      blockScanner = new DataBlockScanner(this, (FSDataset)data, conf);
+    } else {
+      LOG.info("Periodic Block Verification is disabled because " +
+               reason + ".");
+    }
+
+    //create a servlet to serve full-file content
+    InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(
+        conf.get("dfs.datanode.http.address", "0.0.0.0:50075"));
+    String infoHost = infoSocAddr.getHostName();
+    int tmpInfoPort = infoSocAddr.getPort();
+    this.infoServer = new HttpServer("datanode", infoHost, tmpInfoPort,
+        tmpInfoPort == 0, conf);
+    if (conf.getBoolean("dfs.https.enable", false)) {
+      boolean needClientAuth = conf.getBoolean("dfs.https.need.client.auth", false);
+      InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(conf.get(
+          "dfs.datanode.https.address", infoHost + ":" + 0));
+      Configuration sslConf = new Configuration(false);
+      sslConf.addResource(conf.get("dfs.https.server.keystore.resource",
+          "ssl-server.xml"));
+      this.infoServer.addSslListener(secInfoSocAddr, sslConf, needClientAuth);
+    }
+    this.infoServer.addInternalServlet(null, "/streamFile/*", StreamFile.class);
+    this.infoServer.addInternalServlet(null, "/getFileChecksum/*",
+        FileChecksumServlets.GetServlet.class);
+    this.infoServer.setAttribute("datanode.blockScanner", blockScanner);
+    this.infoServer.addServlet(null, "/blockScannerReport", 
+                               DataBlockScanner.Servlet.class);
+    this.infoServer.start();
+    // adjust info port
+    this.dnRegistration.setInfoPort(this.infoServer.getPort());
+    myMetrics = new DataNodeMetrics(conf, dnRegistration.getStorageID());
+    
+    // set service-level authorization security policy
+    if (conf.getBoolean(
+          ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, false)) {
+      PolicyProvider policyProvider = 
+        (PolicyProvider)(ReflectionUtils.newInstance(
+            conf.getClass(PolicyProvider.POLICY_PROVIDER_CONFIG, 
+                HDFSPolicyProvider.class, PolicyProvider.class), 
+            conf));
+      SecurityUtil.setPolicy(new ConfiguredPolicy(conf, policyProvider));
+    }
+
+    //init ipc server
+    InetSocketAddress ipcAddr = NetUtils.createSocketAddr(
+        conf.get("dfs.datanode.ipc.address"));
+    ipcServer = RPC.getServer(this, ipcAddr.getHostName(), ipcAddr.getPort(), 
+        conf.getInt("dfs.datanode.handler.count", 3), false, conf);
+    ipcServer.start();
+    dnRegistration.setIpcPort(ipcServer.getListenerAddress().getPort());
+
+    LOG.info("dnRegistration = " + dnRegistration);
+    
+    plugins = conf.getInstances("dfs.datanode.plugins", ServicePlugin.class);
+    for (ServicePlugin p: plugins) {
+      try {
+        p.start(this);
+        LOG.info("Started plug-in " + p);
+      } catch (Throwable t) {
+        LOG.warn("ServicePlugin " + p + " could not be started", t);
+      }
+    }
+  }
+
+  /**
+   * Creates either NIO or regular depending on socketWriteTimeout.
+   */
+  protected Socket newSocket() throws IOException {
+    return (socketWriteTimeout > 0) ? 
+           SocketChannel.open().socket() : new Socket();                                   
+  }
+  
+  private NamespaceInfo handshake() throws IOException {
+    NamespaceInfo nsInfo = new NamespaceInfo();
+    while (shouldRun) {
+      try {
+        nsInfo = namenode.versionRequest();
+        break;
+      } catch(SocketTimeoutException e) {  // namenode is busy
+        LOG.info("Problem connecting to server: " + getNameNodeAddr());
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException ie) {}
+      }
+    }
+    String errorMsg = null;
+    // verify build version
+    if( ! nsInfo.getBuildVersion().equals( Storage.getBuildVersion() )) {
+      errorMsg = "Incompatible build versions: namenode BV = " 
+        + nsInfo.getBuildVersion() + "; datanode BV = "
+        + Storage.getBuildVersion();
+      LOG.fatal( errorMsg );
+      try {
+        namenode.errorReport( dnRegistration,
+                              DatanodeProtocol.NOTIFY, errorMsg );
+      } catch( SocketTimeoutException e ) {  // namenode is busy
+        LOG.info("Problem connecting to server: " + getNameNodeAddr());
+      }
+      throw new IOException( errorMsg );
+    }
+    assert FSConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
+      "Data-node and name-node layout versions must be the same."
+      + "Expected: "+ FSConstants.LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
+    return nsInfo;
+  }
+
+  private static void setDataNode(DataNode node) {
+    datanodeObject = node;
+  }
+
+  /** Return the DataNode object
+   * 
+   */
+  public static DataNode getDataNode() {
+    return datanodeObject;
+  } 
+
+  public static InterDatanodeProtocol createInterDataNodeProtocolProxy(
+      DatanodeID datanodeid, Configuration conf) throws IOException {
+    InetSocketAddress addr = NetUtils.createSocketAddr(
+        datanodeid.getHost() + ":" + datanodeid.getIpcPort());
+    if (InterDatanodeProtocol.LOG.isDebugEnabled()) {
+      InterDatanodeProtocol.LOG.info("InterDatanodeProtocol addr=" + addr);
+    }
+    return (InterDatanodeProtocol)RPC.getProxy(InterDatanodeProtocol.class,
+        InterDatanodeProtocol.versionID, addr, conf);
+  }
+
+  public InetSocketAddress getNameNodeAddr() {
+    return nameNodeAddr;
+  }
+  
+  public InetSocketAddress getSelfAddr() {
+    return selfAddr;
+  }
+    
+  DataNodeMetrics getMetrics() {
+    return myMetrics;
+  }
+  
+  /**
+   * Return the namenode's identifier
+   */
+  public String getNamenode() {
+    //return namenode.toString();
+    return "<namenode>";
+  }
+
+  public static void setNewStorageID(DatanodeRegistration dnReg) {
+    /* Return 
+     * "DS-randInt-ipaddr-currentTimeMillis"
+     * It is considered extermely rare for all these numbers to match
+     * on a different machine accidentally for the following 
+     * a) SecureRandom(INT_MAX) is pretty much random (1 in 2 billion), and
+     * b) Good chance ip address would be different, and
+     * c) Even on the same machine, Datanode is designed to use different ports.
+     * d) Good chance that these are started at different times.
+     * For a confict to occur all the 4 above have to match!.
+     * The format of this string can be changed anytime in future without
+     * affecting its functionality.
+     */
+    String ip = "unknownIP";
+    try {
+      ip = DNS.getDefaultIP("default");
+    } catch (UnknownHostException ignored) {
+      LOG.warn("Could not find ip address of \"default\" inteface.");
+    }
+    
+    int rand = 0;
+    try {
+      rand = SecureRandom.getInstance("SHA1PRNG").nextInt(Integer.MAX_VALUE);
+    } catch (NoSuchAlgorithmException e) {
+      LOG.warn("Could not use SecureRandom");
+      rand = R.nextInt(Integer.MAX_VALUE);
+    }
+    dnReg.storageID = "DS-" + rand + "-"+ ip + "-" + dnReg.getPort() + "-" + 
+                      System.currentTimeMillis();
+  }
+  /**
+   * Register datanode
+   * <p>
+   * The datanode needs to register with the namenode on startup in order
+   * 1) to report which storage it is serving now and 
+   * 2) to receive a registrationID 
+   * issued by the namenode to recognize registered datanodes.
+   * 
+   * @see FSNamesystem#registerDatanode(DatanodeRegistration)
+   * @throws IOException
+   */
+  private void register() throws IOException {
+    if (dnRegistration.getStorageID().equals("")) {
+      setNewStorageID(dnRegistration);
+    }
+    while(shouldRun) {
+      try {
+        // reset name to machineName. Mainly for web interface.
+        dnRegistration.name = machineName + ":" + dnRegistration.getPort();
+        dnRegistration = namenode.register(dnRegistration);
+        break;
+      } catch(SocketTimeoutException e) {  // namenode is busy
+        LOG.info("Problem connecting to server: " + getNameNodeAddr());
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException ie) {}
+      }
+    }
+    assert ("".equals(storage.getStorageID()) 
+            && !"".equals(dnRegistration.getStorageID()))
+            || storage.getStorageID().equals(dnRegistration.getStorageID()) :
+            "New storageID can be assigned only if data-node is not formatted";
+    if (storage.getStorageID().equals("")) {
+      storage.setStorageID(dnRegistration.getStorageID());
+      storage.writeAll();
+      LOG.info("New storage id " + dnRegistration.getStorageID()
+          + " is assigned to data-node " + dnRegistration.getName());
+    }
+    if(! storage.getStorageID().equals(dnRegistration.getStorageID())) {
+      throw new IOException("Inconsistent storage IDs. Name-node returned "
+          + dnRegistration.getStorageID() 
+          + ". Expecting " + storage.getStorageID());
+    }
+    
+    if (!isAccessTokenInitialized) {
+      /* first time registering with NN */
+      ExportedAccessKeys keys = dnRegistration.exportedKeys;
+      this.isAccessTokenEnabled = keys.isAccessTokenEnabled();
+      if (isAccessTokenEnabled) {
+        long accessKeyUpdateInterval = keys.getKeyUpdateInterval();
+        long accessTokenLifetime = keys.getTokenLifetime();
+        LOG.info("Access token params received from NN: keyUpdateInterval="
+            + accessKeyUpdateInterval / (60 * 1000) + " min(s), tokenLifetime="
+            + accessTokenLifetime / (60 * 1000) + " min(s)");
+        this.accessTokenHandler = new AccessTokenHandler(false,
+            accessKeyUpdateInterval, accessTokenLifetime);
+      }
+      isAccessTokenInitialized = true;
+    }
+
+    if (isAccessTokenEnabled) {
+      accessTokenHandler.setKeys(dnRegistration.exportedKeys);
+      dnRegistration.exportedKeys = ExportedAccessKeys.DUMMY_KEYS;
+    }
+
+    // random short delay - helps scatter the BR from all DNs
+    scheduleBlockReport(initialBlockReportDelay);
+  }
+
+  /**
+   * Shut down this instance of the datanode.
+   * Returns only after shutdown is complete.
+   * This method can only be called by the offerService thread.
+   * Otherwise, deadlock might occur.
+   */
+  public void shutdown() {
+    if (plugins != null) {
+      for (ServicePlugin p : plugins) {
+        try {
+          p.stop();
+          LOG.info("Stopped plug-in " + p);
+        } catch (Throwable t) {
+          LOG.warn("ServicePlugin " + p + " could not be stopped", t);
+        }
+      }
+    }
+    
+    if (infoServer != null) {
+      try {
+        infoServer.stop();
+      } catch (Exception e) {
+        LOG.warn("Exception shutting down DataNode", e);
+      }
+    }
+    if (ipcServer != null) {
+      ipcServer.stop();
+    }
+    this.shouldRun = false;
+    if (dataXceiverServer != null) {
+      ((DataXceiverServer) this.dataXceiverServer.getRunnable()).kill();
+      this.dataXceiverServer.interrupt();
+
+      // wait for all data receiver threads to exit
+      if (this.threadGroup != null) {
+        while (true) {
+          this.threadGroup.interrupt();
+          LOG.info("Waiting for threadgroup to exit, active threads is " +
+                   this.threadGroup.activeCount());
+          if (this.threadGroup.activeCount() == 0) {
+            break;
+          }
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException e) {}
+        }
+      }
+      // wait for dataXceiveServer to terminate
+      try {
+        this.dataXceiverServer.join();
+      } catch (InterruptedException ie) {
+      }
+    }
+    
+    RPC.stopProxy(namenode); // stop the RPC threads
+    
+    if(upgradeManager != null)
+      upgradeManager.shutdownUpgrade();
+    if (blockScannerThread != null) { 
+      blockScannerThread.interrupt();
+      try {
+        blockScannerThread.join(3600000L); // wait for at most 1 hour
+      } catch (InterruptedException ie) {
+      }
+    }
+    if (storage != null) {
+      try {
+        this.storage.unlockAll();
+      } catch (IOException ie) {
+      }
+    }
+    if (dataNodeThread != null) {
+      dataNodeThread.interrupt();
+      try {
+        dataNodeThread.join();
+      } catch (InterruptedException ie) {
+      }
+    }
+    if (data != null) {
+      data.shutdown();
+    }
+    if (myMetrics != null) {
+      myMetrics.shutdown();
+    }
+  }
+  
+  
+  /* Check if there is no space in disk or the disk is read-only
+   *  when IOException occurs. 
+   * If so, handle the error */
+  protected void checkDiskError( IOException e ) throws IOException {
+    if (e.getMessage() != null && 
+        e.getMessage().startsWith("No space left on device")) {
+      throw new DiskOutOfSpaceException("No space left on device");
+    } else {
+      checkDiskError();
+    }
+  }
+  
+  /* Check if there is no disk space and if so, handle the error*/
+  protected void checkDiskError( ) throws IOException {
+    try {
+      data.checkDataDir();
+    } catch(DiskErrorException de) {
+      handleDiskError(de.getMessage());
+    }
+  }
+  
+  private void handleDiskError(String errMsgr) {
+    LOG.warn("DataNode is shutting down.\n" + errMsgr);
+    shouldRun = false;
+    try {
+      namenode.errorReport(
+                           dnRegistration, DatanodeProtocol.DISK_ERROR, errMsgr);
+    } catch(IOException ignored) {              
+    }
+  }
+    
+  /** Number of concurrent xceivers per node. */
+  int getXceiverCount() {
+    return threadGroup == null ? 0 : threadGroup.activeCount();
+  }
+    
+  /**
+   * Main loop for the DataNode.  Runs until shutdown,
+   * forever calling remote NameNode functions.
+   */
+  public void offerService() throws Exception {
+     
+    LOG.info("using BLOCKREPORT_INTERVAL of " + blockReportInterval + "msec" + 
+       " Initial delay: " + initialBlockReportDelay + "msec");
+
+    //
+    // Now loop for a long time....
+    //
+    while (shouldRun) {
+      try {
+        long startTime = now();
+
+        //
+        // Every so often, send heartbeat or block-report
+        //
+        
+        if (startTime - lastHeartbeat > heartBeatInterval) {
+          //
+          // All heartbeat messages include following info:
+          // -- Datanode name
+          // -- data transfer port
+          // -- Total capacity
+          // -- Bytes remaining
+          //
+          lastHeartbeat = startTime;
+          DatanodeCommand[] cmds = namenode.sendHeartbeat(dnRegistration,
+                                                       data.getCapacity(),
+                                                       data.getDfsUsed(),
+                                                       data.getRemaining(),
+                                                       xmitsInProgress.get(),
+                                                       getXceiverCount());
+          myMetrics.heartbeats.inc(now() - startTime);
+          //LOG.info("Just sent heartbeat, with name " + localName);
+          if (!processCommand(cmds))
+            continue;
+        }
+            
+        reportReceivedBlocks();
+
+        DatanodeCommand cmd = blockReport();
+        processCommand(cmd);
+
+        // start block scanner
+        if (blockScanner != null && blockScannerThread == null &&
+            upgradeManager.isUpgradeCompleted()) {
+          LOG.info("Starting Periodic block scanner.");
+          blockScannerThread = new Daemon(blockScanner);
+          blockScannerThread.start();
+        }
+            
+        //
+        // There is no work to do;  sleep until hearbeat timer elapses, 
+        // or work arrives, and then iterate again.
+        //
+        long waitTime = heartBeatInterval - (System.currentTimeMillis() - lastHeartbeat);
+        synchronized(receivedBlockList) {
+          if (waitTime > 0 && receivedBlockList.size() == 0) {
+            try {
+              receivedBlockList.wait(waitTime);
+            } catch (InterruptedException ie) {
+            }
+          }
+        } // synchronized
+      } catch(RemoteException re) {
+        String reClass = re.getClassName();
+        if (UnregisteredNodeException.class.getName().equals(reClass) ||
+            DisallowedDatanodeException.class.getName().equals(reClass) ||
+            IncorrectVersionException.class.getName().equals(reClass)) {
+          LOG.warn("DataNode is shutting down: " + 
+                   StringUtils.stringifyException(re));
+          shutdown();
+          return;
+        }
+        LOG.warn(StringUtils.stringifyException(re));
+      } catch (IOException e) {
+        LOG.warn(StringUtils.stringifyException(e));
+      }
+    } // while (shouldRun)
+  } // offerService
+
+  /**
+   * Process an array of datanode commands
+   * 
+   * @param cmds an array of datanode commands
+   * @return true if further processing may be required or false otherwise. 
+   */
+  private boolean processCommand(DatanodeCommand[] cmds) {
+    if (cmds != null) {
+      for (DatanodeCommand cmd : cmds) {
+        try {
+          if (processCommand(cmd) == false) {
+            return false;
+          }
+        } catch (IOException ioe) {
+          LOG.warn("Error processing datanode Command", ioe);
+        }
+      }
+    }
+    return true;
+  }
+  
+    /**
+     * 
+     * @param cmd
+     * @return true if further processing may be required or false otherwise. 
+     * @throws IOException
+     */
+  private boolean processCommand(DatanodeCommand cmd) throws IOException {
+    if (cmd == null)
+      return true;
+    final BlockCommand bcmd = cmd instanceof BlockCommand? (BlockCommand)cmd: null;
+
+    switch(cmd.getAction()) {
+    case DatanodeProtocol.DNA_TRANSFER:
+      // Send a copy of a block to another datanode
+      transferBlocks(bcmd.getBlocks(), bcmd.getTargets());
+      myMetrics.blocksReplicated.inc(bcmd.getBlocks().length);
+      break;
+    case DatanodeProtocol.DNA_INVALIDATE:
+      //
+      // Some local block(s) are obsolete and can be 
+      // safely garbage-collected.
+      //
+      Block toDelete[] = bcmd.getBlocks();
+      try {
+        if (blockScanner != null) {
+          blockScanner.deleteBlocks(toDelete);
+        }
+        data.invalidate(toDelete);
+      } catch(IOException e) {
+        checkDiskError();
+        throw e;
+      }
+      myMetrics.blocksRemoved.inc(toDelete.length);
+      break;
+    case DatanodeProtocol.DNA_SHUTDOWN:
+      // shut down the data node
+      this.shutdown();
+      return false;
+    case DatanodeProtocol.DNA_REGISTER:
+      // namenode requested a registration - at start or if NN lost contact
+      LOG.info("DatanodeCommand action: DNA_REGISTER");
+      if (shouldRun) {
+        register();
+      }
+      break;
+    case DatanodeProtocol.DNA_FINALIZE:
+      storage.finalizeUpgrade();
+      break;
+    case UpgradeCommand.UC_ACTION_START_UPGRADE:
+      // start distributed upgrade here
+      processDistributedUpgradeCommand((UpgradeCommand)cmd);
+      break;
+    case DatanodeProtocol.DNA_RECOVERBLOCK:
+      recoverBlocks(bcmd.getBlocks(), bcmd.getTargets());
+      break;
+    case DatanodeProtocol.DNA_ACCESSKEYUPDATE:
+      LOG.info("DatanodeCommand action: DNA_ACCESSKEYUPDATE");
+      if (isAccessTokenEnabled) {
+        accessTokenHandler.setKeys(((KeyUpdateCommand) cmd).getExportedKeys());
+      }
+      break;
+    default:
+      LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction());
+    }
+    return true;
+  }
+
+  // Distributed upgrade manager
+  UpgradeManagerDatanode upgradeManager = new UpgradeManagerDatanode(this);
+
+  private void processDistributedUpgradeCommand(UpgradeCommand comm
+                                               ) throws IOException {
+    assert upgradeManager != null : "DataNode.upgradeManager is null.";
+    upgradeManager.processUpgradeCommand(comm);
+  }
+
+  /**
+   * Report received blocks and delete hints to the Namenode
+   * @throws IOException
+   */
+  private void reportReceivedBlocks() throws IOException {
+    //check if there are newly received blocks
+    Block [] blockArray=null;
+    String [] delHintArray=null;
+    synchronized(receivedBlockList) {
+      synchronized(delHints){
+        int numBlocks = receivedBlockList.size();
+        if (numBlocks > 0) {
+          if(numBlocks!=delHints.size()) {
+            LOG.warn("Panic: receiveBlockList and delHints are not of the same length" );
+          }
+          //
+          // Send newly-received blockids to namenode
+          //
+          blockArray = receivedBlockList.toArray(new Block[numBlocks]);
+          delHintArray = delHints.toArray(new String[numBlocks]);
+        }
+      }
+    }
+    if (blockArray != null) {
+      if(delHintArray == null || delHintArray.length != blockArray.length ) {
+        LOG.warn("Panic: block array & delHintArray are not the same" );
+      }
+      namenode.blockReceived(dnRegistration, blockArray, delHintArray);
+      synchronized(receivedBlockList) {
+        synchronized(delHints){
+          for(int i=0; i<blockArray.length; i++) {
+            receivedBlockList.remove(blockArray[i]);
+            delHints.remove(delHintArray[i]);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Report the list blocks to the Namenode
+   * @throws IOException
+   */
+  private DatanodeCommand blockReport() throws IOException {
+    // send block report
+    DatanodeCommand cmd = null;
+    long startTime = now();
+    if (startTime - lastBlockReport > blockReportInterval) {
+      //
+      // Send latest block report if timer has expired.
+      // Get back a list of local block(s) that are obsolete
+      // and can be safely GC'ed.
+      //
+      long brStartTime = now();
+      Block[] bReport = data.getBlockReport();
+
+      cmd = namenode.blockReport(dnRegistration,
+              BlockListAsLongs.convertToArrayLongs(bReport));
+      long brTime = now() - brStartTime;
+      myMetrics.blockReports.inc(brTime);
+      LOG.info("BlockReport of " + bReport.length +
+          " blocks got processed in " + brTime + " msecs");
+      //
+      // If we have sent the first block report, then wait a random
+      // time before we start the periodic block reports.
+      //
+      if (resetBlockReportTime) {
+        lastBlockReport = startTime - R.nextInt((int)(blockReportInterval));
+        resetBlockReportTime = false;
+      } else {
+        /* say the last block report was at 8:20:14. The current report
+         * should have started around 9:20:14 (default 1 hour interval).
+         * If current time is :
+         *   1) normal like 9:20:18, next report should be at 10:20:14
+         *   2) unexpected like 11:35:43, next report should be at 12:20:14
+         */
+        lastBlockReport += (now() - lastBlockReport) /
+                           blockReportInterval * blockReportInterval;
+      }
+    }
+    return cmd;
+  }
+
+  /**
+   * Start distributed upgrade if it should be initiated by the data-node.
+   */
+  private void startDistributedUpgradeIfNeeded() throws IOException {
+    UpgradeManagerDatanode um = DataNode.getDataNode().upgradeManager;
+    assert um != null : "DataNode.upgradeManager is null.";
+    if(!um.getUpgradeState())
+      return;
+    um.setUpgradeState(false, um.getUpgradeVersion());
+    um.startUpgrade();
+    return;
+  }
+
+  private void transferBlock( Block block, 
+                              DatanodeInfo xferTargets[] 
+                              ) throws IOException {
+    if (!data.isValidBlock(block)) {
+      // block does not exist or is under-construction
+      String errStr = "Can't send invalid block " + block;
+      LOG.info(errStr);
+      namenode.errorReport(dnRegistration, 
+                           DatanodeProtocol.INVALID_BLOCK, 
+                           errStr);
+      return;
+    }
+
+    // Check if NN recorded length matches on-disk length 
+    long onDiskLength = data.getLength(block);
+    if (block.getNumBytes() > onDiskLength) {
+      // Shorter on-disk len indicates corruption so report NN the corrupt block
+      namenode.reportBadBlocks(new LocatedBlock[]{
+          new LocatedBlock(block, new DatanodeInfo[] {
+              new DatanodeInfo(dnRegistration)})});
+      LOG.info("Can't replicate block " + block
+          + " because on-disk length " + onDiskLength 
+          + " is shorter than NameNode recorded length " + block.getNumBytes());
+      return;
+    }
+    
+    int numTargets = xferTargets.length;
+    if (numTargets > 0) {
+      if (LOG.isInfoEnabled()) {
+        StringBuilder xfersBuilder = new StringBuilder();
+        for (int i = 0; i < numTargets; i++) {
+          xfersBuilder.append(xferTargets[i].getName());
+          xfersBuilder.append(" ");
+        }
+        LOG.info(dnRegistration + " Starting thread to transfer block " + 
+                 block + " to " + xfersBuilder);                       
+      }
+
+      new Daemon(new DataTransfer(xferTargets, block, this)).start();
+    }
+  }
+
+  private void transferBlocks( Block blocks[], 
+                               DatanodeInfo xferTargets[][] 
+                               ) {
+    for (int i = 0; i < blocks.length; i++) {
+      try {
+        transferBlock(blocks[i], xferTargets[i]);
+      } catch (IOException ie) {
+        LOG.warn("Failed to transfer block " + blocks[i], ie);
+      }
+    }
+  }
+
+  /*
+   * Informing the name node could take a long long time! Should we wait
+   * till namenode is informed before responding with success to the
+   * client? For now we don't.
+   */
+  protected void notifyNamenodeReceivedBlock(Block block, String delHint) {
+    if(block==null || delHint==null) {
+      throw new IllegalArgumentException(block==null?"Block is null":"delHint is null");
+    }
+    synchronized (receivedBlockList) {
+      synchronized (delHints) {
+        receivedBlockList.add(block);
+        delHints.add(delHint);
+        receivedBlockList.notifyAll();
+      }
+    }
+  }
+
+  
+
+
+  /* ********************************************************************
+  Protocol when a client reads data from Datanode (Cur Ver: 9):
+  
+  Client's Request :
+  =================
+   
+     Processed in DataXceiver:
+     +----------------------------------------------+
+     | Common Header   | 1 byte OP == OP_READ_BLOCK |
+     +----------------------------------------------+
+     
+     Processed in readBlock() :
+     +-------------------------------------------------------------------------+
+     | 8 byte Block ID | 8 byte genstamp | 8 byte start offset | 8 byte length |
+     +-------------------------------------------------------------------------+
+     |   vInt length   |  <DFSClient id> |
+     +-----------------------------------+
+     
+     Client sends optional response only at the end of receiving data.
+       
+  DataNode Response :
+  ===================
+   
+    In readBlock() :
+    If there is an error while initializing BlockSender :
+       +---------------------------+
+       | 2 byte OP_STATUS_ERROR    | and connection will be closed.
+       +---------------------------+
+    Otherwise
+       +---------------------------+
+       | 2 byte OP_STATUS_SUCCESS  |
+       +---------------------------+
+       
+    Actual data, sent by BlockSender.sendBlock() :
+    
+      ChecksumHeader :
+      +--------------------------------------------------+
+      | 1 byte CHECKSUM_TYPE | 4 byte BYTES_PER_CHECKSUM |
+      +--------------------------------------------------+
+      Followed by actual data in the form of PACKETS: 
+      +------------------------------------+
+      | Sequence of data PACKETs ....      |
+      +------------------------------------+
+    
+    A "PACKET" is defined further below.
+    
+    The client reads data until it receives a packet with 
+    "LastPacketInBlock" set to true or with a zero length. If there is 
+    no checksum error, it replies to DataNode with OP_STATUS_CHECKSUM_OK:
+    
+    Client optional response at the end of data transmission :
+      +------------------------------+
+      | 2 byte OP_STATUS_CHECKSUM_OK |
+      +------------------------------+
+    
+    PACKET : Contains a packet header, checksum and data. Amount of data
+    ======== carried is set by BUFFER_SIZE.
+    
+      +-----------------------------------------------------+
+      | 4 byte packet length (excluding packet header)      |
+      +-----------------------------------------------------+
+      | 8 byte offset in the block | 8 byte sequence number |
+      +-----------------------------------------------------+
+      | 1 byte isLastPacketInBlock                          |
+      +-----------------------------------------------------+
+      | 4 byte Length of actual data                        |
+      +-----------------------------------------------------+
+      | x byte checksum data. x is defined below            |
+      +-----------------------------------------------------+
+      | actual data ......                                  |
+      +-----------------------------------------------------+
+      
+      x = (length of data + BYTE_PER_CHECKSUM - 1)/BYTES_PER_CHECKSUM *
+          CHECKSUM_SIZE
+          
+      CHECKSUM_SIZE depends on CHECKSUM_TYPE (usually, 4 for CRC32)
+      
+      The above packet format is used while writing data to DFS also.
+      Not all the fields might be used while reading.
+    
+   ************************************************************************ */
+  
+  /** Header size for a packet */
+  public static final int PKT_HEADER_LEN = ( 4 + /* Packet payload length */
+                                      8 + /* offset in block */
+                                      8 + /* seqno */
+                                      1   /* isLastPacketInBlock */);
+  
+
+
+  /**
+   * Used for transferring a block of data.  This class
+   * sends a piece of data to another DataNode.
+   */
+  class DataTransfer implements Runnable {
+    DatanodeInfo targets[];
+    Block b;
+    DataNode datanode;
+
+    /**
+     * Connect to the first item in the target list.  Pass along the 
+     * entire target list, the block, and the data.
+     */
+    public DataTransfer(DatanodeInfo targets[], Block b, DataNode datanode) throws IOException {
+      this.targets = targets;
+      this.b = b;
+      this.datanode = datanode;
+    }
+
+    /**
+     * Do the deed, write the bytes
+     */
+    public void run() {
+      xmitsInProgress.getAndIncrement();
+      Socket sock = null;
+      DataOutputStream out = null;
+      BlockSender blockSender = null;
+      
+      try {
+        InetSocketAddress curTarget = 
+          NetUtils.createSocketAddr(targets[0].getName());
+        sock = newSocket();
+        NetUtils.connect(sock, curTarget, socketTimeout);
+        sock.setSoTimeout(targets.length * socketTimeout);
+
+        long writeTimeout = socketWriteTimeout + 
+                            HdfsConstants.WRITE_TIMEOUT_EXTENSION * (targets.length-1);
+        OutputStream baseStream = NetUtils.getOutputStream(sock, writeTimeout);
+        out = new DataOutputStream(new BufferedOutputStream(baseStream, 
+                                                            SMALL_BUFFER_SIZE));
+
+        blockSender = new BlockSender(b, 0, b.getNumBytes(), false, false, false, 
+            datanode);
+        DatanodeInfo srcNode = new DatanodeInfo(dnRegistration);
+
+        //
+        // Header info
+        //
+        out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
+        out.writeByte(DataTransferProtocol.OP_WRITE_BLOCK);
+        out.writeLong(b.getBlockId());
+        out.writeLong(b.getGenerationStamp());
+        out.writeInt(0);           // no pipelining
+        out.writeBoolean(false);   // not part of recovery
+        Text.writeString(out, ""); // client
+        out.writeBoolean(true); // sending src node information
+        srcNode.write(out); // Write src node DatanodeInfo
+        // write targets
+        out.writeInt(targets.length - 1);
+        for (int i = 1; i < targets.length; i++) {
+          targets[i].write(out);
+        }
+        AccessToken accessToken = AccessToken.DUMMY_TOKEN;
+        if (isAccessTokenEnabled) {
+          accessToken = accessTokenHandler.generateToken(null, b.getBlockId(),
+              EnumSet.of(AccessTokenHandler.AccessMode.WRITE));
+        }
+        accessToken.write(out);
+        // send data & checksum
+        blockSender.sendBlock(out, baseStream, null);
+
+        // no response necessary
+        LOG.info(dnRegistration + ":Transmitted block " + b + " to " + curTarget);
+
+      } catch (IOException ie) {
+        LOG.warn(dnRegistration + ":Failed to transfer " + b + " to " + targets[0].getName()
+            + " got " + StringUtils.stringifyException(ie));
+      } finally {
+        xmitsInProgress.getAndDecrement();
+        IOUtils.closeStream(blockSender);
+        IOUtils.closeStream(out);
+        IOUtils.closeSocket(sock);
+      }
+    }
+  }
+
+  /**
+   * No matter what kind of exception we get, keep retrying to offerService().
+   * That's the loop that connects to the NameNode and provides basic DataNode
+   * functionality.
+   *
+   * Only stop when "shouldRun" is turned off (which can only happen at shutdown).
+   */
+  public void run() {
+    LOG.info(dnRegistration + "In DataNode.run, data = " + data);
+
+    // start dataXceiveServer
+    dataXceiverServer.start();
+        
+    while (shouldRun) {
+      try {
+        startDistributedUpgradeIfNeeded();
+        offerService();
+      } catch (Exception ex) {
+        LOG.error("Exception: " + StringUtils.stringifyException(ex));
+        if (shouldRun) {
+          try {
+            Thread.sleep(5000);
+          } catch (InterruptedException ie) {
+          }
+        }
+      }
+    }
+        
+    LOG.info(dnRegistration + ":Finishing DataNode in: "+data);
+    shutdown();
+  }
+    
+  /** Start a single datanode daemon and wait for it to finish.
+   *  If this thread is specifically interrupted, it will stop waiting.
+   */
+  public static void runDatanodeDaemon(DataNode dn) throws IOException {
+    if (dn != null) {
+      //register datanode
+      dn.register();
+      dn.dataNodeThread = new Thread(dn, dnThreadName);
+      dn.dataNodeThread.setDaemon(true); // needed for JUnit testing
+      dn.dataNodeThread.start();
+    }
+  }
+  
+  static boolean isDatanodeUp(DataNode dn) {
+    return dn.dataNodeThread != null && dn.dataNodeThread.isAlive();
+  }
+
+  /** Instantiate a single datanode object. This must be run by invoking
+   *  {@link DataNode#runDatanodeDaemon(DataNode)} subsequently. 
+   */
+  public static DataNode instantiateDataNode(String args[],
+                                      Configuration conf) throws IOException {
+    if (conf == null)
+      conf = new Configuration();
+    if (!parseArguments(args, conf)) {
+      printUsage();
+      return null;
+    }
+    if (conf.get("dfs.network.script") != null) {
+      LOG.error("This configuration for rack identification is not supported" +
+          " anymore. RackID resolution is handled by the NameNode.");
+      System.exit(-1);
+    }
+    String[] dataDirs = conf.getStrings("dfs.data.dir");
+    dnThreadName = "DataNode: [" +
+                        StringUtils.arrayToString(dataDirs) + "]";
+    return makeInstance(dataDirs, conf);
+  }
+
+  /** Instantiate & Start a single datanode daemon and wait for it to finish.
+   *  If this thread is specifically interrupted, it will stop waiting.
+   */
+  public static DataNode createDataNode(String args[],
+                                 Configuration conf) throws IOException {
+    DataNode dn = instantiateDataNode(args, conf);
+    runDatanodeDaemon(dn);
+    return dn;
+  }
+
+  void join() {
+    if (dataNodeThread != null) {
+      try {
+        dataNodeThread.join();
+      } catch (InterruptedException e) {}
+    }
+  }
+
+  /**
+   * Make an instance of DataNode after ensuring that at least one of the
+   * given data directories (and their parent directories, if necessary)
+   * can be created.
+   * @param dataDirs List of directories, where the new DataNode instance should
+   * keep its files.
+   * @param conf Configuration instance to use.
+   * @return DataNode instance for given list of data dirs and conf, or null if
+   * no directory from this directory list can be created.
+   * @throws IOException
+   */
+  public static DataNode makeInstance(String[] dataDirs, Configuration conf)
+    throws IOException {
+    ArrayList<File> dirs = new ArrayList<File>();
+    for (int i = 0; i < dataDirs.length; i++) {
+      File data = new File(dataDirs[i]);
+      try {
+        DiskChecker.checkDir(data);
+        dirs.add(data);
+      } catch(DiskErrorException e) {
+        LOG.warn("Invalid directory in dfs.data.dir: " + e.getMessage());
+      }
+    }
+    if (dirs.size() > 0) 
+      return new DataNode(conf, dirs);
+    LOG.error("All directories in dfs.data.dir are invalid.");
+    return null;
+  }
+
+  @Override
+  public String toString() {
+    return "DataNode{" +
+      "data=" + data +
+      ", localName='" + dnRegistration.getName() + "'" +
+      ", storageID='" + dnRegistration.getStorageID() + "'" +
+      ", xmitsInProgress=" + xmitsInProgress.get() +
+      "}";
+  }
+  
+  private static void printUsage() {
+    System.err.println("Usage: java DataNode");
+    System.err.println("           [-rollback]");
+  }
+
+  /**
+   * Parse and verify command line arguments and set configuration parameters.
+   *
+   * @return false if passed argements are incorrect
+   */
+  private static boolean parseArguments(String args[], 
+                                        Configuration conf) {
+    int argsLen = (args == null) ? 0 : args.length;
+    StartupOption startOpt = StartupOption.REGULAR;
+    for(int i=0; i < argsLen; i++) {
+      String cmd = args[i];
+      if ("-r".equalsIgnoreCase(cmd) || "--rack".equalsIgnoreCase(cmd)) {
+        LOG.error("-r, --rack arguments are not supported anymore. RackID " +
+            "resolution is handled by the NameNode.");
+        System.exit(-1);
+      } else if ("-rollback".equalsIgnoreCase(cmd)) {
+        startOpt = StartupOption.ROLLBACK;
+      } else if ("-regular".equalsIgnoreCase(cmd)) {
+        startOpt = StartupOption.REGULAR;
+      } else
+        return false;
+    }
+    setStartupOption(conf, startOpt);
+    return true;
+  }
+
+  private static void setStartupOption(Configuration conf, StartupOption opt) {
+    conf.set("dfs.datanode.startup", opt.toString());
+  }
+
+  static StartupOption getStartupOption(Configuration conf) {
+    return StartupOption.valueOf(conf.get("dfs.datanode.startup",
+                                          StartupOption.REGULAR.toString()));
+  }
+
+  /**
+   * This methods  arranges for the data node to send the block report at the next heartbeat.
+   */
+  public void scheduleBlockReport(long delay) {
+    if (delay > 0) { // send BR after random delay
+      lastBlockReport = System.currentTimeMillis()
+                            - ( blockReportInterval - R.nextInt((int)(delay)));
+    } else { // send at next heartbeat
+      lastBlockReport = lastHeartbeat - blockReportInterval;
+    }
+    resetBlockReportTime = true; // reset future BRs for randomness
+  }
+  
+  
+  /**
+   * This method is used for testing. 
+   * Examples are adding and deleting blocks directly.
+   * The most common usage will be when the data node's storage is similated.
+   * 
+   * @return the fsdataset that stores the blocks
+   */
+  public FSDatasetInterface getFSDataset() {
+    return data;
+  }
+
+  /**
+   */
+  public static void main(String args[]) {
+    try {
+      StringUtils.startupShutdownMessage(DataNode.class, args, LOG);
+      DataNode datanode = createDataNode(args, null);
+      if (datanode != null)
+        datanode.join();
+    } catch (Throwable e) {
+      LOG.error(StringUtils.stringifyException(e));
+      System.exit(-1);
+    }
+  }
+
+  // InterDataNodeProtocol implementation
+  /** {@inheritDoc} */
+  public BlockMetaDataInfo getBlockMetaDataInfo(Block block
+      ) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("block=" + block);
+    }
+    Block stored = data.getStoredBlock(block.getBlockId());
+
+    if (stored == null) {
+      return null;
+    }
+    BlockMetaDataInfo info = new BlockMetaDataInfo(stored,
+                                 blockScanner.getLastScanTime(stored));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("getBlockMetaDataInfo successful block=" + stored +
+                " length " + stored.getNumBytes() +
+                " genstamp " + stored.getGenerationStamp());
+    }
+
+    // paranoia! verify that the contents of the stored block
+    // matches the block file on disk.
+    data.validateBlockMetadata(stored);
+    return info;
+  }
+
+  public Daemon recoverBlocks(final Block[] blocks, final DatanodeInfo[][] targets) {
+    Daemon d = new Daemon(threadGroup, new Runnable() {
+      /** Recover a list of blocks. It is run by the primary datanode. */
+      public void run() {
+        for(int i = 0; i < blocks.length; i++) {
+          try {
+            logRecoverBlock("NameNode", blocks[i], targets[i]);
+            recoverBlock(blocks[i], false, targets[i], true);
+          } catch (IOException e) {
+            LOG.warn("recoverBlocks FAILED, blocks[" + i + "]=" + blocks[i], e);
+          }
+        }
+      }
+    });
+    d.start();
+    return d;
+  }
+
+  /** {@inheritDoc} */
+  public void updateBlock(Block oldblock, Block newblock, boolean finalize) throws IOException {
+    LOG.info("oldblock=" + oldblock + "(length=" + oldblock.getNumBytes()
+        + "), newblock=" + newblock + "(length=" + newblock.getNumBytes()
+        + "), datanode=" + dnRegistration.getName());
+    data.updateBlock(oldblock, newblock);
+    if (finalize) {
+      data.finalizeBlock(newblock);
+      myMetrics.blocksWritten.inc(); 
+      notifyNamenodeReceivedBlock(newblock, EMPTY_DEL_HINT);
+      LOG.info("Received block " + newblock +
+                " of size " + newblock.getNumBytes() +
+                " as part of lease recovery.");
+    }
+  }
+
+  /** {@inheritDoc} */
+  public long getProtocolVersion(String protocol, long clientVersion
+      ) throws IOException {
+    if (protocol.equals(InterDatanodeProtocol.class.getName())) {
+      return InterDatanodeProtocol.versionID; 
+    } else if (protocol.equals(ClientDatanodeProtocol.class.getName())) {
+      return ClientDatanodeProtocol.versionID; 
+    }
+    throw new IOException("Unknown protocol to " + getClass().getSimpleName()
+        + ": " + protocol);
+  }
+
+  /** A convenient class used in lease recovery */
+  private static class BlockRecord { 
+    final DatanodeID id;
+    final InterDatanodeProtocol datanode;
+    final Block block;
+    
+    BlockRecord(DatanodeID id, InterDatanodeProtocol datanode, Block block) {
+      this.id = id;
+      this.datanode = datanode;
+      this.block = block;
+    }
+
+    /** {@inheritDoc} */
+    public String toString() {
+      return "block:" + block + " node:" + id;
+    }
+  }
+
+  /** Recover a block */
+  private LocatedBlock recoverBlock(Block block, boolean keepLength,
+      DatanodeID[] datanodeids, boolean closeFile) throws IOException {
+
+    // If the block is already being recovered, then skip recovering it.
+    // This can happen if the namenode and client start recovering the same
+    // file at the same time.
+    synchronized (ongoingRecovery) {
+      Block tmp = new Block();
+      tmp.set(block.getBlockId(), block.getNumBytes(), GenerationStamp.WILDCARD_STAMP);
+      if (ongoingRecovery.get(tmp) != null) {
+        String msg = "Block " + block + " is already being recovered, " +
+                     " ignoring this request to recover it.";
+        LOG.info(msg);
+        throw new IOException(msg);
+      }
+      ongoingRecovery.put(block, block);
+    }
+    try {
+      List<BlockRecord> syncList = new ArrayList<BlockRecord>();
+      long minlength = Long.MAX_VALUE;
+      int errorCount = 0;
+
+      //check generation stamps
+      for(DatanodeID id : datanodeids) {
+        try {
+          InterDatanodeProtocol datanode = dnRegistration.equals(id)?
+              this: DataNode.createInterDataNodeProtocolProxy(id, getConf());
+          BlockMetaDataInfo info = datanode.getBlockMetaDataInfo(block);
+          if (info != null && info.getGenerationStamp() >= block.getGenerationStamp()) {
+            if (keepLength) {
+              if (info.getNumBytes() == block.getNumBytes()) {
+                syncList.add(new BlockRecord(id, datanode, new Block(info)));
+              }
+            }
+            else {
+              syncList.add(new BlockRecord(id, datanode, new Block(info)));
+              if (info.getNumBytes() < minlength) {
+                minlength = info.getNumBytes();
+              }
+            }
+          }
+        } catch (IOException e) {
+          ++errorCount;
+          InterDatanodeProtocol.LOG.warn(
+              "Failed to getBlockMetaDataInfo for block (=" + block 
+              + ") from datanode (=" + id + ")", e);
+        }
+      }
+
+      if (syncList.isEmpty() && errorCount > 0) {
+        throw new IOException("All datanodes failed: block=" + block
+            + ", datanodeids=" + Arrays.asList(datanodeids));
+      }
+      if (!keepLength) {
+        block.setNumBytes(minlength);
+      }
+      return syncBlock(block, syncList, closeFile);
+    } finally {
+      synchronized (ongoingRecovery) {
+        ongoingRecovery.remove(block);
+      }
+    }
+  }
+
+  /** Block synchronization */
+  private LocatedBlock syncBlock(Block block, List<BlockRecord> syncList,
+      boolean closeFile) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("block=" + block + ", (length=" + block.getNumBytes()
+          + "), syncList=" + syncList + ", closeFile=" + closeFile);
+    }
+
+    //syncList.isEmpty() that all datanodes do not have the block
+    //so the block can be deleted.
+    if (syncList.isEmpty()) {
+      namenode.commitBlockSynchronization(block, 0, 0, closeFile, true,
+          DatanodeID.EMPTY_ARRAY);
+      return null;
+    }
+
+    List<DatanodeID> successList = new ArrayList<DatanodeID>();
+
+    long generationstamp = namenode.nextGenerationStamp(block);
+    Block newblock = new Block(block.getBlockId(), block.getNumBytes(), generationstamp);
+
+    for(BlockRecord r : syncList) {
+      try {
+        r.datanode.updateBlock(r.block, newblock, closeFile);
+        successList.add(r.id);
+      } catch (IOException e) {
+        InterDatanodeProtocol.LOG.warn("Failed to updateBlock (newblock="
+            + newblock + ", datanode=" + r.id + ")", e);
+      }
+    }
+
+    if (!successList.isEmpty()) {
+      DatanodeID[] nlist = successList.toArray(new DatanodeID[successList.size()]);
+
+      namenode.commitBlockSynchronization(block,
+          newblock.getGenerationStamp(), newblock.getNumBytes(), closeFile, false,
+          nlist);
+      DatanodeInfo[] info = new DatanodeInfo[nlist.length];
+      for (int i = 0; i < nlist.length; i++) {
+        info[i] = new DatanodeInfo(nlist[i]);
+      }
+      return new LocatedBlock(newblock, info); // success
+    }
+
+    //failed
+    StringBuilder b = new StringBuilder();
+    for(BlockRecord r : syncList) {
+      b.append("\n  " + r.id);
+    }
+    throw new IOException("Cannot recover " + block + ", none of these "
+        + syncList.size() + " datanodes success {" + b + "\n}");
+  }
+  
+  // ClientDataNodeProtocol implementation
+  /** {@inheritDoc} */
+  public LocatedBlock recoverBlock(Block block, boolean keepLength, DatanodeInfo[] targets
+      ) throws IOException {
+    logRecoverBlock("Client", block, targets);
+    return recoverBlock(block, keepLength, targets, false);
+  }
+
+  private static void logRecoverBlock(String who,
+      Block block, DatanodeID[] targets) {
+    StringBuilder msg = new StringBuilder(targets[0].getName());
+    for (int i = 1; i < targets.length; i++) {
+      msg.append(", " + targets[i].getName());
+    }
+    LOG.info(who + " calls recoverBlock(block=" + block
+        + ", targets=[" + msg + "])");
+  }
+}

+ 438 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -0,0 +1,438 @@
+/**
+ * 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.datanode;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileLock;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Properties;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.fs.FileUtil.HardLink;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.Daemon;
+
+/** 
+ * Data storage information file.
+ * <p>
+ * @see Storage
+ */
+public class DataStorage extends Storage {
+  // Constants
+  final static String BLOCK_SUBDIR_PREFIX = "subdir";
+  final static String BLOCK_FILE_PREFIX = "blk_";
+  final static String COPY_FILE_PREFIX = "dncp_";
+  
+  private String storageID;
+
+  DataStorage() {
+    super(NodeType.DATA_NODE);
+    storageID = "";
+  }
+  
+  DataStorage(int nsID, long cT, String strgID) {
+    super(NodeType.DATA_NODE, nsID, cT);
+    this.storageID = strgID;
+  }
+  
+  public DataStorage(StorageInfo storageInfo, String strgID) {
+    super(NodeType.DATA_NODE, storageInfo);
+    this.storageID = strgID;
+  }
+
+  public String getStorageID() {
+    return storageID;
+  }
+  
+  void setStorageID(String newStorageID) {
+    this.storageID = newStorageID;
+  }
+  
+  /**
+   * Analyze storage directories.
+   * Recover from previous transitions if required. 
+   * Perform fs state transition if necessary depending on the namespace info.
+   * Read storage info. 
+   * 
+   * @param nsInfo namespace information
+   * @param dataDirs array of data storage directories
+   * @param startOpt startup option
+   * @throws IOException
+   */
+  void recoverTransitionRead(NamespaceInfo nsInfo,
+                             Collection<File> dataDirs,
+                             StartupOption startOpt
+                             ) throws IOException {
+    assert FSConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
+      "Data-node and name-node layout versions must be the same.";
+    
+    // 1. For each data directory calculate its state and 
+    // check whether all is consistent before transitioning.
+    // Format and recover.
+    this.storageID = "";
+    this.storageDirs = new ArrayList<StorageDirectory>(dataDirs.size());
+    ArrayList<StorageState> dataDirStates = new ArrayList<StorageState>(dataDirs.size());
+    for(Iterator<File> it = dataDirs.iterator(); it.hasNext();) {
+      File dataDir = it.next();
+      StorageDirectory sd = new StorageDirectory(dataDir);
+      StorageState curState;
+      try {
+        curState = sd.analyzeStorage(startOpt);
+        // sd is locked but not opened
+        switch(curState) {
+        case NORMAL:
+          break;
+        case NON_EXISTENT:
+          // ignore this storage
+          LOG.info("Storage directory " + dataDir + " does not exist.");
+          it.remove();
+          continue;
+        case NOT_FORMATTED: // format
+          LOG.info("Storage directory " + dataDir + " is not formatted.");
+          LOG.info("Formatting ...");
+          format(sd, nsInfo);
+          break;
+        default:  // recovery part is common
+          sd.doRecover(curState);
+        }
+      } catch (IOException ioe) {
+        sd.unlock();
+        throw ioe;
+      }
+      // add to the storage list
+      addStorageDir(sd);
+      dataDirStates.add(curState);
+    }
+
+    if (dataDirs.size() == 0)  // none of the data dirs exist
+      throw new IOException(
+                            "All specified directories are not accessible or do not exist.");
+
+    // 2. Do transitions
+    // Each storage directory is treated individually.
+    // During sturtup some of them can upgrade or rollback 
+    // while others could be uptodate for the regular startup.
+    for(int idx = 0; idx < getNumStorageDirs(); idx++) {
+      doTransition(getStorageDir(idx), nsInfo, startOpt);
+      assert this.getLayoutVersion() == nsInfo.getLayoutVersion() :
+        "Data-node and name-node layout versions must be the same.";
+      assert this.getCTime() == nsInfo.getCTime() :
+        "Data-node and name-node CTimes must be the same.";
+    }
+    
+    // 3. Update all storages. Some of them might have just been formatted.
+    this.writeAll();
+  }
+
+  void format(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException {
+    sd.clearDirectory(); // create directory
+    this.layoutVersion = FSConstants.LAYOUT_VERSION;
+    this.namespaceID = nsInfo.getNamespaceID();
+    this.cTime = 0;
+    // store storageID as it currently is
+    sd.write();
+  }
+
+  protected void setFields(Properties props, 
+                           StorageDirectory sd 
+                           ) throws IOException {
+    super.setFields(props, sd);
+    props.setProperty("storageID", storageID);
+  }
+
+  protected void getFields(Properties props, 
+                           StorageDirectory sd 
+                           ) throws IOException {
+    super.getFields(props, sd);
+    String ssid = props.getProperty("storageID");
+    if (ssid == null ||
+        !("".equals(storageID) || "".equals(ssid) ||
+          storageID.equals(ssid)))
+      throw new InconsistentFSStateException(sd.getRoot(),
+                                             "has incompatible storage Id.");
+    if ("".equals(storageID)) // update id only if it was empty
+      storageID = ssid;
+  }
+
+  public boolean isConversionNeeded(StorageDirectory sd) throws IOException {
+    File oldF = new File(sd.getRoot(), "storage");
+    if (!oldF.exists())
+      return false;
+    // check the layout version inside the storage file
+    // Lock and Read old storage file
+    RandomAccessFile oldFile = new RandomAccessFile(oldF, "rws");
+    FileLock oldLock = oldFile.getChannel().tryLock();
+    try {
+      oldFile.seek(0);
+      int oldVersion = oldFile.readInt();
+      if (oldVersion < LAST_PRE_UPGRADE_LAYOUT_VERSION)
+        return false;
+    } finally {
+      oldLock.release();
+      oldFile.close();
+    }
+    return true;
+  }
+  
+  /**
+   * Analize which and whether a transition of the fs state is required
+   * and perform it if necessary.
+   * 
+   * Rollback if previousLV >= LAYOUT_VERSION && prevCTime <= namenode.cTime
+   * Upgrade if this.LV > LAYOUT_VERSION || this.cTime < namenode.cTime
+   * Regular startup if this.LV = LAYOUT_VERSION && this.cTime = namenode.cTime
+   * 
+   * @param sd  storage directory
+   * @param nsInfo  namespace info
+   * @param startOpt  startup option
+   * @throws IOException
+   */
+  private void doTransition( StorageDirectory sd, 
+                             NamespaceInfo nsInfo, 
+                             StartupOption startOpt
+                             ) throws IOException {
+    if (startOpt == StartupOption.ROLLBACK)
+      doRollback(sd, nsInfo); // rollback if applicable
+    sd.read();
+    checkVersionUpgradable(this.layoutVersion);
+    assert this.layoutVersion >= FSConstants.LAYOUT_VERSION :
+      "Future version is not allowed";
+    if (getNamespaceID() != nsInfo.getNamespaceID())
+      throw new IOException(
+                            "Incompatible namespaceIDs in " + sd.getRoot().getCanonicalPath()
+                            + ": namenode namespaceID = " + nsInfo.getNamespaceID() 
+                            + "; datanode namespaceID = " + getNamespaceID());
+    if (this.layoutVersion == FSConstants.LAYOUT_VERSION 
+        && this.cTime == nsInfo.getCTime())
+      return; // regular startup
+    // verify necessity of a distributed upgrade
+    verifyDistributedUpgradeProgress(nsInfo);
+    if (this.layoutVersion > FSConstants.LAYOUT_VERSION
+        || this.cTime < nsInfo.getCTime()) {
+      doUpgrade(sd, nsInfo);  // upgrade
+      return;
+    }
+    // layoutVersion == LAYOUT_VERSION && this.cTime > nsInfo.cTime
+    // must shutdown
+    throw new IOException("Datanode state: LV = " + this.getLayoutVersion() 
+                          + " CTime = " + this.getCTime() 
+                          + " is newer than the namespace state: LV = "
+                          + nsInfo.getLayoutVersion() 
+                          + " CTime = " + nsInfo.getCTime());
+  }
+
+  /**
+   * Move current storage into a backup directory,
+   * and hardlink all its blocks into the new current directory.
+   * 
+   * @param sd  storage directory
+   * @throws IOException
+   */
+  void doUpgrade(StorageDirectory sd,
+                 NamespaceInfo nsInfo
+                 ) throws IOException {
+    LOG.info("Upgrading storage directory " + sd.getRoot()
+             + ".\n   old LV = " + this.getLayoutVersion()
+             + "; old CTime = " + this.getCTime()
+             + ".\n   new LV = " + nsInfo.getLayoutVersion()
+             + "; new CTime = " + nsInfo.getCTime());
+    File curDir = sd.getCurrentDir();
+    File prevDir = sd.getPreviousDir();
+    assert curDir.exists() : "Current directory must exist.";
+    // delete previous dir before upgrading
+    if (prevDir.exists())
+      deleteDir(prevDir);
+    File tmpDir = sd.getPreviousTmp();
+    assert !tmpDir.exists() : "previous.tmp directory must not exist.";
+    // rename current to tmp
+    rename(curDir, tmpDir);
+    // hardlink blocks
+    linkBlocks(tmpDir, curDir, this.getLayoutVersion());
+    // write version file
+    this.layoutVersion = FSConstants.LAYOUT_VERSION;
+    assert this.namespaceID == nsInfo.getNamespaceID() :
+      "Data-node and name-node layout versions must be the same.";
+    this.cTime = nsInfo.getCTime();
+    sd.write();
+    // rename tmp to previous
+    rename(tmpDir, prevDir);
+    LOG.info("Upgrade of " + sd.getRoot()+ " is complete.");
+  }
+
+  void doRollback( StorageDirectory sd,
+                   NamespaceInfo nsInfo
+                   ) throws IOException {
+    File prevDir = sd.getPreviousDir();
+    // regular startup if previous dir does not exist
+    if (!prevDir.exists())
+      return;
+    DataStorage prevInfo = new DataStorage();
+    StorageDirectory prevSD = prevInfo.new StorageDirectory(sd.getRoot());
+    prevSD.read(prevSD.getPreviousVersionFile());
+
+    // We allow rollback to a state, which is either consistent with
+    // the namespace state or can be further upgraded to it.
+    if (!(prevInfo.getLayoutVersion() >= FSConstants.LAYOUT_VERSION
+          && prevInfo.getCTime() <= nsInfo.getCTime()))  // cannot rollback
+      throw new InconsistentFSStateException(prevSD.getRoot(),
+                                             "Cannot rollback to a newer state.\nDatanode previous state: LV = " 
+                                             + prevInfo.getLayoutVersion() + " CTime = " + prevInfo.getCTime() 
+                                             + " is newer than the namespace state: LV = "
+                                             + nsInfo.getLayoutVersion() + " CTime = " + nsInfo.getCTime());
+    LOG.info("Rolling back storage directory " + sd.getRoot()
+             + ".\n   target LV = " + nsInfo.getLayoutVersion()
+             + "; target CTime = " + nsInfo.getCTime());
+    File tmpDir = sd.getRemovedTmp();
+    assert !tmpDir.exists() : "removed.tmp directory must not exist.";
+    // rename current to tmp
+    File curDir = sd.getCurrentDir();
+    assert curDir.exists() : "Current directory must exist.";
+    rename(curDir, tmpDir);
+    // rename previous to current
+    rename(prevDir, curDir);
+    // delete tmp dir
+    deleteDir(tmpDir);
+    LOG.info("Rollback of " + sd.getRoot() + " is complete.");
+  }
+
+  void doFinalize(StorageDirectory sd) throws IOException {
+    File prevDir = sd.getPreviousDir();
+    if (!prevDir.exists())
+      return; // already discarded
+    final String dataDirPath = sd.getRoot().getCanonicalPath();
+    LOG.info("Finalizing upgrade for storage directory " 
+             + dataDirPath 
+             + ".\n   cur LV = " + this.getLayoutVersion()
+             + "; cur CTime = " + this.getCTime());
+    assert sd.getCurrentDir().exists() : "Current directory must exist.";
+    final File tmpDir = sd.getFinalizedTmp();
+    // rename previous to tmp
+    rename(prevDir, tmpDir);
+
+    // delete tmp dir in a separate thread
+    new Daemon(new Runnable() {
+        public void run() {
+          try {
+            deleteDir(tmpDir);
+          } catch(IOException ex) {
+            LOG.error("Finalize upgrade for " + dataDirPath + " failed.", ex);
+          }
+          LOG.info("Finalize upgrade for " + dataDirPath + " is complete.");
+        }
+        public String toString() { return "Finalize " + dataDirPath; }
+      }).start();
+  }
+  
+  void finalizeUpgrade() throws IOException {
+    for (Iterator<StorageDirectory> it = storageDirs.iterator(); it.hasNext();) {
+      doFinalize(it.next());
+    }
+  }
+  
+  static void linkBlocks(File from, File to, int oldLV) throws IOException {
+    if (!from.isDirectory()) {
+      if (from.getName().startsWith(COPY_FILE_PREFIX)) {
+        IOUtils.copyBytes(new FileInputStream(from), 
+                          new FileOutputStream(to), 16*1024, true);
+      } else {
+        
+        //check if we are upgrading from pre-generation stamp version.
+        if (oldLV >= PRE_GENERATIONSTAMP_LAYOUT_VERSION) {
+          // Link to the new file name.
+          to = new File(convertMetatadataFileName(to.getAbsolutePath()));
+        }
+        
+        HardLink.createHardLink(from, to);
+      }
+      return;
+    }
+    // from is a directory
+    if (!to.mkdir())
+      throw new IOException("Cannot create directory " + to);
+    String[] blockNames = from.list(new java.io.FilenameFilter() {
+        public boolean accept(File dir, String name) {
+          return name.startsWith(BLOCK_SUBDIR_PREFIX) 
+            || name.startsWith(BLOCK_FILE_PREFIX)
+            || name.startsWith(COPY_FILE_PREFIX);
+        }
+      });
+    
+    for(int i = 0; i < blockNames.length; i++)
+      linkBlocks(new File(from, blockNames[i]), 
+                 new File(to, blockNames[i]), oldLV);
+  }
+
+  protected void corruptPreUpgradeStorage(File rootDir) throws IOException {
+    File oldF = new File(rootDir, "storage");
+    if (oldF.exists())
+      return;
+    // recreate old storage file to let pre-upgrade versions fail
+    if (!oldF.createNewFile())
+      throw new IOException("Cannot create file " + oldF);
+    RandomAccessFile oldFile = new RandomAccessFile(oldF, "rws");
+    // write new version into old storage file
+    try {
+      writeCorruptedData(oldFile);
+    } finally {
+      oldFile.close();
+    }
+  }
+
+  private void verifyDistributedUpgradeProgress(
+                  NamespaceInfo nsInfo
+                ) throws IOException {
+    UpgradeManagerDatanode um = DataNode.getDataNode().upgradeManager;
+    assert um != null : "DataNode.upgradeManager is null.";
+    um.setUpgradeState(false, getLayoutVersion());
+    um.initializeUpgrade(nsInfo);
+  }
+  
+  private static final Pattern PRE_GENSTAMP_META_FILE_PATTERN = 
+    Pattern.compile("(.*blk_[-]*\\d+)\\.meta$");
+  /**
+   * This is invoked on target file names when upgrading from pre generation 
+   * stamp version (version -13) to correct the metatadata file name.
+   * @param oldFileName
+   * @return the new metadata file name with the default generation stamp.
+   */
+  private static String convertMetatadataFileName(String oldFileName) {
+    Matcher matcher = PRE_GENSTAMP_META_FILE_PATTERN.matcher(oldFileName); 
+    if (matcher.matches()) {
+      //return the current metadata file name
+      return FSDataset.getMetaFileName(matcher.group(1),
+                                       Block.GRANDFATHER_GENERATION_STAMP); 
+    }
+    return oldFileName;
+  }
+}

+ 680 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -0,0 +1,680 @@
+/**
+ * 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.datanode;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketException;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.MetaDataInputStream;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessToken;
+import org.apache.hadoop.security.AccessTokenHandler;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.StringUtils;
+import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
+
+/**
+ * Thread for processing incoming/outgoing data stream.
+ */
+class DataXceiver implements Runnable, FSConstants {
+  public static final Log LOG = DataNode.LOG;
+  static final Log ClientTraceLog = DataNode.ClientTraceLog;
+  
+  Socket s;
+  final String remoteAddress; // address of remote side
+  final String localAddress;  // local address of this daemon
+  DataNode datanode;
+  DataXceiverServer dataXceiverServer;
+  
+  public DataXceiver(Socket s, DataNode datanode, 
+      DataXceiverServer dataXceiverServer) {
+    
+    this.s = s;
+    this.datanode = datanode;
+    this.dataXceiverServer = dataXceiverServer;
+    dataXceiverServer.childSockets.put(s, s);
+    remoteAddress = s.getRemoteSocketAddress().toString();
+    localAddress = s.getLocalSocketAddress().toString();
+    LOG.debug("Number of active connections is: " + datanode.getXceiverCount());
+  }
+
+  /**
+   * Read/write data from/to the DataXceiveServer.
+   */
+  public void run() {
+    DataInputStream in=null; 
+    try {
+      in = new DataInputStream(
+          new BufferedInputStream(NetUtils.getInputStream(s), 
+                                  SMALL_BUFFER_SIZE));
+      short version = in.readShort();
+      if ( version != DataTransferProtocol.DATA_TRANSFER_VERSION ) {
+        throw new IOException( "Version Mismatch" );
+      }
+      boolean local = s.getInetAddress().equals(s.getLocalAddress());
+      byte op = in.readByte();
+      // Make sure the xciver count is not exceeded
+      int curXceiverCount = datanode.getXceiverCount();
+      if (curXceiverCount > dataXceiverServer.maxXceiverCount) {
+        throw new IOException("xceiverCount " + curXceiverCount
+                              + " exceeds the limit of concurrent xcievers "
+                              + dataXceiverServer.maxXceiverCount);
+      }
+      long startTime = DataNode.now();
+      switch ( op ) {
+      case DataTransferProtocol.OP_READ_BLOCK:
+        readBlock( in );
+        datanode.myMetrics.readBlockOp.inc(DataNode.now() - startTime);
+        if (local)
+          datanode.myMetrics.readsFromLocalClient.inc();
+        else
+          datanode.myMetrics.readsFromRemoteClient.inc();
+        break;
+      case DataTransferProtocol.OP_WRITE_BLOCK:
+        writeBlock( in );
+        datanode.myMetrics.writeBlockOp.inc(DataNode.now() - startTime);
+        if (local)
+          datanode.myMetrics.writesFromLocalClient.inc();
+        else
+          datanode.myMetrics.writesFromRemoteClient.inc();
+        break;
+      case DataTransferProtocol.OP_REPLACE_BLOCK: // for balancing purpose; send to a destination
+        replaceBlock(in);
+        datanode.myMetrics.replaceBlockOp.inc(DataNode.now() - startTime);
+        break;
+      case DataTransferProtocol.OP_COPY_BLOCK:
+            // for balancing purpose; send to a proxy source
+        copyBlock(in);
+        datanode.myMetrics.copyBlockOp.inc(DataNode.now() - startTime);
+        break;
+      case DataTransferProtocol.OP_BLOCK_CHECKSUM: //get the checksum of a block
+        getBlockChecksum(in);
+        datanode.myMetrics.blockChecksumOp.inc(DataNode.now() - startTime);
+        break;
+      default:
+        throw new IOException("Unknown opcode " + op + " in data stream");
+      }
+    } catch (Throwable t) {
+      LOG.error(datanode.dnRegistration + ":DataXceiver",t);
+    } finally {
+      LOG.debug(datanode.dnRegistration + ":Number of active connections is: "
+                               + datanode.getXceiverCount());
+      IOUtils.closeStream(in);
+      IOUtils.closeSocket(s);
+      dataXceiverServer.childSockets.remove(s);
+    }
+  }
+
+  /**
+   * Read a block from the disk.
+   * @param in The stream to read from
+   * @throws IOException
+   */
+  private void readBlock(DataInputStream in) throws IOException {
+    //
+    // Read in the header
+    //
+    long blockId = in.readLong();          
+    Block block = new Block( blockId, 0 , in.readLong());
+
+    long startOffset = in.readLong();
+    long length = in.readLong();
+    String clientName = Text.readString(in);
+    AccessToken accessToken = new AccessToken();
+    accessToken.readFields(in);
+    OutputStream baseStream = NetUtils.getOutputStream(s, 
+        datanode.socketWriteTimeout);
+    DataOutputStream out = new DataOutputStream(
+                 new BufferedOutputStream(baseStream, SMALL_BUFFER_SIZE));
+    
+    if (datanode.isAccessTokenEnabled
+        && !datanode.accessTokenHandler.checkAccess(accessToken, null, blockId,
+            AccessTokenHandler.AccessMode.READ)) {
+      try {
+        out.writeShort(DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN);
+        out.flush();
+        throw new IOException("Access token verification failed, on client "
+            + "request for reading block " + block);
+      } finally {
+        IOUtils.closeStream(out);
+      }
+    }
+    // send the block
+    BlockSender blockSender = null;
+    final String clientTraceFmt =
+      clientName.length() > 0 && ClientTraceLog.isInfoEnabled()
+        ? String.format(DN_CLIENTTRACE_FORMAT, localAddress, remoteAddress,
+            "%d", "HDFS_READ", clientName, "%d",
+            datanode.dnRegistration.getStorageID(), block, "%d")
+        : datanode.dnRegistration + " Served block " + block + " to " +
+            s.getInetAddress();
+    try {
+      try {
+        blockSender = new BlockSender(block, startOffset, length,
+            true, true, false, datanode, clientTraceFmt);
+      } catch(IOException e) {
+        out.writeShort(DataTransferProtocol.OP_STATUS_ERROR);
+        throw e;
+      }
+
+      out.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS); // send op status
+      long read = blockSender.sendBlock(out, baseStream, null); // send data
+
+      if (blockSender.isBlockReadFully()) {
+        // See if client verification succeeded. 
+        // This is an optional response from client.
+        try {
+          if (in.readShort() == DataTransferProtocol.OP_STATUS_CHECKSUM_OK  && 
+              datanode.blockScanner != null) {
+            datanode.blockScanner.verifiedByClient(block);
+          }
+        } catch (IOException ignored) {}
+      }
+      
+      datanode.myMetrics.bytesRead.inc((int) read);
+      datanode.myMetrics.blocksRead.inc();
+    } catch ( SocketException ignored ) {
+      // Its ok for remote side to close the connection anytime.
+      datanode.myMetrics.blocksRead.inc();
+    } catch ( IOException ioe ) {
+      /* What exactly should we do here?
+       * Earlier version shutdown() datanode if there is disk error.
+       */
+      LOG.warn(datanode.dnRegistration +  ":Got exception while serving " + 
+          block + " to " +
+                s.getInetAddress() + ":\n" + 
+                StringUtils.stringifyException(ioe) );
+      throw ioe;
+    } finally {
+      IOUtils.closeStream(out);
+      IOUtils.closeStream(blockSender);
+    }
+  }
+
+  /**
+   * Write a block to disk.
+   * 
+   * @param in The stream to read from
+   * @throws IOException
+   */
+  private void writeBlock(DataInputStream in) throws IOException {
+    DatanodeInfo srcDataNode = null;
+    LOG.debug("writeBlock receive buf size " + s.getReceiveBufferSize() +
+              " tcp no delay " + s.getTcpNoDelay());
+    //
+    // Read in the header
+    //
+    Block block = new Block(in.readLong(), 
+        dataXceiverServer.estimateBlockSize, in.readLong());
+    LOG.info("Receiving block " + block + 
+             " src: " + remoteAddress +
+             " dest: " + localAddress);
+    int pipelineSize = in.readInt(); // num of datanodes in entire pipeline
+    boolean isRecovery = in.readBoolean(); // is this part of recovery?
+    String client = Text.readString(in); // working on behalf of this client
+    boolean hasSrcDataNode = in.readBoolean(); // is src node info present
+    if (hasSrcDataNode) {
+      srcDataNode = new DatanodeInfo();
+      srcDataNode.readFields(in);
+    }
+    int numTargets = in.readInt();
+    if (numTargets < 0) {
+      throw new IOException("Mislabelled incoming datastream.");
+    }
+    DatanodeInfo targets[] = new DatanodeInfo[numTargets];
+    for (int i = 0; i < targets.length; i++) {
+      DatanodeInfo tmp = new DatanodeInfo();
+      tmp.readFields(in);
+      targets[i] = tmp;
+    }
+    AccessToken accessToken = new AccessToken();
+    accessToken.readFields(in);
+    DataOutputStream replyOut = null;   // stream to prev target
+    replyOut = new DataOutputStream(
+                   NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
+    if (datanode.isAccessTokenEnabled
+        && !datanode.accessTokenHandler.checkAccess(accessToken, null, block
+            .getBlockId(), AccessTokenHandler.AccessMode.WRITE)) {
+      try {
+        if (client.length() != 0) {
+          Text.writeString(replyOut, datanode.dnRegistration.getName());
+          replyOut.flush();
+        }
+        throw new IOException("Access token verification failed, on client "
+            + "request for writing block " + block);
+      } finally {
+        IOUtils.closeStream(replyOut);
+      }
+    }
+
+    DataOutputStream mirrorOut = null;  // stream to next target
+    DataInputStream mirrorIn = null;    // reply from next target
+    Socket mirrorSock = null;           // socket to next target
+    BlockReceiver blockReceiver = null; // responsible for data handling
+    String mirrorNode = null;           // the name:port of next target
+    String firstBadLink = "";           // first datanode that failed in connection setup
+    try {
+      // open a block receiver and check if the block does not exist
+      blockReceiver = new BlockReceiver(block, in, 
+          s.getRemoteSocketAddress().toString(),
+          s.getLocalSocketAddress().toString(),
+          isRecovery, client, srcDataNode, datanode);
+
+      //
+      // Open network conn to backup machine, if 
+      // appropriate
+      //
+      if (targets.length > 0) {
+        InetSocketAddress mirrorTarget = null;
+        // Connect to backup machine
+        mirrorNode = targets[0].getName();
+        mirrorTarget = NetUtils.createSocketAddr(mirrorNode);
+        mirrorSock = datanode.newSocket();
+        try {
+          int timeoutValue = numTargets * datanode.socketTimeout;
+          int writeTimeout = datanode.socketWriteTimeout + 
+                             (HdfsConstants.WRITE_TIMEOUT_EXTENSION * numTargets);
+          NetUtils.connect(mirrorSock, mirrorTarget, timeoutValue);
+          mirrorSock.setSoTimeout(timeoutValue);
+          mirrorSock.setSendBufferSize(DEFAULT_DATA_SOCKET_SIZE);
+          mirrorOut = new DataOutputStream(
+             new BufferedOutputStream(
+                         NetUtils.getOutputStream(mirrorSock, writeTimeout),
+                         SMALL_BUFFER_SIZE));
+          mirrorIn = new DataInputStream(NetUtils.getInputStream(mirrorSock));
+
+          // Write header: Copied from DFSClient.java!
+          mirrorOut.writeShort( DataTransferProtocol.DATA_TRANSFER_VERSION );
+          mirrorOut.write( DataTransferProtocol.OP_WRITE_BLOCK );
+          mirrorOut.writeLong( block.getBlockId() );
+          mirrorOut.writeLong( block.getGenerationStamp() );
+          mirrorOut.writeInt( pipelineSize );
+          mirrorOut.writeBoolean( isRecovery );
+          Text.writeString( mirrorOut, client );
+          mirrorOut.writeBoolean(hasSrcDataNode);
+          if (hasSrcDataNode) { // pass src node information
+            srcDataNode.write(mirrorOut);
+          }
+          mirrorOut.writeInt( targets.length - 1 );
+          for ( int i = 1; i < targets.length; i++ ) {
+            targets[i].write( mirrorOut );
+          }
+          accessToken.write(mirrorOut);
+
+          blockReceiver.writeChecksumHeader(mirrorOut);
+          mirrorOut.flush();
+
+          // read connect ack (only for clients, not for replication req)
+          if (client.length() != 0) {
+            firstBadLink = Text.readString(mirrorIn);
+            if (LOG.isDebugEnabled() || firstBadLink.length() > 0) {
+              LOG.info("Datanode " + targets.length +
+                       " got response for connect ack " +
+                       " from downstream datanode with firstbadlink as " +
+                       firstBadLink);
+            }
+          }
+
+        } catch (IOException e) {
+          if (client.length() != 0) {
+            Text.writeString(replyOut, mirrorNode);
+            replyOut.flush();
+          }
+          IOUtils.closeStream(mirrorOut);
+          mirrorOut = null;
+          IOUtils.closeStream(mirrorIn);
+          mirrorIn = null;
+          IOUtils.closeSocket(mirrorSock);
+          mirrorSock = null;
+          if (client.length() > 0) {
+            throw e;
+          } else {
+            LOG.info(datanode.dnRegistration + ":Exception transfering block " +
+                     block + " to mirror " + mirrorNode +
+                     ". continuing without the mirror.\n" +
+                     StringUtils.stringifyException(e));
+          }
+        }
+      }
+
+      // send connect ack back to source (only for clients)
+      if (client.length() != 0) {
+        if (LOG.isDebugEnabled() || firstBadLink.length() > 0) {
+          LOG.info("Datanode " + targets.length +
+                   " forwarding connect ack to upstream firstbadlink is " +
+                   firstBadLink);
+        }
+        Text.writeString(replyOut, firstBadLink);
+        replyOut.flush();
+      }
+
+      // receive the block and mirror to the next target
+      String mirrorAddr = (mirrorSock == null) ? null : mirrorNode;
+      blockReceiver.receiveBlock(mirrorOut, mirrorIn, replyOut,
+                                 mirrorAddr, null, targets.length);
+
+      // if this write is for a replication request (and not
+      // from a client), then confirm block. For client-writes,
+      // the block is finalized in the PacketResponder.
+      if (client.length() == 0) {
+        datanode.notifyNamenodeReceivedBlock(block, DataNode.EMPTY_DEL_HINT);
+        LOG.info("Received block " + block + 
+                 " src: " + remoteAddress +
+                 " dest: " + localAddress +
+                 " of size " + block.getNumBytes());
+      }
+
+      if (datanode.blockScanner != null) {
+        datanode.blockScanner.addBlock(block);
+      }
+      
+    } catch (IOException ioe) {
+      LOG.info("writeBlock " + block + " received exception " + ioe);
+      throw ioe;
+    } finally {
+      // close all opened streams
+      IOUtils.closeStream(mirrorOut);
+      IOUtils.closeStream(mirrorIn);
+      IOUtils.closeStream(replyOut);
+      IOUtils.closeSocket(mirrorSock);
+      IOUtils.closeStream(blockReceiver);
+    }
+  }
+
+  /**
+   * Get block checksum (MD5 of CRC32).
+   * @param in
+   */
+  void getBlockChecksum(DataInputStream in) throws IOException {
+    final Block block = new Block(in.readLong(), 0 , in.readLong());
+    AccessToken accessToken = new AccessToken();
+    accessToken.readFields(in);
+    DataOutputStream out = new DataOutputStream(NetUtils.getOutputStream(s,
+        datanode.socketWriteTimeout));
+    if (datanode.isAccessTokenEnabled
+        && !datanode.accessTokenHandler.checkAccess(accessToken, null, block
+            .getBlockId(), AccessTokenHandler.AccessMode.READ)) {
+      try {
+        out.writeShort(DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN);
+        out.flush();
+        throw new IOException(
+            "Access token verification failed, on getBlockChecksum() "
+                + "for block " + block);
+      } finally {
+        IOUtils.closeStream(out);
+      }
+    }
+
+    final MetaDataInputStream metadataIn = datanode.data.getMetaDataInputStream(block);
+    final DataInputStream checksumIn = new DataInputStream(new BufferedInputStream(
+        metadataIn, BUFFER_SIZE));
+
+    try {
+      //read metadata file
+      final BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
+      final DataChecksum checksum = header.getChecksum(); 
+      final int bytesPerCRC = checksum.getBytesPerChecksum();
+      final long crcPerBlock = (metadataIn.getLength()
+          - BlockMetadataHeader.getHeaderSize())/checksum.getChecksumSize();
+      
+      //compute block checksum
+      final MD5Hash md5 = MD5Hash.digest(checksumIn);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("block=" + block + ", bytesPerCRC=" + bytesPerCRC
+            + ", crcPerBlock=" + crcPerBlock + ", md5=" + md5);
+      }
+
+      //write reply
+      out.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS);
+      out.writeInt(bytesPerCRC);
+      out.writeLong(crcPerBlock);
+      md5.write(out);
+      out.flush();
+    } finally {
+      IOUtils.closeStream(out);
+      IOUtils.closeStream(checksumIn);
+      IOUtils.closeStream(metadataIn);
+    }
+  }
+
+  /**
+   * Read a block from the disk and then sends it to a destination.
+   * 
+   * @param in The stream to read from
+   * @throws IOException
+   */
+  private void copyBlock(DataInputStream in) throws IOException {
+    // Read in the header
+    long blockId = in.readLong(); // read block id
+    Block block = new Block(blockId, 0, in.readLong());
+    AccessToken accessToken = new AccessToken();
+    accessToken.readFields(in);
+    if (datanode.isAccessTokenEnabled
+        && !datanode.accessTokenHandler.checkAccess(accessToken, null, blockId,
+            AccessTokenHandler.AccessMode.COPY)) {
+      LOG.warn("Invalid access token in request from "
+          + s.getRemoteSocketAddress() + " for copying block " + block);
+      sendResponse(s,
+          (short) DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN,
+          datanode.socketWriteTimeout);
+      return;
+    }
+
+    if (!dataXceiverServer.balanceThrottler.acquire()) { // not able to start
+      LOG.info("Not able to copy block " + blockId + " to " 
+          + s.getRemoteSocketAddress() + " because threads quota is exceeded.");
+      sendResponse(s, (short)DataTransferProtocol.OP_STATUS_ERROR, 
+          datanode.socketWriteTimeout);
+      return;
+    }
+
+    BlockSender blockSender = null;
+    DataOutputStream reply = null;
+    boolean isOpSuccess = true;
+
+    try {
+      // check if the block exists or not
+      blockSender = new BlockSender(block, 0, -1, false, false, false, 
+          datanode);
+
+      // set up response stream
+      OutputStream baseStream = NetUtils.getOutputStream(
+          s, datanode.socketWriteTimeout);
+      reply = new DataOutputStream(new BufferedOutputStream(
+          baseStream, SMALL_BUFFER_SIZE));
+
+      // send status first
+      reply.writeShort((short)DataTransferProtocol.OP_STATUS_SUCCESS);
+      // send block content to the target
+      long read = blockSender.sendBlock(reply, baseStream, 
+                                        dataXceiverServer.balanceThrottler);
+
+      datanode.myMetrics.bytesRead.inc((int) read);
+      datanode.myMetrics.blocksRead.inc();
+      
+      LOG.info("Copied block " + block + " to " + s.getRemoteSocketAddress());
+    } catch (IOException ioe) {
+      isOpSuccess = false;
+      throw ioe;
+    } finally {
+      dataXceiverServer.balanceThrottler.release();
+      if (isOpSuccess) {
+        try {
+          // send one last byte to indicate that the resource is cleaned.
+          reply.writeChar('d');
+        } catch (IOException ignored) {
+        }
+      }
+      IOUtils.closeStream(reply);
+      IOUtils.closeStream(blockSender);
+    }
+  }
+
+  /**
+   * Receive a block and write it to disk, it then notifies the namenode to
+   * remove the copy from the source.
+   * 
+   * @param in The stream to read from
+   * @throws IOException
+   */
+  private void replaceBlock(DataInputStream in) throws IOException {
+    /* read header */
+    long blockId = in.readLong();
+    Block block = new Block(blockId, dataXceiverServer.estimateBlockSize,
+        in.readLong()); // block id & generation stamp
+    String sourceID = Text.readString(in); // read del hint
+    DatanodeInfo proxySource = new DatanodeInfo(); // read proxy source
+    proxySource.readFields(in);
+    AccessToken accessToken = new AccessToken();
+    accessToken.readFields(in);
+    if (datanode.isAccessTokenEnabled
+        && !datanode.accessTokenHandler.checkAccess(accessToken, null, blockId,
+            AccessTokenHandler.AccessMode.REPLACE)) {
+      LOG.warn("Invalid access token in request from "
+          + s.getRemoteSocketAddress() + " for replacing block " + block);
+      sendResponse(s, (short)DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN,
+          datanode.socketWriteTimeout);
+      return;
+    }
+
+    if (!dataXceiverServer.balanceThrottler.acquire()) { // not able to start
+      LOG.warn("Not able to receive block " + blockId + " from " 
+          + s.getRemoteSocketAddress() + " because threads quota is exceeded.");
+      sendResponse(s, (short)DataTransferProtocol.OP_STATUS_ERROR, 
+          datanode.socketWriteTimeout);
+      return;
+    }
+
+    Socket proxySock = null;
+    DataOutputStream proxyOut = null;
+    short opStatus = DataTransferProtocol.OP_STATUS_SUCCESS;
+    BlockReceiver blockReceiver = null;
+    DataInputStream proxyReply = null;
+    
+    try {
+      // get the output stream to the proxy
+      InetSocketAddress proxyAddr = NetUtils.createSocketAddr(
+          proxySource.getName());
+      proxySock = datanode.newSocket();
+      NetUtils.connect(proxySock, proxyAddr, datanode.socketTimeout);
+      proxySock.setSoTimeout(datanode.socketTimeout);
+
+      OutputStream baseStream = NetUtils.getOutputStream(proxySock, 
+          datanode.socketWriteTimeout);
+      proxyOut = new DataOutputStream(
+                     new BufferedOutputStream(baseStream, SMALL_BUFFER_SIZE));
+
+      /* send request to the proxy */
+      proxyOut.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION); // transfer version
+      proxyOut.writeByte(DataTransferProtocol.OP_COPY_BLOCK); // op code
+      proxyOut.writeLong(block.getBlockId()); // block id
+      proxyOut.writeLong(block.getGenerationStamp()); // block id
+      accessToken.write(proxyOut);
+      proxyOut.flush();
+
+      // receive the response from the proxy
+      proxyReply = new DataInputStream(new BufferedInputStream(
+          NetUtils.getInputStream(proxySock), BUFFER_SIZE));
+      short status = proxyReply.readShort();
+      if (status != DataTransferProtocol.OP_STATUS_SUCCESS) {
+        if (status == DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN) {
+          throw new IOException("Copy block " + block + " from "
+              + proxySock.getRemoteSocketAddress()
+              + " failed due to access token error");
+        }
+        throw new IOException("Copy block " + block + " from "
+            + proxySock.getRemoteSocketAddress() + " failed");
+      }
+      // open a block receiver and check if the block does not exist
+      blockReceiver = new BlockReceiver(
+          block, proxyReply, proxySock.getRemoteSocketAddress().toString(),
+          proxySock.getLocalSocketAddress().toString(),
+          false, "", null, datanode);
+
+      // receive a block
+      blockReceiver.receiveBlock(null, null, null, null, 
+          dataXceiverServer.balanceThrottler, -1);
+                    
+      // notify name node
+      datanode.notifyNamenodeReceivedBlock(block, sourceID);
+
+      LOG.info("Moved block " + block + 
+          " from " + s.getRemoteSocketAddress());
+      
+    } catch (IOException ioe) {
+      opStatus = DataTransferProtocol.OP_STATUS_ERROR;
+      throw ioe;
+    } finally {
+      // receive the last byte that indicates the proxy released its thread resource
+      if (opStatus == DataTransferProtocol.OP_STATUS_SUCCESS) {
+        try {
+          proxyReply.readChar();
+        } catch (IOException ignored) {
+        }
+      }
+      
+      // now release the thread resource
+      dataXceiverServer.balanceThrottler.release();
+      
+      // send response back
+      try {
+        sendResponse(s, opStatus, datanode.socketWriteTimeout);
+      } catch (IOException ioe) {
+        LOG.warn("Error writing reply back to " + s.getRemoteSocketAddress());
+      }
+      IOUtils.closeStream(proxyOut);
+      IOUtils.closeStream(blockReceiver);
+      IOUtils.closeStream(proxyReply);
+    }
+  }
+  
+  /**
+   * Utility function for sending a response.
+   * @param s socket to write to
+   * @param opStatus status message to write
+   * @param timeout send timeout
+   **/
+  private void sendResponse(Socket s, short opStatus, long timeout) 
+                                                       throws IOException {
+    DataOutputStream reply = 
+      new DataOutputStream(NetUtils.getOutputStream(s, timeout));
+    try {
+      reply.writeShort(opStatus);
+      reply.flush();
+    } finally {
+      IOUtils.closeStream(reply);
+    }
+  }
+}

+ 175 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java

@@ -0,0 +1,175 @@
+/**
+ * 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.datanode;
+
+import java.io.IOException;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.balancer.Balancer;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Server used for receiving/sending a block of data.
+ * This is created to listen for requests from clients or 
+ * other DataNodes.  This small server does not use the 
+ * Hadoop IPC mechanism.
+ */
+class DataXceiverServer implements Runnable, FSConstants {
+  public static final Log LOG = DataNode.LOG;
+  
+  ServerSocket ss;
+  DataNode datanode;
+  // Record all sockets opend for data transfer
+  Map<Socket, Socket> childSockets = Collections.synchronizedMap(
+                                       new HashMap<Socket, Socket>());
+  
+  /**
+   * Maximal number of concurrent xceivers per node.
+   * Enforcing the limit is required in order to avoid data-node
+   * running out of memory.
+   */
+  static final int MAX_XCEIVER_COUNT = 256;
+  int maxXceiverCount = MAX_XCEIVER_COUNT;
+
+  /** A manager to make sure that cluster balancing does not
+   * take too much resources.
+   * 
+   * It limits the number of block moves for balancing and
+   * the total amount of bandwidth they can use.
+   */
+  static class BlockBalanceThrottler extends BlockTransferThrottler {
+   private int numThreads;
+   
+   /**Constructor
+    * 
+    * @param bandwidth Total amount of bandwidth can be used for balancing 
+    */
+   private BlockBalanceThrottler(long bandwidth) {
+     super(bandwidth);
+     LOG.info("Balancing bandwith is "+ bandwidth + " bytes/s");
+   }
+   
+   /** Check if the block move can start. 
+    * 
+    * Return true if the thread quota is not exceeded and 
+    * the counter is incremented; False otherwise.
+    */
+   synchronized boolean acquire() {
+     if (numThreads >= Balancer.MAX_NUM_CONCURRENT_MOVES) {
+       return false;
+     }
+     numThreads++;
+     return true;
+   }
+   
+   /** Mark that the move is completed. The thread counter is decremented. */
+   synchronized void release() {
+     numThreads--;
+   }
+  }
+
+  BlockBalanceThrottler balanceThrottler;
+  
+  /**
+   * We need an estimate for block size to check if the disk partition has
+   * enough space. For now we set it to be the default block size set
+   * in the server side configuration, which is not ideal because the
+   * default block size should be a client-size configuration. 
+   * A better solution is to include in the header the estimated block size,
+   * i.e. either the actual block size or the default block size.
+   */
+  long estimateBlockSize;
+  
+  
+  DataXceiverServer(ServerSocket ss, Configuration conf, 
+      DataNode datanode) {
+    
+    this.ss = ss;
+    this.datanode = datanode;
+    
+    this.maxXceiverCount = conf.getInt("dfs.datanode.max.xcievers",
+        MAX_XCEIVER_COUNT);
+    
+    this.estimateBlockSize = conf.getLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    
+    //set up parameter for cluster balancing
+    this.balanceThrottler = new BlockBalanceThrottler(
+      conf.getLong("dfs.balance.bandwidthPerSec", 1024L*1024));
+  }
+
+  /**
+   */
+  public void run() {
+    while (datanode.shouldRun) {
+      try {
+        Socket s = ss.accept();
+        s.setTcpNoDelay(true);
+        new Daemon(datanode.threadGroup, 
+            new DataXceiver(s, datanode, this)).start();
+      } catch (SocketTimeoutException ignored) {
+        // wake up to see if should continue to run
+      } catch (IOException ie) {
+        LOG.warn(datanode.dnRegistration + ":DataXceiveServer: " 
+                                + StringUtils.stringifyException(ie));
+      } catch (Throwable te) {
+        LOG.error(datanode.dnRegistration + ":DataXceiveServer: Exiting due to:" 
+                                 + StringUtils.stringifyException(te));
+        datanode.shouldRun = false;
+      }
+    }
+    try {
+      ss.close();
+    } catch (IOException ie) {
+      LOG.warn(datanode.dnRegistration + ":DataXceiveServer: " 
+                              + StringUtils.stringifyException(ie));
+    }
+  }
+  
+  void kill() {
+    assert datanode.shouldRun == false :
+      "shoudRun should be set to false before killing";
+    try {
+      this.ss.close();
+    } catch (IOException ie) {
+      LOG.warn(datanode.dnRegistration + ":DataXceiveServer.kill(): " 
+                              + StringUtils.stringifyException(ie));
+    }
+
+    // close all the sockets that were accepted earlier
+    synchronized (childSockets) {
+      for (Iterator<Socket> it = childSockets.values().iterator();
+           it.hasNext();) {
+        Socket thissock = it.next();
+        try {
+          thissock.close();
+        } catch (IOException e) {
+        }
+      }
+    }
+  }
+}

+ 138 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/DatanodeBlockInfo.java

@@ -0,0 +1,138 @@
+/**
+ * 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.datanode;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FileUtil.HardLink;
+import org.apache.hadoop.io.IOUtils;
+
+/**
+ * This class is used by the datanode to maintain the map from a block 
+ * to its metadata.
+ */
+class DatanodeBlockInfo {
+
+  private FSVolume volume;       // volume where the block belongs
+  private File     file;         // block file
+  private boolean detached;      // copy-on-write done for block
+
+  DatanodeBlockInfo(FSVolume vol, File file) {
+    this.volume = vol;
+    this.file = file;
+    detached = false;
+  }
+
+  DatanodeBlockInfo(FSVolume vol) {
+    this.volume = vol;
+    this.file = null;
+    detached = false;
+  }
+
+  FSVolume getVolume() {
+    return volume;
+  }
+
+  File getFile() {
+    return file;
+  }
+
+  void setFile(File f) {
+    file = f;
+  }
+
+  /**
+   * Is this block already detached?
+   */
+  boolean isDetached() {
+    return detached;
+  }
+
+  /**
+   *  Block has been successfully detached
+   */
+  void setDetached() {
+    detached = true;
+  }
+
+  /**
+   * Copy specified file into a temporary file. Then rename the
+   * temporary file to the original name. This will cause any
+   * hardlinks to the original file to be removed. The temporary
+   * files are created in the detachDir. The temporary files will
+   * be recovered (especially on Windows) on datanode restart.
+   */
+  private void detachFile(File file, Block b) throws IOException {
+    File tmpFile = volume.createDetachFile(b, file.getName());
+    try {
+      IOUtils.copyBytes(new FileInputStream(file),
+                        new FileOutputStream(tmpFile),
+                        16*1024, true);
+      if (file.length() != tmpFile.length()) {
+        throw new IOException("Copy of file " + file + " size " + file.length()+
+                              " into file " + tmpFile +
+                              " resulted in a size of " + tmpFile.length());
+      }
+      FileUtil.replaceFile(tmpFile, file);
+    } catch (IOException e) {
+      boolean done = tmpFile.delete();
+      if (!done) {
+        DataNode.LOG.info("detachFile failed to delete temporary file " +
+                          tmpFile);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Returns true if this block was copied, otherwise returns false.
+   */
+  boolean detachBlock(Block block, int numLinks) throws IOException {
+    if (isDetached()) {
+      return false;
+    }
+    if (file == null || volume == null) {
+      throw new IOException("detachBlock:Block not found. " + block);
+    }
+    File meta = FSDataset.getMetaFile(file, block);
+    if (meta == null) {
+      throw new IOException("Meta file not found for block " + block);
+    }
+
+    if (HardLink.getLinkCount(file) > numLinks) {
+      DataNode.LOG.info("CopyOnWrite for block " + block);
+      detachFile(file, block);
+    }
+    if (HardLink.getLinkCount(meta) > numLinks) {
+      detachFile(meta, block);
+    }
+    setDetached();
+    return true;
+  }
+  
+  public String toString() {
+    return getClass().getSimpleName() + "(volume=" + volume
+        + ", file=" + file + ", detached=" + detached + ")";
+  }
+}

+ 314 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java

@@ -0,0 +1,314 @@
+/**
+ * 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.datanode;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.Random;
+
+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.datanode.FSDataset.FSVolume;
+
+/**
+ * Periodically scans the data directories for block and block metadata files.
+ * Reconciles the differences with block information maintained in
+ * {@link FSDataset}
+ */
+public class DirectoryScanner {
+  private static final Log LOG = LogFactory.getLog(DirectoryScanner.class);
+  private static final int DEFAULT_SCAN_INTERVAL = 21600;
+
+  private final FSDataset dataset;
+  private long scanPeriod;
+  private long lastScanTime;
+
+  LinkedList<ScanInfo> diff = new LinkedList<ScanInfo>();
+
+  /** Stats tracked for reporting and testing */
+  long totalBlocks;
+  long missingMetaFile;
+  long missingBlockFile;
+  long missingMemoryBlocks;
+  long mismatchBlocks;
+
+  /**
+   * Tracks the files and other information related to a block on the disk
+   * Missing file is indicated by setting the corresponding member
+   * to null.
+   */
+  static class ScanInfo implements Comparable<ScanInfo> {
+    private final long blockId;
+    private final File metaFile;
+    private final File blockFile;
+    private final FSVolume volume;
+
+    ScanInfo(long blockId) {
+      this(blockId, null, null, null);
+    }
+
+    ScanInfo(long blockId, File blockFile, File metaFile, FSVolume vol) {
+      this.blockId = blockId;
+      this.metaFile = metaFile;
+      this.blockFile = blockFile;
+      this.volume = vol;
+    }
+
+    File getMetaFile() {
+      return metaFile;
+    }
+
+    File getBlockFile() {
+      return blockFile;
+    }
+
+    long getBlockId() {
+      return blockId;
+    }
+
+    FSVolume getVolume() {
+      return volume;
+    }
+
+    @Override // Comparable
+    public int compareTo(ScanInfo b) {
+      if (blockId < b.blockId) {
+        return -1;
+      } else if (blockId == b.blockId) {
+        return 0;
+      } else {
+        return 1;
+      }
+    }
+
+    @Override // Object
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof ScanInfo)) {
+        return false;
+      }
+      return blockId == ((ScanInfo) o).blockId;
+    }
+
+    @Override // Object
+    public int hashCode() {
+      return 37 * 17 + (int) (blockId^(blockId>>>32));
+    }
+
+    public long getGenStamp() {
+      return metaFile != null ? Block.getGenerationStamp(metaFile.getName()) :
+        Block.GRANDFATHER_GENERATION_STAMP;
+    }
+  }
+
+  DirectoryScanner(FSDataset dataset, Configuration conf) {
+    this.dataset = dataset;
+    int interval = conf.getInt("dfs.datanode.directoryscan.interval",
+        DEFAULT_SCAN_INTERVAL);
+    scanPeriod = interval * 1000L;
+
+    Random rand = new Random();
+    lastScanTime = System.currentTimeMillis() - (rand.nextInt(interval) * 1000L);
+    LOG.info("scan starts at " + (lastScanTime + scanPeriod)
+        + " with interval " + scanPeriod);
+  }
+
+  boolean newScanPeriod(long now) {
+    return now > lastScanTime + scanPeriod;
+  }
+
+  private void clear() {
+    diff.clear();
+    totalBlocks = 0;
+    missingMetaFile = 0;
+    missingBlockFile = 0;
+    missingMemoryBlocks = 0;
+    mismatchBlocks = 0;
+  }
+
+  /**
+   * Reconcile differences between disk and in-memory blocks
+   */
+  void reconcile() {
+    scan();
+    for (ScanInfo info : diff) {
+      dataset.checkAndUpdate(info.getBlockId(), info.getBlockFile(), info
+          .getMetaFile(), info.getVolume());
+    }
+  }
+
+  /**
+   * Scan for the differences between disk and in-memory blocks
+   */
+  void scan() {
+    clear();
+    ScanInfo[] diskReport = getDiskReport();
+    totalBlocks = diskReport.length;
+
+    // Hold FSDataset lock to prevent further changes to the block map
+    synchronized(dataset) {
+      Block[] memReport = dataset.getBlockList(false);
+      Arrays.sort(memReport); // Sort based on blockId
+
+      int d = 0; // index for diskReport
+      int m = 0; // index for memReprot
+      while (m < memReport.length && d < diskReport.length) {
+        Block memBlock = memReport[Math.min(m, memReport.length - 1)];
+        ScanInfo info = diskReport[Math.min(d, diskReport.length - 1)];
+        if (info.getBlockId() < memBlock.getBlockId()) {
+          // Block is missing in memory
+          missingMemoryBlocks++;
+          addDifference(info);
+          d++;
+          continue;
+        }
+        if (info.getBlockId() > memBlock.getBlockId()) {
+          // Block is missing on the disk
+          addDifference(memBlock.getBlockId());
+          m++;
+          continue;
+        }
+        // Block file and/or metadata file exists on the disk
+        // Block exists in memory
+        if (info.getBlockFile() == null) {
+          // Block metadata file exits and block file is missing
+          addDifference(info);
+        } else if (info.getGenStamp() != memBlock.getGenerationStamp()
+            || info.getBlockFile().length() != memBlock.getNumBytes()) {
+          mismatchBlocks++;
+          addDifference(info);
+        }
+        d++;
+        m++;
+      }
+      while (m < memReport.length) {
+        addDifference(memReport[m++].getBlockId());
+      }
+      while (d < diskReport.length) {
+        missingMemoryBlocks++;
+        addDifference(diskReport[d++]);
+      }
+    }
+    LOG.info("Total blocks: " + totalBlocks + ", missing metadata files:"
+        + missingMetaFile + ", missing block files:" + missingBlockFile
+        + ", missing blocks in memory:" + missingMemoryBlocks
+        + ", mismatched blocks:" + mismatchBlocks);
+    lastScanTime = System.currentTimeMillis();
+  }
+
+  /**
+   * Block is found on the disk. In-memory block is missing or does not match
+   * the block on the disk
+   */
+  private void addDifference(ScanInfo info) {
+    missingMetaFile += info.getMetaFile() == null ? 1 : 0;
+    missingBlockFile += info.getBlockFile() == null ? 1 : 0;
+    diff.add(info);
+  }
+
+  /** Block is not found on the disk */
+  private void addDifference(long blockId) {
+    missingBlockFile++;
+    missingMetaFile++;
+    diff.add(new ScanInfo(blockId));
+  }
+
+  /** Get list of blocks on the disk sorted by blockId */
+  private ScanInfo[] getDiskReport() {
+    // First get list of data directories
+    FSDataset.FSVolume[] volumes = dataset.volumes.volumes;
+    ArrayList<LinkedList<ScanInfo>> dirReports =
+      new ArrayList<LinkedList<ScanInfo>>(volumes.length);
+    for (int i = 0; i < volumes.length; i++) {
+      if (!dataset.volumes.isValid(volumes[i])) { // volume is still valid
+        dirReports.add(i, null);
+      } else {
+        LinkedList<ScanInfo> dirReport = new LinkedList<ScanInfo>();
+        dirReports.add(i, compileReport(volumes[i], volumes[i].getDir(),
+            dirReport));
+      }
+    }
+
+    // Compile consolidated report for all the volumes
+    LinkedList<ScanInfo> list = new LinkedList<ScanInfo>();
+    for (int i = 0; i < volumes.length; i++) {
+      if (dataset.volumes.isValid(volumes[i])) { // volume is still valid
+        list.addAll(dirReports.get(i));
+      }
+    }
+
+    ScanInfo[] report = list.toArray(new ScanInfo[list.size()]);
+    // Sort the report based on blockId
+    Arrays.sort(report);
+    return report;
+  }
+
+  private static boolean isBlockMetaFile(String blockId, String metaFile) {
+    return metaFile.startsWith(blockId)
+        && metaFile.endsWith(Block.METADATA_EXTENSION);
+  }
+
+  /** Compile list {@link ScanInfo} for the blocks in the directory <dir>*/
+  private LinkedList<ScanInfo> compileReport(FSVolume vol, File dir,
+      LinkedList<ScanInfo> report) {
+    File[] files = dir.listFiles();
+    Arrays.sort(files);
+
+    /* Assumption: In the sorted list of files block file appears immediately
+     * before block metadata file. This is true for the current naming
+     * convention for block file blk_<blockid> and meta file
+     * blk_<blockid>_<genstamp>.meta
+     */
+    for (int i = 0; i < files.length; i++) {
+      if (files[i].isDirectory()) {
+        compileReport(vol, files[i], report);
+        continue;
+      }
+      if (!Block.isBlockFilename(files[i])) {
+        if (isBlockMetaFile("blk_", files[i].getName())) {
+          long blockId = Block.getBlockId(files[i].getName());
+          report.add(new ScanInfo(blockId, null, files[i], vol));
+        }
+        continue;
+      }
+      File blockFile = files[i];
+      long blockId = Block.filename2id(blockFile.getName());
+      File metaFile = null;
+
+      // Skip all the files that start with block name until
+      // getting to the metafile for the block
+      while (i + 1 < files.length
+          && files[i+1].isFile()
+          && files[i + 1].getName().startsWith(blockFile.getName())) {
+        i++;
+        if (isBlockMetaFile(blockFile.getName(), files[i].getName())) {
+          metaFile = files[i];
+          break;
+        }
+      }
+      report.add(new ScanInfo(blockId, blockFile, metaFile, vol));
+    }
+    return report;
+  }
+}

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

@@ -0,0 +1,1622 @@
+/**
+ * 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.datanode;
+
+import java.io.*;
+import java.util.*;
+
+import javax.management.NotCompliantMBeanException;
+import javax.management.ObjectName;
+import javax.management.StandardMBean;
+
+import org.apache.hadoop.fs.*;
+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.metrics.util.MBeanUtil;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.DiskChecker;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
+import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
+
+/**************************************************
+ * FSDataset manages a set of data blocks.  Each block
+ * has a unique name and an extent on disk.
+ *
+ ***************************************************/
+public class FSDataset implements FSConstants, FSDatasetInterface {
+
+
+  /**
+   * A node type that can be built into a tree reflecting the
+   * hierarchy of blocks on the local disk.
+   */
+  class FSDir {
+    File dir;
+    int numBlocks = 0;
+    FSDir children[];
+    int lastChildIdx = 0;
+    /**
+     */
+    public FSDir(File dir) 
+      throws IOException {
+      this.dir = dir;
+      this.children = null;
+      if (!dir.exists()) {
+        if (!dir.mkdirs()) {
+          throw new IOException("Mkdirs failed to create " + 
+                                dir.toString());
+        }
+      } else {
+        File[] files = dir.listFiles();
+        int numChildren = 0;
+        for (int idx = 0; idx < files.length; idx++) {
+          if (files[idx].isDirectory()) {
+            numChildren++;
+          } else if (Block.isBlockFilename(files[idx])) {
+            numBlocks++;
+          }
+        }
+        if (numChildren > 0) {
+          children = new FSDir[numChildren];
+          int curdir = 0;
+          for (int idx = 0; idx < files.length; idx++) {
+            if (files[idx].isDirectory()) {
+              children[curdir] = new FSDir(files[idx]);
+              curdir++;
+            }
+          }
+        }
+      }
+    }
+        
+    public File addBlock(Block b, File src) throws IOException {
+      //First try without creating subdirectories
+      File file = addBlock(b, src, false, false);          
+      return (file != null) ? file : addBlock(b, src, true, true);
+    }
+
+    private File addBlock(Block b, File src, boolean createOk, 
+                          boolean resetIdx) throws IOException {
+      if (numBlocks < maxBlocksPerDir) {
+        File dest = new File(dir, b.getBlockName());
+        File metaData = getMetaFile( src, b );
+        File newmeta = getMetaFile(dest, b);
+        if ( ! metaData.renameTo( newmeta ) ||
+            ! src.renameTo( dest ) ) {
+          throw new IOException( "could not move files for " + b +
+                                 " from tmp to " + 
+                                 dest.getAbsolutePath() );
+        }
+        if (DataNode.LOG.isDebugEnabled()) {
+          DataNode.LOG.debug("addBlock: Moved " + metaData + " to " + newmeta);
+          DataNode.LOG.debug("addBlock: Moved " + src + " to " + dest);
+        }
+
+        numBlocks += 1;
+        return dest;
+      }
+            
+      if (lastChildIdx < 0 && resetIdx) {
+        //reset so that all children will be checked
+        lastChildIdx = random.nextInt(children.length);              
+      }
+            
+      if (lastChildIdx >= 0 && children != null) {
+        //Check if any child-tree has room for a block.
+        for (int i=0; i < children.length; i++) {
+          int idx = (lastChildIdx + i)%children.length;
+          File file = children[idx].addBlock(b, src, false, resetIdx);
+          if (file != null) {
+            lastChildIdx = idx;
+            return file; 
+          }
+        }
+        lastChildIdx = -1;
+      }
+            
+      if (!createOk) {
+        return null;
+      }
+            
+      if (children == null || children.length == 0) {
+        children = new FSDir[maxBlocksPerDir];
+        for (int idx = 0; idx < maxBlocksPerDir; idx++) {
+          children[idx] = new FSDir(new File(dir, DataStorage.BLOCK_SUBDIR_PREFIX+idx));
+        }
+      }
+            
+      //now pick a child randomly for creating a new set of subdirs.
+      lastChildIdx = random.nextInt(children.length);
+      return children[ lastChildIdx ].addBlock(b, src, true, false); 
+    }
+
+    /** Find the metadata file for the specified block file.
+     * Return the generation stamp from the name of the metafile.
+     */
+    long getGenerationStampFromFile(File[] listdir, File blockFile) {
+      String blockName = blockFile.getName();
+      for (int j = 0; j < listdir.length; j++) {
+        String path = listdir[j].getName();
+        if (!path.startsWith(blockName)) {
+          continue;
+        }
+        if (blockFile == listdir[j]) {
+          continue;
+        }
+        return Block.getGenerationStamp(listdir[j].getName());
+      }
+      DataNode.LOG.warn("Block " + blockFile + 
+                        " does not have a metafile!");
+      return Block.GRANDFATHER_GENERATION_STAMP;
+    }
+
+    void getVolumeMap(HashMap<Block, DatanodeBlockInfo> volumeMap, FSVolume volume) {
+      if (children != null) {
+        for (int i = 0; i < children.length; i++) {
+          children[i].getVolumeMap(volumeMap, volume);
+        }
+      }
+
+      File blockFiles[] = dir.listFiles();
+      for (int i = 0; i < blockFiles.length; i++) {
+        if (Block.isBlockFilename(blockFiles[i])) {
+          long genStamp = getGenerationStampFromFile(blockFiles, blockFiles[i]);
+          volumeMap.put(new Block(blockFiles[i], blockFiles[i].length(), genStamp), 
+                        new DatanodeBlockInfo(volume, blockFiles[i]));
+        }
+      }
+    }
+        
+    /**
+     * check if a data diretory is healthy
+     * @throws DiskErrorException
+     */
+    public void checkDirTree() throws DiskErrorException {
+      DiskChecker.checkDir(dir);
+            
+      if (children != null) {
+        for (int i = 0; i < children.length; i++) {
+          children[i].checkDirTree();
+        }
+      }
+    }
+        
+    void clearPath(File f) {
+      String root = dir.getAbsolutePath();
+      String dir = f.getAbsolutePath();
+      if (dir.startsWith(root)) {
+        String[] dirNames = dir.substring(root.length()).
+          split(File.separator + "subdir");
+        if (clearPath(f, dirNames, 1))
+          return;
+      }
+      clearPath(f, null, -1);
+    }
+        
+    /*
+     * dirNames is an array of string integers derived from
+     * usual directory structure data/subdirN/subdirXY/subdirM ...
+     * If dirName array is non-null, we only check the child at 
+     * the children[dirNames[idx]]. This avoids iterating over
+     * children in common case. If directory structure changes 
+     * in later versions, we need to revisit this.
+     */
+    private boolean clearPath(File f, String[] dirNames, int idx) {
+      if ((dirNames == null || idx == dirNames.length) &&
+          dir.compareTo(f) == 0) {
+        numBlocks--;
+        return true;
+      }
+          
+      if (dirNames != null) {
+        //guess the child index from the directory name
+        if (idx > (dirNames.length - 1) || children == null) {
+          return false;
+        }
+        int childIdx; 
+        try {
+          childIdx = Integer.parseInt(dirNames[idx]);
+        } catch (NumberFormatException ignored) {
+          // layout changed? we could print a warning.
+          return false;
+        }
+        return (childIdx >= 0 && childIdx < children.length) ?
+          children[childIdx].clearPath(f, dirNames, idx+1) : false;
+      }
+
+      //guesses failed. back to blind iteration.
+      if (children != null) {
+        for(int i=0; i < children.length; i++) {
+          if (children[i].clearPath(f, null, -1)){
+            return true;
+          }
+        }
+      }
+      return false;
+    }
+        
+    public String toString() {
+      return "FSDir{" +
+        "dir=" + dir +
+        ", children=" + (children == null ? null : Arrays.asList(children)) +
+        "}";
+    }
+  }
+
+  class FSVolume {
+    private FSDir dataDir;
+    private File tmpDir;
+    private File detachDir; // copy on write for blocks in snapshot
+    private DF usage;
+    private DU dfsUsage;
+    private long reserved;
+
+    
+    FSVolume(File currentDir, Configuration conf) throws IOException {
+      this.reserved = conf.getLong("dfs.datanode.du.reserved", 0);
+      boolean supportAppends = conf.getBoolean("dfs.support.append", false);
+      File parent = currentDir.getParentFile();
+
+      this.detachDir = new File(parent, "detach");
+      if (detachDir.exists()) {
+        recoverDetachedBlocks(currentDir, detachDir);
+      }
+
+      // Files that were being written when the datanode was last shutdown
+      // are now moved back to the data directory. It is possible that
+      // in the future, we might want to do some sort of datanode-local
+      // recovery for these blocks. For example, crc validation.
+      //
+      this.tmpDir = new File(parent, "tmp");
+      if (tmpDir.exists()) {
+        if (supportAppends) {
+          recoverDetachedBlocks(currentDir, tmpDir);
+        } else {
+          FileUtil.fullyDelete(tmpDir);
+        }
+      }
+      this.dataDir = new FSDir(currentDir);
+      if (!tmpDir.mkdirs()) {
+        if (!tmpDir.isDirectory()) {
+          throw new IOException("Mkdirs failed to create " + tmpDir.toString());
+        }
+      }
+      if (!detachDir.mkdirs()) {
+        if (!detachDir.isDirectory()) {
+          throw new IOException("Mkdirs failed to create " + detachDir.toString());
+        }
+      }
+      this.usage = new DF(parent, conf);
+      this.dfsUsage = new DU(parent, conf);
+      this.dfsUsage.start();
+    }
+
+    void decDfsUsed(long value) {
+      dfsUsage.decDfsUsed(value);
+    }
+    
+    long getDfsUsed() throws IOException {
+      return dfsUsage.getUsed();
+    }
+    
+    long getCapacity() throws IOException {
+      if (reserved > usage.getCapacity()) {
+        return 0;
+      }
+
+      return usage.getCapacity()-reserved;
+    }
+      
+    long getAvailable() throws IOException {
+      long remaining = getCapacity()-getDfsUsed();
+      long available = usage.getAvailable();
+      if (remaining>available) {
+        remaining = available;
+      }
+      return (remaining > 0) ? remaining : 0;
+    }
+      
+    String getMount() throws IOException {
+      return usage.getMount();
+    }
+      
+    File getDir() {
+      return dataDir.dir;
+    }
+    
+    /**
+     * Temporary files. They get moved to the real block directory either when
+     * the block is finalized or the datanode restarts.
+     */
+    File createTmpFile(Block b) throws IOException {
+      File f = new File(tmpDir, b.getBlockName());
+      return createTmpFile(b, f);
+    }
+
+    /**
+     * Returns the name of the temporary file for this block.
+     */
+    File getTmpFile(Block b) throws IOException {
+      File f = new File(tmpDir, b.getBlockName());
+      return f;
+    }
+
+    /**
+     * Files used for copy-on-write. They need recovery when datanode
+     * restarts.
+     */
+    File createDetachFile(Block b, String filename) throws IOException {
+      File f = new File(detachDir, filename);
+      return createTmpFile(b, f);
+    }
+
+    private File createTmpFile(Block b, File f) throws IOException {
+      if (f.exists()) {
+        throw new IOException("Unexpected problem in creating temporary file for "+
+                              b + ".  File " + f + " should not be present, but is.");
+      }
+      // Create the zero-length temp file
+      //
+      boolean fileCreated = false;
+      try {
+        fileCreated = f.createNewFile();
+      } catch (IOException ioe) {
+        throw (IOException)new IOException(DISK_ERROR +f).initCause(ioe);
+      }
+      if (!fileCreated) {
+        throw new IOException("Unexpected problem in creating temporary file for "+
+                              b + ".  File " + f + " should be creatable, but is already present.");
+      }
+      return f;
+    }
+      
+    File addBlock(Block b, File f) throws IOException {
+      File blockFile = dataDir.addBlock(b, f);
+      File metaFile = getMetaFile( blockFile , b);
+      dfsUsage.incDfsUsed(b.getNumBytes()+metaFile.length());
+      return blockFile;
+    }
+      
+    void checkDirs() throws DiskErrorException {
+      dataDir.checkDirTree();
+      DiskChecker.checkDir(tmpDir);
+    }
+      
+    void getVolumeMap(HashMap<Block, DatanodeBlockInfo> volumeMap) {
+      dataDir.getVolumeMap(volumeMap, this);
+    }
+      
+    void clearPath(File f) {
+      dataDir.clearPath(f);
+    }
+      
+    public String toString() {
+      return dataDir.dir.getAbsolutePath();
+    }
+
+    /**
+     * Recover detached files on datanode restart. If a detached block
+     * does not exist in the original directory, then it is moved to the
+     * original directory.
+     */
+    private void recoverDetachedBlocks(File dataDir, File dir) 
+                                           throws IOException {
+      File contents[] = dir.listFiles();
+      if (contents == null) {
+        return;
+      }
+      for (int i = 0; i < contents.length; i++) {
+        if (!contents[i].isFile()) {
+          throw new IOException ("Found " + contents[i] + " in " + dir +
+                                 " but it is not a file.");
+        }
+
+        //
+        // If the original block file still exists, then no recovery
+        // is needed.
+        //
+        File blk = new File(dataDir, contents[i].getName());
+        if (!blk.exists()) {
+          if (!contents[i].renameTo(blk)) {
+            throw new IOException("Unable to recover detached file " +
+                                  contents[i]);
+          }
+          continue;
+        }
+        if (!contents[i].delete()) {
+            throw new IOException("Unable to cleanup detached file " +
+                                  contents[i]);
+        }
+      }
+    }
+  }
+    
+  static class FSVolumeSet {
+    FSVolume[] volumes = null;
+    int curVolume = 0;
+      
+    FSVolumeSet(FSVolume[] volumes) {
+      this.volumes = volumes;
+    }
+      
+    synchronized FSVolume getNextVolume(long blockSize) throws IOException {
+      int startVolume = curVolume;
+      while (true) {
+        FSVolume volume = volumes[curVolume];
+        curVolume = (curVolume + 1) % volumes.length;
+        if (volume.getAvailable() > blockSize) { return volume; }
+        if (curVolume == startVolume) {
+          throw new DiskOutOfSpaceException("Insufficient space for an additional block");
+        }
+      }
+    }
+      
+    long getDfsUsed() throws IOException {
+      long dfsUsed = 0L;
+      for (int idx = 0; idx < volumes.length; idx++) {
+        dfsUsed += volumes[idx].getDfsUsed();
+      }
+      return dfsUsed;
+    }
+
+    long getCapacity() throws IOException {
+      long capacity = 0L;
+      for (int idx = 0; idx < volumes.length; idx++) {
+        capacity += volumes[idx].getCapacity();
+      }
+      return capacity;
+    }
+      
+    long getRemaining() throws IOException {
+      long remaining = 0L;
+      for (int idx = 0; idx < volumes.length; idx++) {
+        remaining += volumes[idx].getAvailable();
+      }
+      return remaining;
+    }
+      
+    synchronized void getVolumeMap(HashMap<Block, DatanodeBlockInfo> volumeMap) {
+      for (int idx = 0; idx < volumes.length; idx++) {
+        volumes[idx].getVolumeMap(volumeMap);
+      }
+    }
+      
+    synchronized void checkDirs() throws DiskErrorException {
+      for (int idx = 0; idx < volumes.length; idx++) {
+        volumes[idx].checkDirs();
+      }
+    }
+      
+    public String toString() {
+      StringBuffer sb = new StringBuffer();
+      for (int idx = 0; idx < volumes.length; idx++) {
+        sb.append(volumes[idx].toString());
+        if (idx != volumes.length - 1) { sb.append(","); }
+      }
+      return sb.toString();
+    }
+
+    public boolean isValid(FSVolume volume) {
+      for (int idx = 0; idx < volumes.length; idx++) {
+        if (volumes[idx] == volume) {
+          return true;
+        }
+      }
+      return false;
+    }
+  }
+  
+  //////////////////////////////////////////////////////
+  //
+  // FSDataSet
+  //
+  //////////////////////////////////////////////////////
+
+  //Find better place?
+  public static final String METADATA_EXTENSION = ".meta";
+  public static final short METADATA_VERSION = 1;
+    
+
+  static class ActiveFile {
+    final File file;
+    final List<Thread> threads = new ArrayList<Thread>(2);
+
+    ActiveFile(File f, List<Thread> list) {
+      file = f;
+      if (list != null) {
+        threads.addAll(list);
+      }
+      threads.add(Thread.currentThread());
+    }
+    
+    public String toString() {
+      return getClass().getSimpleName() + "(file=" + file
+          + ", threads=" + threads + ")";
+    }
+  } 
+  
+  static String getMetaFileName(String blockFileName, long genStamp) {
+    return blockFileName + "_" + genStamp + METADATA_EXTENSION;
+  }
+  
+  static File getMetaFile(File f , Block b) {
+    return new File(getMetaFileName(f.getAbsolutePath(),
+                                    b.getGenerationStamp())); 
+  }
+  protected File getMetaFile(Block b) throws IOException {
+    return getMetaFile(getBlockFile(b), b);
+  }
+
+  /** Find the corresponding meta data file from a given block file */
+  private static File findMetaFile(final File blockFile) throws IOException {
+    final String prefix = blockFile.getName() + "_";
+    final File parent = blockFile.getParentFile();
+    File[] matches = parent.listFiles(new FilenameFilter() {
+      public boolean accept(File dir, String name) {
+        return dir.equals(parent)
+            && name.startsWith(prefix) && name.endsWith(METADATA_EXTENSION);
+      }
+    });
+
+    if (matches == null || matches.length == 0) {
+      throw new IOException("Meta file not found, blockFile=" + blockFile);
+    }
+    else if (matches.length > 1) {
+      throw new IOException("Found more than one meta files: " 
+          + Arrays.asList(matches));
+    }
+    return matches[0];
+  }
+  
+  /** Find the corresponding meta data file from a given block file */
+  private static long parseGenerationStamp(File blockFile, File metaFile
+      ) throws IOException {
+    String metaname = metaFile.getName();
+    String gs = metaname.substring(blockFile.getName().length() + 1,
+        metaname.length() - METADATA_EXTENSION.length());
+    try {
+      return Long.parseLong(gs);
+    } catch(NumberFormatException nfe) {
+      throw (IOException)new IOException("blockFile=" + blockFile
+          + ", metaFile=" + metaFile).initCause(nfe);
+    }
+  }
+
+  /** Return the block file for the given ID */ 
+  public File findBlockFile(long blockId) {
+    final Block b = new Block(blockId);
+    File blockfile = null;
+    ActiveFile activefile = ongoingCreates.get(b);
+    if (activefile != null) {
+      blockfile = activefile.file;
+    }
+    if (blockfile == null) {
+      blockfile = getFile(b);
+    }
+    if (blockfile == null) {
+      if (DataNode.LOG.isDebugEnabled()) {
+        DataNode.LOG.debug("ongoingCreates=" + ongoingCreates);
+        DataNode.LOG.debug("volumeMap=" + volumeMap);
+      }
+    }
+    return blockfile;
+  }
+
+  /** {@inheritDoc} */
+  public synchronized Block getStoredBlock(long blkid) throws IOException {
+    File blockfile = findBlockFile(blkid);
+    if (blockfile == null) {
+      return null;
+    }
+    File metafile = findMetaFile(blockfile);
+    return new Block(blkid, blockfile.length(),
+        parseGenerationStamp(blockfile, metafile));
+  }
+
+  public boolean metaFileExists(Block b) throws IOException {
+    return getMetaFile(b).exists();
+  }
+  
+  public long getMetaDataLength(Block b) throws IOException {
+    File checksumFile = getMetaFile( b );
+    return checksumFile.length();
+  }
+
+  public MetaDataInputStream getMetaDataInputStream(Block b)
+      throws IOException {
+    File checksumFile = getMetaFile( b );
+    return new MetaDataInputStream(new FileInputStream(checksumFile),
+                                                    checksumFile.length());
+  }
+
+  FSVolumeSet volumes;
+  private HashMap<Block,ActiveFile> ongoingCreates = new HashMap<Block,ActiveFile>();
+  private int maxBlocksPerDir = 0;
+  HashMap<Block,DatanodeBlockInfo> volumeMap = null;
+  static  Random random = new Random();
+
+  // Used for synchronizing access to usage stats
+  private Object statsLock = new Object();
+
+  /**
+   * An FSDataset has a directory where it loads its data files.
+   */
+  public FSDataset(DataStorage storage, Configuration conf) throws IOException {
+    this.maxBlocksPerDir = conf.getInt("dfs.datanode.numblocks", 64);
+    FSVolume[] volArray = new FSVolume[storage.getNumStorageDirs()];
+    for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
+      volArray[idx] = new FSVolume(storage.getStorageDir(idx).getCurrentDir(), conf);
+    }
+    volumes = new FSVolumeSet(volArray);
+    volumeMap = new HashMap<Block, DatanodeBlockInfo>();
+    volumes.getVolumeMap(volumeMap);
+    registerMBean(storage.getStorageID());
+  }
+
+  /**
+   * Return the total space used by dfs datanode
+   */
+  public long getDfsUsed() throws IOException {
+    synchronized(statsLock) {
+      return volumes.getDfsUsed();
+    }
+  }
+  
+  /**
+   * Return total capacity, used and unused
+   */
+  public long getCapacity() throws IOException {
+    synchronized(statsLock) {
+      return volumes.getCapacity();
+    }
+  }
+
+  /**
+   * Return how many bytes can still be stored in the FSDataset
+   */
+  public long getRemaining() throws IOException {
+    synchronized(statsLock) {
+      return volumes.getRemaining();
+    }
+  }
+
+  /**
+   * Find the block's on-disk length
+   */
+  public long getLength(Block b) throws IOException {
+    return getBlockFile(b).length();
+  }
+
+  /**
+   * Get File name for a given block.
+   */
+  public synchronized File getBlockFile(Block b) throws IOException {
+    File f = validateBlockFile(b);
+    if(f == null) {
+      if (InterDatanodeProtocol.LOG.isDebugEnabled()) {
+        InterDatanodeProtocol.LOG.debug("b=" + b + ", volumeMap=" + volumeMap);
+      }
+      throw new IOException("Block " + b + " is not valid.");
+    }
+    return f;
+  }
+  
+  public synchronized InputStream getBlockInputStream(Block b) throws IOException {
+    return new FileInputStream(getBlockFile(b));
+  }
+
+  public synchronized InputStream getBlockInputStream(Block b, long seekOffset) throws IOException {
+
+    File blockFile = getBlockFile(b);
+    RandomAccessFile blockInFile = new RandomAccessFile(blockFile, "r");
+    if (seekOffset > 0) {
+      blockInFile.seek(seekOffset);
+    }
+    return new FileInputStream(blockInFile.getFD());
+  }
+
+  /**
+   * Returns handles to the block file and its metadata file
+   */
+  public synchronized BlockInputStreams getTmpInputStreams(Block b, 
+                          long blkOffset, long ckoff) throws IOException {
+
+    DatanodeBlockInfo info = volumeMap.get(b);
+    if (info == null) {
+      throw new IOException("Block " + b + " does not exist in volumeMap.");
+    }
+    FSVolume v = info.getVolume();
+    File blockFile = v.getTmpFile(b);
+    RandomAccessFile blockInFile = new RandomAccessFile(blockFile, "r");
+    if (blkOffset > 0) {
+      blockInFile.seek(blkOffset);
+    }
+    File metaFile = getMetaFile(blockFile, b);
+    RandomAccessFile metaInFile = new RandomAccessFile(metaFile, "r");
+    if (ckoff > 0) {
+      metaInFile.seek(ckoff);
+    }
+    return new BlockInputStreams(new FileInputStream(blockInFile.getFD()),
+                                new FileInputStream(metaInFile.getFD()));
+  }
+    
+  private BlockWriteStreams createBlockWriteStreams( File f , File metafile) throws IOException {
+      return new BlockWriteStreams(new FileOutputStream(new RandomAccessFile( f , "rw" ).getFD()),
+          new FileOutputStream( new RandomAccessFile( metafile , "rw" ).getFD() ));
+
+  }
+
+  /**
+   * Make a copy of the block if this block is linked to an existing
+   * snapshot. This ensures that modifying this block does not modify
+   * data in any existing snapshots.
+   * @param block Block
+   * @param numLinks Detach if the number of links exceed this value
+   * @throws IOException
+   * @return - true if the specified block was detached
+   */
+  public boolean detachBlock(Block block, int numLinks) throws IOException {
+    DatanodeBlockInfo info = null;
+
+    synchronized (this) {
+      info = volumeMap.get(block);
+    }
+    return info.detachBlock(block, numLinks);
+  }
+
+  static private <T> void updateBlockMap(Map<Block, T> blockmap,
+      Block oldblock, Block newblock) throws IOException {
+    if (blockmap.containsKey(oldblock)) {
+      T value = blockmap.remove(oldblock);
+      blockmap.put(newblock, value);
+    }
+  }
+
+  /** {@inheritDoc} */
+  public void updateBlock(Block oldblock, Block newblock) throws IOException {
+    if (oldblock.getBlockId() != newblock.getBlockId()) {
+      throw new IOException("Cannot update oldblock (=" + oldblock
+          + ") to newblock (=" + newblock + ").");
+    }
+    
+    for(;;) {
+      final List<Thread> threads = tryUpdateBlock(oldblock, newblock);
+      if (threads == null) {
+        return;
+      }
+
+      // interrupt and wait for all ongoing create threads
+      for(Thread t : threads) {
+        t.interrupt();
+      }
+      for(Thread t : threads) {
+        try {
+          t.join();
+        } catch (InterruptedException e) {
+          DataNode.LOG.warn("interruptOngoingCreates: t=" + t, e);
+        }
+      }
+    }
+  }
+
+  /**
+   * Try to update an old block to a new block.
+   * If there are ongoing create threads running for the old block,
+   * the threads will be returned without updating the block. 
+   * 
+   * @return ongoing create threads if there is any. Otherwise, return null.
+   */
+  private synchronized List<Thread> tryUpdateBlock(
+      Block oldblock, Block newblock) throws IOException {
+    //check ongoing create threads
+    final ActiveFile activefile = ongoingCreates.get(oldblock);
+    if (activefile != null && !activefile.threads.isEmpty()) {
+      //remove dead threads
+      for(Iterator<Thread> i = activefile.threads.iterator(); i.hasNext(); ) {
+        final Thread t = i.next();
+        if (!t.isAlive()) {
+          i.remove();
+        }
+      }
+
+      //return living threads
+      if (!activefile.threads.isEmpty()) {
+        return new ArrayList<Thread>(activefile.threads);
+      }
+    }
+
+    //No ongoing create threads is alive.  Update block.
+    File blockFile = findBlockFile(oldblock.getBlockId());
+    if (blockFile == null) {
+      throw new IOException("Block " + oldblock + " does not exist.");
+    }
+
+    File oldMetaFile = findMetaFile(blockFile);
+    long oldgs = parseGenerationStamp(blockFile, oldMetaFile);
+    
+    //rename meta file to a tmp file
+    File tmpMetaFile = new File(oldMetaFile.getParent(),
+        oldMetaFile.getName()+"_tmp" + newblock.getGenerationStamp());
+    if (!oldMetaFile.renameTo(tmpMetaFile)){
+      throw new IOException("Cannot rename block meta file to " + tmpMetaFile);
+    }
+
+    //update generation stamp
+    if (oldgs > newblock.getGenerationStamp()) {
+      throw new IOException("Cannot update block (id=" + newblock.getBlockId()
+          + ") generation stamp from " + oldgs
+          + " to " + newblock.getGenerationStamp());
+    }
+    
+    //update length
+    if (newblock.getNumBytes() > oldblock.getNumBytes()) {
+      throw new IOException("Cannot update block file (=" + blockFile
+          + ") length from " + oldblock.getNumBytes() + " to " + newblock.getNumBytes());
+    }
+    if (newblock.getNumBytes() < oldblock.getNumBytes()) {
+      truncateBlock(blockFile, tmpMetaFile, oldblock.getNumBytes(), newblock.getNumBytes());
+    }
+
+    //rename the tmp file to the new meta file (with new generation stamp)
+    File newMetaFile = getMetaFile(blockFile, newblock);
+    if (!tmpMetaFile.renameTo(newMetaFile)) {
+      throw new IOException("Cannot rename tmp meta file to " + newMetaFile);
+    }
+
+    updateBlockMap(ongoingCreates, oldblock, newblock);
+    updateBlockMap(volumeMap, oldblock, newblock);
+
+    // paranoia! verify that the contents of the stored block 
+    // matches the block file on disk.
+    validateBlockMetadata(newblock);
+    return null;
+  }
+
+  static private void truncateBlock(File blockFile, File metaFile,
+      long oldlen, long newlen) throws IOException {
+    if (newlen == oldlen) {
+      return;
+    }
+    if (newlen > oldlen) {
+      throw new IOException("Cannout truncate block to from oldlen (=" + oldlen
+          + ") to newlen (=" + newlen + ")");
+    }
+
+    DataChecksum dcs = BlockMetadataHeader.readHeader(metaFile).getChecksum(); 
+    int checksumsize = dcs.getChecksumSize();
+    int bpc = dcs.getBytesPerChecksum();
+    long n = (newlen - 1)/bpc + 1;
+    long newmetalen = BlockMetadataHeader.getHeaderSize() + n*checksumsize;
+    long lastchunkoffset = (n - 1)*bpc;
+    int lastchunksize = (int)(newlen - lastchunkoffset); 
+    byte[] b = new byte[Math.max(lastchunksize, checksumsize)]; 
+
+    RandomAccessFile blockRAF = new RandomAccessFile(blockFile, "rw");
+    try {
+      //truncate blockFile 
+      blockRAF.setLength(newlen);
+ 
+      //read last chunk
+      blockRAF.seek(lastchunkoffset);
+      blockRAF.readFully(b, 0, lastchunksize);
+    } finally {
+      blockRAF.close();
+    }
+
+    //compute checksum
+    dcs.update(b, 0, lastchunksize);
+    dcs.writeValue(b, 0, false);
+
+    //update metaFile 
+    RandomAccessFile metaRAF = new RandomAccessFile(metaFile, "rw");
+    try {
+      metaRAF.setLength(newmetalen);
+      metaRAF.seek(newmetalen - checksumsize);
+      metaRAF.write(b, 0, checksumsize);
+    } finally {
+      metaRAF.close();
+    }
+  }
+
+  private final static String DISK_ERROR = "Possible disk error on file creation: ";
+  /** Get the cause of an I/O exception if caused by a possible disk error
+   * @param ioe an I/O exception
+   * @return cause if the I/O exception is caused by a possible disk error;
+   *         null otherwise.
+   */ 
+  static IOException getCauseIfDiskError(IOException ioe) {
+    if (ioe.getMessage()!=null && ioe.getMessage().startsWith(DISK_ERROR)) {
+      return (IOException)ioe.getCause();
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Start writing to a block file
+   * If isRecovery is true and the block pre-exists, then we kill all
+      volumeMap.put(b, v);
+      volumeMap.put(b, v);
+   * other threads that might be writing to this block, and then reopen the file.
+   */
+  public BlockWriteStreams writeToBlock(Block b, boolean isRecovery) throws IOException {
+    //
+    // Make sure the block isn't a valid one - we're still creating it!
+    //
+    if (isValidBlock(b)) {
+      if (!isRecovery) {
+        throw new BlockAlreadyExistsException("Block " + b + " is valid, and cannot be written to.");
+      }
+      // If the block was successfully finalized because all packets
+      // were successfully processed at the Datanode but the ack for
+      // some of the packets were not received by the client. The client 
+      // re-opens the connection and retries sending those packets.
+      // The other reason is that an "append" is occurring to this block.
+      detachBlock(b, 1);
+    }
+    long blockSize = b.getNumBytes();
+
+    //
+    // Serialize access to /tmp, and check if file already there.
+    //
+    File f = null;
+    List<Thread> threads = null;
+    synchronized (this) {
+      //
+      // Is it already in the create process?
+      //
+      ActiveFile activeFile = ongoingCreates.get(b);
+      if (activeFile != null) {
+        f = activeFile.file;
+        threads = activeFile.threads;
+        
+        if (!isRecovery) {
+          throw new BlockAlreadyExistsException("Block " + b +
+                                  " has already been started (though not completed), and thus cannot be created.");
+        } else {
+          for (Thread thread:threads) {
+            thread.interrupt();
+          }
+        }
+        ongoingCreates.remove(b);
+      }
+      FSVolume v = null;
+      if (!isRecovery) {
+        v = volumes.getNextVolume(blockSize);
+        // create temporary file to hold block in the designated volume
+        f = createTmpFile(v, b);
+        volumeMap.put(b, new DatanodeBlockInfo(v));
+      } else if (f != null) {
+        DataNode.LOG.info("Reopen already-open Block for append " + b);
+        // create or reuse temporary file to hold block in the designated volume
+        v = volumeMap.get(b).getVolume();
+        volumeMap.put(b, new DatanodeBlockInfo(v));
+      } else {
+        // reopening block for appending to it.
+        DataNode.LOG.info("Reopen Block for append " + b);
+        v = volumeMap.get(b).getVolume();
+        f = createTmpFile(v, b);
+        File blkfile = getBlockFile(b);
+        File oldmeta = getMetaFile(b);
+        File newmeta = getMetaFile(f, b);
+
+        // rename meta file to tmp directory
+        DataNode.LOG.debug("Renaming " + oldmeta + " to " + newmeta);
+        if (!oldmeta.renameTo(newmeta)) {
+          throw new IOException("Block " + b + " reopen failed. " +
+                                " Unable to move meta file  " + oldmeta +
+                                " to tmp dir " + newmeta);
+        }
+
+        // rename block file to tmp directory
+        DataNode.LOG.debug("Renaming " + blkfile + " to " + f);
+        if (!blkfile.renameTo(f)) {
+          if (!f.delete()) {
+            throw new IOException("Block " + b + " reopen failed. " +
+                                  " Unable to remove file " + f);
+          }
+          if (!blkfile.renameTo(f)) {
+            throw new IOException("Block " + b + " reopen failed. " +
+                                  " Unable to move block file " + blkfile +
+                                  " to tmp dir " + f);
+          }
+        }
+        volumeMap.put(b, new DatanodeBlockInfo(v));
+      }
+      if (f == null) {
+        DataNode.LOG.warn("Block " + b + " reopen failed " +
+                          " Unable to locate tmp file.");
+        throw new IOException("Block " + b + " reopen failed " +
+                              " Unable to locate tmp file.");
+      }
+      ongoingCreates.put(b, new ActiveFile(f, threads));
+    }
+
+    try {
+      if (threads != null) {
+        for (Thread thread:threads) {
+          thread.join();
+        }
+      }
+    } catch (InterruptedException e) {
+      throw new IOException("Recovery waiting for thread interrupted.");
+    }
+
+    //
+    // Finally, allow a writer to the block file
+    // REMIND - mjc - make this a filter stream that enforces a max
+    // block size, so clients can't go crazy
+    //
+    File metafile = getMetaFile(f, b);
+    DataNode.LOG.debug("writeTo blockfile is " + f + " of size " + f.length());
+    DataNode.LOG.debug("writeTo metafile is " + metafile + " of size " + metafile.length());
+    return createBlockWriteStreams( f , metafile);
+  }
+
+  /**
+   * Retrieves the offset in the block to which the
+   * the next write will write data to.
+   */
+  public long getChannelPosition(Block b, BlockWriteStreams streams) 
+                                 throws IOException {
+    FileOutputStream file = (FileOutputStream) streams.dataOut;
+    return file.getChannel().position();
+  }
+
+  /**
+   * Sets the offset in the block to which the
+   * the next write will write data to.
+   */
+  public void setChannelPosition(Block b, BlockWriteStreams streams, 
+                                 long dataOffset, long ckOffset) 
+                                 throws IOException {
+    long size = 0;
+    synchronized (this) {
+      FSVolume vol = volumeMap.get(b).getVolume();
+      size = vol.getTmpFile(b).length();
+    }
+    if (size < dataOffset) {
+      String msg = "Trying to change block file offset of block " + b +
+                     " to " + dataOffset +
+                     " but actual size of file is " +
+                     size;
+      throw new IOException(msg);
+    }
+    FileOutputStream file = (FileOutputStream) streams.dataOut;
+    file.getChannel().position(dataOffset);
+    file = (FileOutputStream) streams.checksumOut;
+    file.getChannel().position(ckOffset);
+  }
+
+  synchronized File createTmpFile( FSVolume vol, Block blk ) throws IOException {
+    if ( vol == null ) {
+      vol = volumeMap.get( blk ).getVolume();
+      if ( vol == null ) {
+        throw new IOException("Could not find volume for block " + blk);
+      }
+    }
+    return vol.createTmpFile(blk);
+  }
+
+  //
+  // REMIND - mjc - eventually we should have a timeout system
+  // in place to clean up block files left by abandoned clients.
+  // We should have some timer in place, so that if a blockfile
+  // is created but non-valid, and has been idle for >48 hours,
+  // we can GC it safely.
+  //
+
+  /**
+   * Complete the block write!
+   */
+  public synchronized void finalizeBlock(Block b) throws IOException {
+    ActiveFile activeFile = ongoingCreates.get(b);
+    if (activeFile == null) {
+      throw new IOException("Block " + b + " is already finalized.");
+    }
+    File f = activeFile.file;
+    if (f == null || !f.exists()) {
+      throw new IOException("No temporary file " + f + " for block " + b);
+    }
+    FSVolume v = volumeMap.get(b).getVolume();
+    if (v == null) {
+      throw new IOException("No volume for temporary file " + f + 
+                            " for block " + b);
+    }
+        
+    File dest = null;
+    dest = v.addBlock(b, f);
+    volumeMap.put(b, new DatanodeBlockInfo(v, dest));
+    ongoingCreates.remove(b);
+  }
+
+  /**
+   * Remove the temporary block file (if any)
+   */
+  public synchronized void unfinalizeBlock(Block b) throws IOException {
+    // remove the block from in-memory data structure
+    ActiveFile activefile = ongoingCreates.remove(b);
+    if (activefile == null) {
+      return;
+    }
+    volumeMap.remove(b);
+    
+    // delete the on-disk temp file
+    if (delBlockFromDisk(activefile.file, getMetaFile(activefile.file, b), b)) {
+      DataNode.LOG.warn("Block " + b + " unfinalized and removed. " );
+    }
+  }
+
+  /**
+   * Remove a block from disk
+   * @param blockFile block file
+   * @param metaFile block meta file
+   * @param b a block
+   * @return true if on-disk files are deleted; false otherwise
+   */
+  private boolean delBlockFromDisk(File blockFile, File metaFile, Block b) {
+    if (blockFile == null) {
+      DataNode.LOG.warn("No file exists for block: " + b);
+      return true;
+    }
+    
+    if (!blockFile.delete()) {
+      DataNode.LOG.warn("Not able to delete the block file: " + blockFile);
+      return false;
+    } else { // remove the meta file
+      if (metaFile != null && !metaFile.delete()) {
+        DataNode.LOG.warn(
+            "Not able to delete the meta block file: " + metaFile);
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Return finalized blocks from the in-memory blockmap
+   */
+  public Block[] getBlockReport() {
+    ArrayList<Block> list =  new ArrayList<Block>(volumeMap.size());
+    synchronized(this) {
+      for (Block b : volumeMap.keySet()) {
+        if (!ongoingCreates.containsKey(b)) {
+          list.add(new Block(b));
+        }
+      }
+    }
+    return list.toArray(new Block[list.size()]);
+  }
+
+  /**
+   * Get the block list from in-memory blockmap. Note if <deepcopy>
+   * is false, reference to the block in the volumeMap is returned. This block
+   * should not be changed. Suitable synchronization using {@link FSDataset}
+   * is needed to handle concurrent modification to the block.
+   */
+  synchronized Block[] getBlockList(boolean deepcopy) {
+    Block[] list = volumeMap.keySet().toArray(new Block[volumeMap.size()]);
+    if (deepcopy) {
+      for (int i = 0; i < list.length; i++) {
+        list[i] = new Block(list[i]);
+      }
+    }
+    return list;
+  }
+
+  /**
+   * Check whether the given block is a valid one.
+   */
+  public boolean isValidBlock(Block b) {
+    return validateBlockFile(b) != null;
+  }
+
+  /**
+   * Find the file corresponding to the block and return it if it exists.
+   */
+  File validateBlockFile(Block b) {
+    //Should we check for metadata file too?
+    File f = getFile(b);
+    if(f != null && f.exists())
+      return f;
+    if (InterDatanodeProtocol.LOG.isDebugEnabled()) {
+      InterDatanodeProtocol.LOG.debug("b=" + b + ", f=" + f);
+    }
+    return null;
+  }
+
+  /** {@inheritDoc} */
+  public void validateBlockMetadata(Block b) throws IOException {
+    DatanodeBlockInfo info = volumeMap.get(b);
+    if (info == null) {
+      throw new IOException("Block " + b + " does not exist in volumeMap.");
+    }
+    FSVolume v = info.getVolume();
+    File tmp = v.getTmpFile(b);
+    File f = getFile(b);
+    if (f == null) {
+      f = tmp;
+    }
+    if (f == null) {
+      throw new IOException("Block " + b + " does not exist on disk.");
+    }
+    if (!f.exists()) {
+      throw new IOException("Block " + b + 
+                            " block file " + f +
+                            " does not exist on disk.");
+    }
+    if (b.getNumBytes() != f.length()) {
+      throw new IOException("Block " + b + 
+                            " length is " + b.getNumBytes()  +
+                            " does not match block file length " +
+                            f.length());
+    }
+    File meta = getMetaFile(f, b);
+    if (meta == null) {
+      throw new IOException("Block " + b + 
+                            " metafile does not exist.");
+    }
+    if (!meta.exists()) {
+      throw new IOException("Block " + b + 
+                            " metafile " + meta +
+                            " does not exist on disk.");
+    }
+    if (meta.length() == 0) {
+      throw new IOException("Block " + b + " metafile " + meta + " is empty.");
+    }
+    long stamp = parseGenerationStamp(f, meta);
+    if (stamp != b.getGenerationStamp()) {
+      throw new IOException("Block " + b + 
+                            " genstamp is " + b.getGenerationStamp()  +
+                            " does not match meta file stamp " +
+                            stamp);
+    }
+  }
+
+  /**
+   * We're informed that a block is no longer valid.  We
+   * could lazily garbage-collect the block, but why bother?
+   * just get rid of it.
+   */
+  public void invalidate(Block invalidBlks[]) throws IOException {
+    boolean error = false;
+    for (int i = 0; i < invalidBlks.length; i++) {
+      File f = null;
+      FSVolume v;
+      synchronized (this) {
+        f = getFile(invalidBlks[i]);
+        DatanodeBlockInfo dinfo = volumeMap.get(invalidBlks[i]);
+        if (dinfo == null) {
+          DataNode.LOG.warn("Unexpected error trying to delete block "
+                           + invalidBlks[i] + 
+                           ". BlockInfo not found in volumeMap.");
+          error = true;
+          continue;
+        }
+        v = dinfo.getVolume();
+        if (f == null) {
+          DataNode.LOG.warn("Unexpected error trying to delete block "
+                            + invalidBlks[i] + 
+                            ". Block not found in blockMap." +
+                            ((v == null) ? " " : " Block found in volumeMap."));
+          error = true;
+          continue;
+        }
+        if (v == null) {
+          DataNode.LOG.warn("Unexpected error trying to delete block "
+                            + invalidBlks[i] + 
+                            ". No volume for this block." +
+                            " Block found in blockMap. " + f + ".");
+          error = true;
+          continue;
+        }
+        File parent = f.getParentFile();
+        if (parent == null) {
+          DataNode.LOG.warn("Unexpected error trying to delete block "
+                            + invalidBlks[i] + 
+                            ". Parent not found for file " + f + ".");
+          error = true;
+          continue;
+        }
+        v.clearPath(parent);
+        volumeMap.remove(invalidBlks[i]);
+      }
+      File metaFile = getMetaFile( f, invalidBlks[i] );
+      long blockSize = f.length()+metaFile.length();
+      if ( !f.delete() || ( !metaFile.delete() && metaFile.exists() ) ) {
+        DataNode.LOG.warn("Unexpected error trying to delete block "
+                          + invalidBlks[i] + " at file " + f);
+        error = true;
+        continue;
+      }
+      v.decDfsUsed(blockSize);
+      DataNode.LOG.info("Deleting block " + invalidBlks[i] + " file " + f);
+      if (f.exists()) {
+        //
+        // This is a temporary check especially for hadoop-1220. 
+        // This will go away in the future.
+        //
+        DataNode.LOG.info("File " + f + " was deleted but still exists!");
+      }
+    }
+    if (error) {
+      throw new IOException("Error in deleting blocks.");
+    }
+  }
+
+  /**
+   * Turn the block identifier into a filename.
+   */
+  public synchronized File getFile(Block b) {
+    DatanodeBlockInfo info = volumeMap.get(b);
+    if (info != null) {
+      return info.getFile();
+    }
+    return null;
+  }
+
+  /**
+   * check if a data directory is healthy
+   * @throws DiskErrorException
+   */
+  public void checkDataDir() throws DiskErrorException {
+    volumes.checkDirs();
+  }
+    
+
+  public String toString() {
+    return "FSDataset{dirpath='"+volumes+"'}";
+  }
+
+  private ObjectName mbeanName;
+  private Random rand = new Random();
+  
+  /**
+   * Register the FSDataset MBean using the name
+   *        "hadoop:service=DataNode,name=FSDatasetState-<storageid>"
+   */
+  void registerMBean(final String storageId) {
+    // We wrap to bypass standard mbean naming convetion.
+    // This wraping can be removed in java 6 as it is more flexible in 
+    // package naming for mbeans and their impl.
+    StandardMBean bean;
+    String storageName;
+    if (storageId == null || storageId.equals("")) {// Temp fix for the uninitialized storage
+      storageName = "UndefinedStorageId" + rand.nextInt();
+    } else {
+      storageName = storageId;
+    }
+    try {
+      bean = new StandardMBean(this,FSDatasetMBean.class);
+      mbeanName = MBeanUtil.registerMBean("DataNode", "FSDatasetState-" + storageName, bean);
+    } catch (NotCompliantMBeanException e) {
+      e.printStackTrace();
+    }
+ 
+    DataNode.LOG.info("Registered FSDatasetStatusMBean");
+  }
+
+  public void shutdown() {
+    if (mbeanName != null)
+      MBeanUtil.unregisterMBean(mbeanName);
+    
+    if(volumes != null) {
+      for (FSVolume volume : volumes.volumes) {
+        if(volume != null) {
+          volume.dfsUsage.shutdown();
+        }
+      }
+    }
+  }
+
+  public String getStorageInfo() {
+    return toString();
+  }
+
+  /**
+   * Reconcile the difference between blocks on the disk and blocks in
+   * volumeMap
+   *
+   * Check the given block for inconsistencies. Look at the
+   * current state of the block and reconcile the differences as follows:
+   * <ul>
+   * <li>If the block file is missing, delete the block from volumeMap</li>
+   * <li>If the block file exists and the block is missing in volumeMap,
+   * add the block to volumeMap <li>
+   * <li>If generation stamp does not match, then update the block with right
+   * generation stamp</li>
+   * <li>If the block length in memory does not match the actual block file length
+   * then mark the block as corrupt and update the block length in memory</li>
+   * <li>If the file in {@link DatanodeBlockInfo} does not match the file on
+   * the disk, update {@link DatanodeBlockInfo} with the correct file</li>
+   * </ul>
+   *
+   * @param blockId Block that differs
+   * @param diskFile Block file on the disk
+   * @param diskMetaFile Metadata file from on the disk
+   * @param vol Volume of the block file
+   */
+  public void checkAndUpdate(long blockId, File diskFile,
+      File diskMetaFile, FSVolume vol) {
+    Block block = new Block(blockId);
+    DataNode datanode = DataNode.getDataNode();
+    Block corruptBlock = null;
+    synchronized (this) {
+      if (ongoingCreates.get(block) != null) {
+        // Block is not finalized - ignore the difference
+        return;
+      }
+
+      final long diskGS = diskMetaFile != null && diskMetaFile.exists() ?
+          Block.getGenerationStamp(diskMetaFile.getName()) :
+            Block.GRANDFATHER_GENERATION_STAMP;
+
+      DatanodeBlockInfo memBlockInfo = volumeMap.get(block);
+      if (diskFile == null || !diskFile.exists()) {
+        if (memBlockInfo == null) {
+          // Block file does not exist and block does not exist in memory
+          // If metadata file exists then delete it
+          if (diskMetaFile != null && diskMetaFile.exists()
+              && diskMetaFile.delete()) {
+            DataNode.LOG.warn("Deleted a metadata file without a block "
+                + diskMetaFile.getAbsolutePath());
+          }
+          return;
+        }
+        if (!memBlockInfo.getFile().exists()) {
+          // Block is in memory and not on the disk
+          // Remove the block from volumeMap
+          volumeMap.remove(block);
+          if (datanode.blockScanner != null) {
+            datanode.blockScanner.deleteBlock(block);
+          }
+          DataNode.LOG.warn("Removed block " + block.getBlockId()
+              + " from memory with missing block file on the disk");
+          // Finally remove the metadata file
+          if (diskMetaFile != null && diskMetaFile.exists()
+              && diskMetaFile.delete()) {
+            DataNode.LOG.warn("Deleted a metadata file for the deleted block "
+                + diskMetaFile.getAbsolutePath());
+          }
+        }
+        return;
+      }
+      /*
+       * Block file exists on the disk
+       */
+      if (memBlockInfo == null) {
+        // Block is missing in memory - add the block to volumeMap
+        DatanodeBlockInfo diskBlockInfo = new DatanodeBlockInfo(vol, diskFile);
+        Block diskBlock = new Block(diskFile, diskFile.length(), diskGS);
+        volumeMap.put(diskBlock, diskBlockInfo);
+        if (datanode.blockScanner != null) {
+          datanode.blockScanner.addBlock(diskBlock);
+        }
+        DataNode.LOG.warn("Added missing block to memory " + diskBlock);
+        return;
+      }
+      /*
+       * Block exists in volumeMap and the block file exists on the disk
+       */
+      // Iterate to get key from volumeMap for the blockId
+      Block memBlock = getBlockKey(blockId);
+
+      // Compare block files
+      File memFile = memBlockInfo.getFile();
+      if (memFile.exists()) {
+        if (memFile.compareTo(diskFile) != 0) {
+          DataNode.LOG.warn("Block file " + memFile.getAbsolutePath()
+              + " does not match file found by scan "
+              + diskFile.getAbsolutePath());
+          // TODO: Should the diskFile be deleted?
+        }
+      } else {
+        // Block refers to a block file that does not exist.
+        // Update the block with the file found on the disk. Since the block
+        // file and metadata file are found as a pair on the disk, update
+        // the block based on the metadata file found on the disk
+        DataNode.LOG.warn("Block file in volumeMap "
+            + memFile.getAbsolutePath()
+            + " does not exist. Updating it to the file found during scan "
+            + diskFile.getAbsolutePath());
+        DatanodeBlockInfo info = volumeMap.remove(memBlock);
+        info.setFile(diskFile);
+        memFile = diskFile;
+
+        DataNode.LOG.warn("Updating generation stamp for block " + blockId
+            + " from " + memBlock.getGenerationStamp() + " to " + diskGS);
+        memBlock.setGenerationStamp(diskGS);
+        volumeMap.put(memBlock, info);
+      }
+
+      // Compare generation stamp
+      if (memBlock.getGenerationStamp() != diskGS) {
+        File memMetaFile = getMetaFile(diskFile, memBlock);
+        if (memMetaFile.exists()) {
+          if (memMetaFile.compareTo(diskMetaFile) != 0) {
+            DataNode.LOG.warn("Metadata file in memory "
+                + memMetaFile.getAbsolutePath()
+                + " does not match file found by scan "
+                + diskMetaFile.getAbsolutePath());
+          }
+        } else {
+          // Metadata file corresponding to block in memory is missing
+          // If metadata file found during the scan is on the same directory
+          // as the block file, then use the generation stamp from it
+          long gs = diskMetaFile != null && diskMetaFile.exists()
+              && diskMetaFile.getParent().equals(memFile.getParent()) ? diskGS
+              : Block.GRANDFATHER_GENERATION_STAMP;
+
+          DataNode.LOG.warn("Updating generation stamp for block " + blockId
+              + " from " + memBlock.getGenerationStamp() + " to " + gs);
+
+          DatanodeBlockInfo info = volumeMap.remove(memBlock);
+          memBlock.setGenerationStamp(gs);
+          volumeMap.put(memBlock, info);
+        }
+      }
+
+      // Compare block size
+      if (memBlock.getNumBytes() != memFile.length()) {
+        // Update the length based on the block file
+        corruptBlock = new Block(memBlock);
+        DataNode.LOG.warn("Updating size of block " + blockId + " from "
+            + memBlock.getNumBytes() + " to " + memFile.length());
+        DatanodeBlockInfo info = volumeMap.remove(memBlock);
+        memBlock.setNumBytes(memFile.length());
+        volumeMap.put(memBlock, info);
+      }
+    }
+
+    // Send corrupt block report outside the lock
+    if (corruptBlock != null) {
+      DatanodeInfo[] dnArr = { new DatanodeInfo(datanode.dnRegistration) };
+      LocatedBlock[] blocks = { new LocatedBlock(corruptBlock, dnArr) };
+      try {
+        datanode.namenode.reportBadBlocks(blocks);
+        DataNode.LOG.warn("Reporting the block " + corruptBlock
+            + " as corrupt due to length mismatch");
+      } catch (IOException e) {
+        DataNode.LOG.warn("Failed to repot bad block " + corruptBlock
+            + "Exception:" + StringUtils.stringifyException(e));
+      }
+    }
+  }
+
+  /**
+   * Get reference to the key in the volumeMap. To be called from methods that
+   * are synchronized on {@link FSDataset}
+   * @param blockId
+   * @return key from the volumeMap
+   */
+  Block getBlockKey(long blockId) {
+    assert(Thread.holdsLock(this));
+    for (Block b : volumeMap.keySet()) {
+      if (b.getBlockId() == blockId) {
+        return b;
+      }
+    }
+    return null;
+  }
+}

+ 267 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java

@@ -0,0 +1,267 @@
+/**
+ * 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.datanode;
+
+
+import java.io.Closeable;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+
+
+
+import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+
+/**
+ * This is an interface for the underlying storage that stores blocks for
+ * a data node. 
+ * Examples are the FSDataset (which stores blocks on dirs)  and 
+ * SimulatedFSDataset (which simulates data).
+ *
+ */
+public interface FSDatasetInterface extends FSDatasetMBean {
+  
+  
+  /**
+   * Returns the length of the metadata file of the specified block
+   * @param b - the block for which the metadata length is desired
+   * @return the length of the metadata file for the specified block.
+   * @throws IOException
+   */
+  public long getMetaDataLength(Block b) throws IOException;
+  
+  /**
+   * This class provides the input stream and length of the metadata
+   * of a block
+   *
+   */
+  static class MetaDataInputStream extends FilterInputStream {
+    MetaDataInputStream(InputStream stream, long len) {
+      super(stream);
+      length = len;
+    }
+    private long length;
+    
+    public long getLength() {
+      return length;
+    }
+  }
+  
+  /**
+   * Returns metaData of block b as an input stream (and its length)
+   * @param b - the block
+   * @return the metadata input stream; 
+   * @throws IOException
+   */
+  public MetaDataInputStream getMetaDataInputStream(Block b)
+        throws IOException;
+  
+  /**
+   * Does the meta file exist for this block?
+   * @param b - the block
+   * @return true of the metafile for specified block exits
+   * @throws IOException
+   */
+  public boolean metaFileExists(Block b) throws IOException;
+
+
+  /**
+   * Returns the specified block's on-disk length (excluding metadata)
+   * @param b
+   * @return   the specified block's on-disk length (excluding metadta)
+   * @throws IOException
+   */
+  public long getLength(Block b) throws IOException;
+
+  /**
+   * @return the generation stamp stored with the block.
+   */
+  public Block getStoredBlock(long blkid) throws IOException;
+
+  /**
+   * Returns an input stream to read the contents of the specified block
+   * @param b
+   * @return an input stream to read the contents of the specified block
+   * @throws IOException
+   */
+  public InputStream getBlockInputStream(Block b) throws IOException;
+  
+  /**
+   * Returns an input stream at specified offset of the specified block
+   * @param b
+   * @param seekOffset
+   * @return an input stream to read the contents of the specified block,
+   *  starting at the offset
+   * @throws IOException
+   */
+  public InputStream getBlockInputStream(Block b, long seekOffset)
+            throws IOException;
+
+  /**
+   * Returns an input stream at specified offset of the specified block
+   * The block is still in the tmp directory and is not finalized
+   * @param b
+   * @param blkoff
+   * @param ckoff
+   * @return an input stream to read the contents of the specified block,
+   *  starting at the offset
+   * @throws IOException
+   */
+  public BlockInputStreams getTmpInputStreams(Block b, long blkoff, long ckoff)
+            throws IOException;
+
+     /**
+      * 
+      * This class contains the output streams for the data and checksum
+      * of a block
+      *
+      */
+     static class BlockWriteStreams {
+      OutputStream dataOut;
+      OutputStream checksumOut;
+      BlockWriteStreams(OutputStream dOut, OutputStream cOut) {
+        dataOut = dOut;
+        checksumOut = cOut;
+      }
+      
+    }
+
+  /**
+   * This class contains the input streams for the data and checksum
+   * of a block
+   */
+  static class BlockInputStreams implements Closeable {
+    final InputStream dataIn;
+    final InputStream checksumIn;
+
+    BlockInputStreams(InputStream dataIn, InputStream checksumIn) {
+      this.dataIn = dataIn;
+      this.checksumIn = checksumIn;
+    }
+
+    /** {@inheritDoc} */
+    public void close() {
+      IOUtils.closeStream(dataIn);
+      IOUtils.closeStream(checksumIn);
+    }
+  }
+    
+  /**
+   * Creates the block and returns output streams to write data and CRC
+   * @param b
+   * @param isRecovery True if this is part of erro recovery, otherwise false
+   * @return a BlockWriteStreams object to allow writing the block data
+   *  and CRC
+   * @throws IOException
+   */
+  public BlockWriteStreams writeToBlock(Block b, boolean isRecovery) throws IOException;
+
+  /**
+   * Update the block to the new generation stamp and length.  
+   */
+  public void updateBlock(Block oldblock, Block newblock) throws IOException;
+  
+  /**
+   * Finalizes the block previously opened for writing using writeToBlock.
+   * The block size is what is in the parameter b and it must match the amount
+   *  of data written
+   * @param b
+   * @throws IOException
+   */
+  public void finalizeBlock(Block b) throws IOException;
+
+  /**
+   * Unfinalizes the block previously opened for writing using writeToBlock.
+   * The temporary file associated with this block is deleted.
+   * @param b
+   * @throws IOException
+   */
+  public void unfinalizeBlock(Block b) throws IOException;
+
+  /**
+   * Returns the block report - the full list of blocks stored
+   * @return - the block report - the full list of blocks stored
+   */
+  public Block[] getBlockReport();
+
+  /**
+   * Is the block valid?
+   * @param b
+   * @return - true if the specified block is valid
+   */
+  public boolean isValidBlock(Block b);
+
+  /**
+   * Invalidates the specified blocks
+   * @param invalidBlks - the blocks to be invalidated
+   * @throws IOException
+   */
+  public void invalidate(Block invalidBlks[]) throws IOException;
+
+    /**
+     * Check if all the data directories are healthy
+     * @throws DiskErrorException
+     */
+  public void checkDataDir() throws DiskErrorException;
+      
+    /**
+     * Stringifies the name of the storage
+     */
+  public String toString();
+  
+  /**
+   * Shutdown the FSDataset
+   */
+  public void shutdown();
+
+  /**
+   * Returns the current offset in the data stream.
+   * @param b
+   * @param stream The stream to the data file and checksum file
+   * @return the position of the file pointer in the data stream
+   * @throws IOException
+   */
+  public long getChannelPosition(Block b, BlockWriteStreams stream) throws IOException;
+
+  /**
+   * Sets the file pointer of the data stream and checksum stream to
+   * the specified values.
+   * @param b
+   * @param stream The stream for the data file and checksum file
+   * @param dataOffset The position to which the file pointre for the data stream
+   *        should be set
+   * @param ckOffset The position to which the file pointre for the checksum stream
+   *        should be set
+   * @throws IOException
+   */
+  public void setChannelPosition(Block b, BlockWriteStreams stream, long dataOffset,
+                                 long ckOffset) throws IOException;
+
+  /**
+   * Validate that the contents in the Block matches
+   * the file on disk. Returns true if everything is fine.
+   * @param b The block to be verified.
+   * @throws IOException
+   */
+  public void validateBlockMetadata(Block b) throws IOException;
+}

+ 151 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java

@@ -0,0 +1,151 @@
+/**
+ * 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.datanode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.UpgradeManager;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.util.Daemon;
+
+/**
+ * Upgrade manager for data-nodes.
+ *
+ * Distributed upgrades for a data-node are performed in a separate thread.
+ * The upgrade starts when the data-node receives the start upgrade command
+ * from the namenode. At that point the manager finds a respective upgrade
+ * object and starts a daemon in order to perform the upgrade defined by the 
+ * object.
+ */
+class UpgradeManagerDatanode extends UpgradeManager {
+  DataNode dataNode = null;
+  Daemon upgradeDaemon = null;
+
+  UpgradeManagerDatanode(DataNode dataNode) {
+    super();
+    this.dataNode = dataNode;
+  }
+
+  public HdfsConstants.NodeType getType() {
+    return HdfsConstants.NodeType.DATA_NODE;
+  }
+
+  synchronized void initializeUpgrade(NamespaceInfo nsInfo) throws IOException {
+    if( ! super.initializeUpgrade())
+      return; // distr upgrade is not needed
+    DataNode.LOG.info("\n   Distributed upgrade for DataNode " 
+        + dataNode.dnRegistration.getName() 
+        + " version " + getUpgradeVersion() + " to current LV " 
+        + FSConstants.LAYOUT_VERSION + " is initialized.");
+    UpgradeObjectDatanode curUO = (UpgradeObjectDatanode)currentUpgrades.first();
+    curUO.setDatanode(dataNode);
+    upgradeState = curUO.preUpgradeAction(nsInfo);
+    // upgradeState is true if the data-node should start the upgrade itself
+  }
+
+  /**
+   * Start distributed upgrade.
+   * Instantiates distributed upgrade objects.
+   * 
+   * @return true if distributed upgrade is required or false otherwise
+   * @throws IOException
+   */
+  public synchronized boolean startUpgrade() throws IOException {
+    if(upgradeState) {  // upgrade is already in progress
+      assert currentUpgrades != null : 
+        "UpgradeManagerDatanode.currentUpgrades is null.";
+      UpgradeObjectDatanode curUO = (UpgradeObjectDatanode)currentUpgrades.first();
+      curUO.startUpgrade();
+      return true;
+    }
+    if(broadcastCommand != null) {
+      if(broadcastCommand.getVersion() > this.getUpgradeVersion()) {
+        // stop broadcasting, the cluster moved on
+        // start upgrade for the next version
+        broadcastCommand = null;
+      } else {
+        // the upgrade has been finished by this data-node,
+        // but the cluster is still running it, 
+        // reply with the broadcast command
+        assert currentUpgrades == null : 
+          "UpgradeManagerDatanode.currentUpgrades is not null.";
+        assert upgradeDaemon == null : 
+          "UpgradeManagerDatanode.upgradeDaemon is not null.";
+        dataNode.namenode.processUpgradeCommand(broadcastCommand);
+        return true;
+      }
+    }
+    if(currentUpgrades == null)
+      currentUpgrades = getDistributedUpgrades();
+    if(currentUpgrades == null) {
+      DataNode.LOG.info("\n   Distributed upgrade for DataNode version " 
+          + getUpgradeVersion() + " to current LV " 
+          + FSConstants.LAYOUT_VERSION + " cannot be started. "
+          + "The upgrade object is not defined.");
+      return false;
+    }
+    upgradeState = true;
+    UpgradeObjectDatanode curUO = (UpgradeObjectDatanode)currentUpgrades.first();
+    curUO.setDatanode(dataNode);
+    curUO.startUpgrade();
+    upgradeDaemon = new Daemon(curUO);
+    upgradeDaemon.start();
+    DataNode.LOG.info("\n   Distributed upgrade for DataNode " 
+        + dataNode.dnRegistration.getName() 
+        + " version " + getUpgradeVersion() + " to current LV " 
+        + FSConstants.LAYOUT_VERSION + " is started.");
+    return true;
+  }
+
+  synchronized void processUpgradeCommand(UpgradeCommand command
+                                          ) throws IOException {
+    assert command.getAction() == UpgradeCommand.UC_ACTION_START_UPGRADE :
+      "Only start upgrade action can be processed at this time.";
+    this.upgradeVersion = command.getVersion();
+    // Start distributed upgrade
+    if(startUpgrade()) // upgrade started
+      return;
+    throw new IOException(
+        "Distributed upgrade for DataNode " + dataNode.dnRegistration.getName() 
+        + " version " + getUpgradeVersion() + " to current LV " 
+        + FSConstants.LAYOUT_VERSION + " cannot be started. "
+        + "The upgrade object is not defined.");
+  }
+
+  public synchronized void completeUpgrade() throws IOException {
+    assert currentUpgrades != null : 
+      "UpgradeManagerDatanode.currentUpgrades is null.";
+    UpgradeObjectDatanode curUO = (UpgradeObjectDatanode)currentUpgrades.first();
+    broadcastCommand = curUO.completeUpgrade();
+    upgradeState = false;
+    currentUpgrades = null;
+    upgradeDaemon = null;
+    DataNode.LOG.info("\n   Distributed upgrade for DataNode " 
+        + dataNode.dnRegistration.getName() 
+        + " version " + getUpgradeVersion() + " to current LV " 
+        + FSConstants.LAYOUT_VERSION + " is complete.");
+  }
+
+  synchronized void shutdownUpgrade() {
+    if(upgradeDaemon != null)
+      upgradeDaemon.interrupt();
+  }
+}

+ 134 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java

@@ -0,0 +1,134 @@
+/**
+ * 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.datanode;
+
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.UpgradeObject;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.util.StringUtils;
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+
+/**
+ * Base class for data-node upgrade objects.
+ * Data-node upgrades are run in separate threads.
+ */
+public abstract class UpgradeObjectDatanode extends UpgradeObject implements Runnable {
+  private DataNode dataNode = null;
+
+  public HdfsConstants.NodeType getType() {
+    return HdfsConstants.NodeType.DATA_NODE;
+  }
+
+  protected DataNode getDatanode() {
+    return dataNode;
+  }
+
+  void setDatanode(DataNode dataNode) {
+    this.dataNode = dataNode;
+  }
+
+  /**
+   * Specifies how the upgrade is performed. 
+   * @throws IOException
+   */
+  public abstract void doUpgrade() throws IOException;
+
+  /**
+   * Specifies what to do before the upgrade is started.
+   * 
+   * The default implementation checks whether the data-node missed the upgrade
+   * and throws an exception if it did. This leads to the data-node shutdown.
+   * 
+   * Data-nodes usually start distributed upgrade when the name-node replies
+   * to its heartbeat with a start upgrade command.
+   * Sometimes though, e.g. when a data-node missed the upgrade and wants to
+   * catchup with the rest of the cluster, it is necessary to initiate the 
+   * upgrade directly on the data-node, since the name-node might not ever 
+   * start it. An override of this method should then return true.
+   * And the upgrade will start after data-ndoe registration but before sending
+   * its first heartbeat.
+   * 
+   * @param nsInfo name-node versions, verify that the upgrade
+   * object can talk to this name-node version if necessary.
+   * 
+   * @throws IOException
+   * @return true if data-node itself should start the upgrade or 
+   * false if it should wait until the name-node starts the upgrade.
+   */
+  boolean preUpgradeAction(NamespaceInfo nsInfo) throws IOException {
+    int nsUpgradeVersion = nsInfo.getDistributedUpgradeVersion();
+    if(nsUpgradeVersion >= getVersion())
+      return false; // name-node will perform the upgrade
+    // Missed the upgrade. Report problem to the name-node and throw exception
+    String errorMsg = 
+              "\n   Data-node missed a distributed upgrade and will shutdown."
+            + "\n   " + getDescription() + "."
+            + " Name-node version = " + nsInfo.getLayoutVersion() + ".";
+    DataNode.LOG.fatal( errorMsg );
+    try {
+      dataNode.namenode.errorReport(dataNode.dnRegistration,
+                                    DatanodeProtocol.NOTIFY, errorMsg);
+    } catch(SocketTimeoutException e) {  // namenode is busy
+      DataNode.LOG.info("Problem connecting to server: " 
+                        + dataNode.getNameNodeAddr());
+    }
+    throw new IOException(errorMsg);
+  }
+
+  public void run() {
+    assert dataNode != null : "UpgradeObjectDatanode.dataNode is null";
+    while(dataNode.shouldRun) {
+      try {
+        doUpgrade();
+      } catch(Exception e) {
+        DataNode.LOG.error(StringUtils.stringifyException(e));
+      }
+      break;
+    }
+
+    // report results
+    if(getUpgradeStatus() < 100) {
+      DataNode.LOG.info("\n   Distributed upgrade for DataNode version " 
+          + getVersion() + " to current LV " 
+          + FSConstants.LAYOUT_VERSION + " cannot be completed.");
+    }
+
+    // Complete the upgrade by calling the manager method
+    try {
+      dataNode.upgradeManager.completeUpgrade();
+    } catch(IOException e) {
+      DataNode.LOG.error(StringUtils.stringifyException(e));
+    }
+  }
+
+  /**
+   * Complete upgrade and return a status complete command for broadcasting.
+   * 
+   * Data-nodes finish upgrade at different times.
+   * The data-node needs to re-confirm with the name-node that the upgrade
+   * is complete while other nodes are still upgrading.
+   */
+  public UpgradeCommand completeUpgrade() throws IOException {
+    return new UpgradeCommand(UpgradeCommand.UC_ACTION_REPORT_STATUS,
+                              getVersion(), (short)100);
+  }
+}

+ 76 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeActivityMBean.java

@@ -0,0 +1,76 @@
+/**
+ * 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.datanode.metrics;
+import java.util.Random;
+
+import javax.management.ObjectName;
+import org.apache.hadoop.metrics.util.MBeanUtil;
+import org.apache.hadoop.metrics.util.MetricsDynamicMBeanBase;
+import org.apache.hadoop.metrics.util.MetricsRegistry;
+
+/**
+ * 
+ * This is the JMX MBean for reporting the DataNode Activity.
+ * The MBean is register using the name
+ *        "hadoop:service=DataNode,name=DataNodeActivity-<storageid>"
+ * 
+ * Many of the activity metrics are sampled and averaged on an interval 
+ * which can be specified in the metrics config file.
+ * <p>
+ * For the metrics that are sampled and averaged, one must specify 
+ * a metrics context that does periodic update calls. Most metrics contexts do.
+ * The default Null metrics context however does NOT. So if you aren't
+ * using any other metrics context then you can turn on the viewing and averaging
+ * of sampled metrics by  specifying the following two lines
+ *  in the hadoop-meterics.properties file:
+*  <pre>
+ *        dfs.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
+ *        dfs.period=10
+ *  </pre>
+ *<p>
+ * Note that the metrics are collected regardless of the context used.
+ * The context with the update thread is used to average the data periodically
+ *
+ *
+ *
+ * Impl details: We use a dynamic mbean that gets the list of the metrics
+ * from the metrics registry passed as an argument to the constructor
+ */
+
+public class DataNodeActivityMBean extends MetricsDynamicMBeanBase {
+  final private ObjectName mbeanName;
+  private Random rand = new Random(); 
+
+  public DataNodeActivityMBean(final MetricsRegistry mr, final String storageId) {
+    super(mr, "Activity statistics at the DataNode");
+    String storageName;
+    if (storageId.equals("")) {// Temp fix for the uninitialized storage
+      storageName = "UndefinedStorageId" + rand.nextInt();
+    } else {
+      storageName = storageId;
+    }
+    mbeanName = MBeanUtil.registerMBean("DataNode", "DataNodeActivity-" + storageName, this);
+  }
+  
+
+  public void shutdown() {
+    if (mbeanName != null)
+      MBeanUtil.unregisterMBean(mbeanName);
+  }
+}

+ 135 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java

@@ -0,0 +1,135 @@
+/**
+ * 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.datanode.metrics;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.metrics.MetricsContext;
+import org.apache.hadoop.metrics.MetricsRecord;
+import org.apache.hadoop.metrics.MetricsUtil;
+import org.apache.hadoop.metrics.Updater;
+import org.apache.hadoop.metrics.jvm.JvmMetrics;
+import org.apache.hadoop.metrics.util.MetricsBase;
+import org.apache.hadoop.metrics.util.MetricsRegistry;
+import org.apache.hadoop.metrics.util.MetricsTimeVaryingInt;
+import org.apache.hadoop.metrics.util.MetricsTimeVaryingLong;
+import org.apache.hadoop.metrics.util.MetricsTimeVaryingRate;
+
+
+/**
+ * 
+ * This class is for maintaining  the various DataNode statistics
+ * and publishing them through the metrics interfaces.
+ * This also registers the JMX MBean for RPC.
+ * <p>
+ * This class has a number of metrics variables that are publicly accessible;
+ * these variables (objects) have methods to update their values;
+ *  for example:
+ *  <p> {@link #blocksRead}.inc()
+ *
+ */
+public class DataNodeMetrics implements Updater {
+  private final MetricsRecord metricsRecord;
+  private DataNodeActivityMBean datanodeActivityMBean;
+  public MetricsRegistry registry = new MetricsRegistry();
+  
+  
+  public MetricsTimeVaryingLong bytesWritten = 
+                      new MetricsTimeVaryingLong("bytes_written", registry);
+  public MetricsTimeVaryingLong bytesRead = 
+                      new MetricsTimeVaryingLong("bytes_read", registry);
+  public MetricsTimeVaryingInt blocksWritten = 
+                      new MetricsTimeVaryingInt("blocks_written", registry);
+  public MetricsTimeVaryingInt blocksRead = 
+                      new MetricsTimeVaryingInt("blocks_read", registry);
+  public MetricsTimeVaryingInt blocksReplicated =
+                      new MetricsTimeVaryingInt("blocks_replicated", registry);
+  public MetricsTimeVaryingInt blocksRemoved =
+                       new MetricsTimeVaryingInt("blocks_removed", registry);
+  public MetricsTimeVaryingInt blocksVerified = 
+                        new MetricsTimeVaryingInt("blocks_verified", registry);
+  public MetricsTimeVaryingInt blockVerificationFailures =
+                       new MetricsTimeVaryingInt("block_verification_failures", registry);
+  
+  public MetricsTimeVaryingInt readsFromLocalClient = 
+                new MetricsTimeVaryingInt("reads_from_local_client", registry);
+  public MetricsTimeVaryingInt readsFromRemoteClient = 
+                new MetricsTimeVaryingInt("reads_from_remote_client", registry);
+  public MetricsTimeVaryingInt writesFromLocalClient = 
+              new MetricsTimeVaryingInt("writes_from_local_client", registry);
+  public MetricsTimeVaryingInt writesFromRemoteClient = 
+              new MetricsTimeVaryingInt("writes_from_remote_client", registry);
+  
+  public MetricsTimeVaryingRate readBlockOp = 
+                new MetricsTimeVaryingRate("readBlockOp", registry);
+  public MetricsTimeVaryingRate writeBlockOp = 
+                new MetricsTimeVaryingRate("writeBlockOp", registry);
+  public MetricsTimeVaryingRate blockChecksumOp = 
+                new MetricsTimeVaryingRate("blockChecksumOp", registry);
+  public MetricsTimeVaryingRate copyBlockOp = 
+                new MetricsTimeVaryingRate("copyBlockOp", registry);
+  public MetricsTimeVaryingRate replaceBlockOp = 
+                new MetricsTimeVaryingRate("replaceBlockOp", registry);
+  public MetricsTimeVaryingRate heartbeats = 
+                    new MetricsTimeVaryingRate("heartBeats", registry);
+  public MetricsTimeVaryingRate blockReports = 
+                    new MetricsTimeVaryingRate("blockReports", registry);
+
+    
+  public DataNodeMetrics(Configuration conf, String storageId) {
+    String sessionId = conf.get("session.id"); 
+    // Initiate reporting of Java VM metrics
+    JvmMetrics.init("DataNode", sessionId);
+    
+
+    // Now the MBean for the data node
+    datanodeActivityMBean = new DataNodeActivityMBean(registry, storageId);
+    
+    // Create record for DataNode metrics
+    MetricsContext context = MetricsUtil.getContext("dfs");
+    metricsRecord = MetricsUtil.createRecord(context, "datanode");
+    metricsRecord.setTag("sessionId", sessionId);
+    context.registerUpdater(this);
+  }
+  
+  public void shutdown() {
+    if (datanodeActivityMBean != null) 
+      datanodeActivityMBean.shutdown();
+  }
+    
+  /**
+   * Since this object is a registered updater, this method will be called
+   * periodically, e.g. every 5 seconds.
+   */
+  public void doUpdates(MetricsContext unused) {
+    synchronized (this) {
+      for (MetricsBase m : registry.getMetricsList()) {
+        m.pushMetric(metricsRecord);
+      }
+    }
+    metricsRecord.update();
+  }
+  public void resetAllMinMax() {
+    readBlockOp.resetMinMax();
+    writeBlockOp.resetMinMax();
+    blockChecksumOp.resetMinMax();
+    copyBlockOp.resetMinMax();
+    replaceBlockOp.resetMinMax();
+    heartbeats.resetMinMax();
+    blockReports.resetMinMax();
+  }
+}

+ 0 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeStatistics.java


+ 0 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeStatisticsMBean.java


+ 65 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java

@@ -0,0 +1,65 @@
+/**
+ * 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.datanode.metrics;
+
+import java.io.IOException;
+
+/**
+ * 
+ * This Interface defines the methods to get the status of a the FSDataset of
+ * a data node.
+ * It is also used for publishing via JMX (hence we follow the JMX naming
+ * convention.) 
+ *  * Note we have not used the MetricsDynamicMBeanBase to implement this
+ * because the interface for the FSDatasetMBean is stable and should
+ * be published as an interface.
+ * 
+ * <p>
+ * Data Node runtime statistic  info is report in another MBean
+ * @see org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeStatisticsMBean
+ *
+ */
+public interface FSDatasetMBean {
+  
+  /**
+   * Returns the total space (in bytes) used by dfs datanode
+   * @return  the total space used by dfs datanode
+   * @throws IOException
+   */  
+  public long getDfsUsed() throws IOException;
+    
+  /**
+   * Returns total capacity (in bytes) of storage (used and unused)
+   * @return  total capacity of storage (used and unused)
+   * @throws IOException
+   */
+  public long getCapacity() throws IOException;
+
+  /**
+   * Returns the amount of free storage space (in bytes)
+   * @return The amount of free storage space
+   * @throws IOException
+   */
+  public long getRemaining() throws IOException;
+  
+  /**
+   * Returns the storage id of the underlying storage
+   */
+  public String getStorageInfo();
+
+}

+ 342 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java

@@ -0,0 +1,342 @@
+/**
+ * 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 java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketTimeoutException;
+
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.namenode.FSImage.CheckpointStates;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.DNS;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.Daemon;
+
+/**
+ * BackupNode.
+ * <p>
+ * Backup node can play two roles.
+ * <ol>
+ * <li>{@link NamenodeRole#CHECKPOINT} node periodically creates checkpoints, 
+ * that is downloads image and edits from the active node, merges them, and
+ * uploads the new image back to the active.</li>
+ * <li>{@link NamenodeRole#BACKUP} node keeps its namespace in sync with the
+ * active node, and periodically creates checkpoints by simply saving the
+ * namespace image to local disk(s).</li>
+ * </ol>
+ */
+class BackupNode extends NameNode {
+  private static final String BN_ADDRESS_NAME_KEY = "dfs.backup.address";
+  private static final String BN_ADDRESS_DEFAULT = "localhost:50100";
+  private static final String BN_HTTP_ADDRESS_NAME_KEY = "dfs.backup.http.address";
+  private static final String BN_HTTP_ADDRESS_DEFAULT = "0.0.0.0:50105";
+
+  /** Name-node proxy */
+  NamenodeProtocol namenode;
+  /** Name-node RPC address */
+  String nnRpcAddress;
+  /** Name-node HTTP address */
+  String nnHttpAddress;
+  /** Checkpoint manager */
+  Checkpointer checkpointManager;
+  /** Checkpoint daemon */
+  private Daemon cpDaemon;
+
+  BackupNode(Configuration conf, NamenodeRole role) throws IOException {
+    super(conf, role);
+  }
+
+  /////////////////////////////////////////////////////
+  // Common NameNode methods implementation for backup node.
+  /////////////////////////////////////////////////////
+  @Override // NameNode
+  protected InetSocketAddress getRpcServerAddress(Configuration conf) throws IOException {
+    String addr = conf.get(BN_ADDRESS_NAME_KEY, BN_ADDRESS_DEFAULT);
+    int port = NetUtils.createSocketAddr(addr).getPort();
+    String hostName = DNS.getDefaultHost("default");
+    return new InetSocketAddress(hostName, port);
+  }
+
+  @Override // NameNode
+  protected void setRpcServerAddress(Configuration conf) {
+    conf.set(BN_ADDRESS_NAME_KEY, getHostPortString(rpcAddress));
+  }
+
+  @Override // NameNode
+  protected InetSocketAddress getHttpServerAddress(Configuration conf) {
+    assert rpcAddress != null : "rpcAddress should be calculated first";
+    String addr = conf.get(BN_HTTP_ADDRESS_NAME_KEY, BN_HTTP_ADDRESS_DEFAULT);
+    int port = NetUtils.createSocketAddr(addr).getPort();
+    String hostName = rpcAddress.getHostName();
+    return new InetSocketAddress(hostName, port);
+  }
+
+  @Override // NameNode
+  protected void setHttpServerAddress(Configuration conf){
+    conf.set(BN_HTTP_ADDRESS_NAME_KEY, getHostPortString(httpAddress));
+  }
+
+  @Override // NameNode
+  protected void loadNamesystem(Configuration conf) throws IOException {
+    BackupStorage bnImage = new BackupStorage();
+    this.namesystem = new FSNamesystem(conf, bnImage);
+    bnImage.recoverCreateRead(FSNamesystem.getNamespaceDirs(conf),
+                              FSNamesystem.getNamespaceEditsDirs(conf));
+  }
+
+  @Override // NameNode
+  protected void initialize(Configuration conf) throws IOException {
+    // Trash is disabled in BackupNameNode,
+    // but should be turned back on if it ever becomes active.
+    conf.setLong("fs.trash.interval", 0L);
+    NamespaceInfo nsInfo = handshake(conf);
+    super.initialize(conf);
+    // Backup node should never do lease recovery,
+    // therefore lease hard limit should never expire.
+    namesystem.leaseManager.setLeasePeriod(
+        FSConstants.LEASE_SOFTLIMIT_PERIOD, Long.MAX_VALUE);
+    // register with the active name-node 
+    registerWith(nsInfo);
+    // Checkpoint daemon should start after the rpc server started
+    runCheckpointDaemon(conf);
+  }
+
+  @Override // NameNode
+  public void stop() {
+    if(checkpointManager != null) checkpointManager.shouldRun = false;
+    if(cpDaemon != null) cpDaemon.interrupt();
+    if(namenode != null && getRegistration() != null) {
+      try {
+        namenode.errorReport(getRegistration(), NamenodeProtocol.FATAL,
+            "Shutting down.");
+      } catch(IOException e) {
+        LOG.error("Failed to report to name-node.", e);
+      }
+    }
+    RPC.stopProxy(namenode); // stop the RPC threads
+    super.stop();
+  }
+
+  /////////////////////////////////////////////////////
+  // NamenodeProtocol implementation for backup node.
+  /////////////////////////////////////////////////////
+  @Override // NamenodeProtocol
+  public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
+  throws IOException {
+    throw new UnsupportedActionException("getBlocks");
+  }
+
+  // Only active name-node can register other nodes.
+  @Override // NamenodeProtocol
+  public NamenodeRegistration register(NamenodeRegistration registration
+  ) throws IOException {
+    throw new UnsupportedActionException("journal");
+  }
+
+  @Override // NamenodeProtocol
+  public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
+  throws IOException {
+    throw new UnsupportedActionException("startCheckpoint");
+  }
+
+  @Override // NamenodeProtocol
+  public void endCheckpoint(NamenodeRegistration registration,
+                            CheckpointSignature sig) throws IOException {
+    throw new UnsupportedActionException("endCheckpoint");
+  }
+
+  @Override // NamenodeProtocol
+  public void journal(NamenodeRegistration nnReg,
+                      int jAction,
+                      int length,
+                      byte[] args) throws IOException {
+    verifyRequest(nnReg);
+    if(!nnRpcAddress.equals(nnReg.getAddress()))
+      throw new IOException("Journal request from unexpected name-node: "
+          + nnReg.getAddress() + " expecting " + nnRpcAddress);
+    BackupStorage bnImage = (BackupStorage)getFSImage();
+    switch(jAction) {
+      case (int)JA_IS_ALIVE:
+        return;
+      case (int)JA_JOURNAL:
+        bnImage.journal(length, args);
+        return;
+      case (int)JA_JSPOOL_START:
+        bnImage.startJournalSpool(nnReg);
+        return;
+      case (int)JA_CHECKPOINT_TIME:
+        bnImage.setCheckpointTime(length, args);
+        setRegistration(); // keep registration up to date
+        return;
+      default:
+        throw new IOException("Unexpected journal action: " + jAction);
+    }
+  }
+
+  boolean shouldCheckpointAtStartup() {
+    FSImage fsImage = getFSImage();
+    if(isRole(NamenodeRole.CHECKPOINT)) {
+      assert fsImage.getNumStorageDirs() > 0;
+      return ! fsImage.getStorageDir(0).getVersionFile().exists();
+    }
+    if(namesystem == null || namesystem.dir == null || getFSImage() == null)
+      return true;
+    return fsImage.getEditLog().getNumEditStreams() == 0;
+  }
+
+  private NamespaceInfo handshake(Configuration conf) throws IOException {
+    // connect to name node
+    InetSocketAddress nnAddress = super.getRpcServerAddress(conf);
+    this.namenode =
+      (NamenodeProtocol) RPC.waitForProxy(NamenodeProtocol.class,
+          NamenodeProtocol.versionID, nnAddress, conf);
+    this.nnRpcAddress = getHostPortString(nnAddress);
+    this.nnHttpAddress = getHostPortString(super.getHttpServerAddress(conf));
+    // get version and id info from the name-node
+    NamespaceInfo nsInfo = null;
+    while(!stopRequested) {
+      try {
+        nsInfo = handshake(namenode);
+        break;
+      } catch(SocketTimeoutException e) {  // name-node is busy
+        LOG.info("Problem connecting to server: " + nnAddress);
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException ie) {}
+      }
+    }
+    return nsInfo;
+  }
+
+  /**
+   * Start a backup node daemon.
+   */
+  private void runCheckpointDaemon(Configuration conf) throws IOException {
+    checkpointManager = new Checkpointer(conf, this);
+    cpDaemon = new Daemon(checkpointManager);
+    cpDaemon.start();
+  }
+
+  /**
+   * Checkpoint.<br>
+   * Tests may use it to initiate a checkpoint process.
+   * @throws IOException
+   */
+  void doCheckpoint() throws IOException {
+    checkpointManager.doCheckpoint();
+  }
+
+  CheckpointStates getCheckpointState() {
+    return getFSImage().getCheckpointState();
+  }
+
+  void setCheckpointState(CheckpointStates cs) {
+    getFSImage().setCheckpointState(cs);
+  }
+
+  /**
+   * Register this backup node with the active name-node.
+   * @param nsInfo
+   * @throws IOException
+   */
+  private void registerWith(NamespaceInfo nsInfo) throws IOException {
+    BackupStorage bnImage = (BackupStorage)getFSImage();
+    // verify namespaceID
+    if(bnImage.getNamespaceID() == 0) // new backup storage
+      bnImage.setStorageInfo(nsInfo);
+    else if(bnImage.getNamespaceID() != nsInfo.getNamespaceID())
+      throw new IOException("Incompatible namespaceIDs"
+          + ": active node namespaceID = " + nsInfo.getNamespaceID() 
+          + "; backup node namespaceID = " + bnImage.getNamespaceID());
+
+    setRegistration();
+    NamenodeRegistration nnReg = null;
+    while(!stopRequested) {
+      try {
+        nnReg = namenode.register(getRegistration());
+        break;
+      } catch(SocketTimeoutException e) {  // name-node is busy
+        LOG.info("Problem connecting to name-node: " + nnRpcAddress);
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException ie) {}
+      }
+    }
+
+    String msg = null;
+    if(nnReg == null) // consider as a rejection
+      msg = "Registration rejected by " + nnRpcAddress;
+    else if(!nnReg.isRole(NamenodeRole.ACTIVE)) {
+      msg = "Name-node " + nnRpcAddress + " is not active";
+    }
+    if(msg != null) {
+      msg += ". Shutting down.";
+      LOG.error(msg);
+      throw new IOException(msg); // stop the node
+    }
+    nnRpcAddress = nnReg.getAddress();
+  }
+
+  /**
+   * Reset node namespace state in memory and in storage directories.
+   * @throws IOException
+   */
+  void resetNamespace() throws IOException {
+    ((BackupStorage)getFSImage()).reset();
+  }
+
+  /**
+   * Get size of the local journal (edit log).
+   * @return size of the current journal
+   * @throws IOException
+   */
+  long journalSize() throws IOException {
+    return namesystem.getEditLogSize();
+  }
+
+  // TODO: move to a common with DataNode util class
+  private static NamespaceInfo handshake(NamenodeProtocol namenode)
+  throws IOException, SocketTimeoutException {
+    NamespaceInfo nsInfo;
+    nsInfo = namenode.versionRequest();  // throws SocketTimeoutException 
+    String errorMsg = null;
+    // verify build version
+    if( ! nsInfo.getBuildVersion().equals( Storage.getBuildVersion())) {
+      errorMsg = "Incompatible build versions: active name-node BV = " 
+        + nsInfo.getBuildVersion() + "; backup node BV = "
+        + Storage.getBuildVersion();
+      LOG.fatal(errorMsg);
+      throw new IOException(errorMsg);
+    }
+    assert FSConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
+      "Active and backup node layout versions must be the same. Expected: "
+      + FSConstants.LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
+    return nsInfo;
+  }
+}

+ 388 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/BackupStorage.java

@@ -0,0 +1,388 @@
+/**
+ * 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 java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.io.LongWritable;
+
+public class BackupStorage extends FSImage {
+  // Names of the journal spool directory and the spool file
+  private static final String STORAGE_JSPOOL_DIR = "jspool";
+  private static final String STORAGE_JSPOOL_FILE = 
+                                              NameNodeFile.EDITS_NEW.getName();
+
+  /** Backup input stream for loading edits into memory */
+  private EditLogBackupInputStream backupInputStream;
+  /** Is journal spooling in progress */
+  volatile JSpoolState jsState;
+
+  static enum JSpoolState {
+    OFF,
+    INPROGRESS,
+    WAIT;
+  }
+
+  /**
+   */
+  BackupStorage() {
+    super();
+    jsState = JSpoolState.OFF;
+  }
+
+  @Override
+  public boolean isConversionNeeded(StorageDirectory sd) {
+    return false;
+  }
+
+  /**
+   * Analyze backup storage directories for consistency.<br>
+   * Recover from incomplete checkpoints if required.<br>
+   * Read VERSION and fstime files if exist.<br>
+   * Do not load image or edits.
+   * 
+   * @param imageDirs list of image directories.
+   * @param editsDirs list of edits directories.
+   * @throws IOException if the node should shutdown.
+   */
+  void recoverCreateRead(Collection<File> imageDirs,
+                         Collection<File> editsDirs) throws IOException {
+    setStorageDirectories(imageDirs, editsDirs);
+    this.checkpointTime = 0L;
+    for(Iterator<StorageDirectory> it = dirIterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      StorageState curState;
+      try {
+        curState = sd.analyzeStorage(HdfsConstants.StartupOption.REGULAR);
+        // sd is locked but not opened
+        switch(curState) {
+        case NON_EXISTENT:
+          // fail if any of the configured storage dirs are inaccessible 
+          throw new InconsistentFSStateException(sd.getRoot(),
+                "checkpoint directory does not exist or is not accessible.");
+        case NOT_FORMATTED:
+          // for backup node all directories may be unformatted initially
+          LOG.info("Storage directory " + sd.getRoot() + " is not formatted.");
+          LOG.info("Formatting ...");
+          sd.clearDirectory(); // create empty current
+          break;
+        case NORMAL:
+          break;
+        default:  // recovery is possible
+          sd.doRecover(curState);
+        }
+        if(curState != StorageState.NOT_FORMATTED) {
+          sd.read(); // read and verify consistency with other directories
+        }
+      } catch(IOException ioe) {
+        sd.unlock();
+        throw ioe;
+      }
+    }
+  }
+
+  /**
+   * Reset storage directories.
+   * <p>
+   * Unlock the storage.
+   * Rename <code>current</code> to <code>lastcheckpoint.tmp</code>
+   * and recreate empty <code>current</code>.
+   * @throws IOException
+   */
+  synchronized void reset() throws IOException {
+    // reset NameSpace tree
+    FSDirectory fsDir = getFSNamesystem().dir;
+    fsDir.reset();
+
+    // unlock, close and rename storage directories
+    unlockAll();
+    // recover from unsuccessful checkpoint if necessary
+    recoverCreateRead(getImageDirectories(), getEditsDirectories());
+    // rename and recreate
+    for(StorageDirectory sd : storageDirs) {
+      File curDir = sd.getCurrentDir();
+      File tmpCkptDir = sd.getLastCheckpointTmp();
+      assert !tmpCkptDir.exists() : 
+        tmpCkptDir.getName() + " directory must not exist.";
+      if(!sd.getVersionFile().exists())
+        continue;
+      // rename current to lastcheckpoint.tmp
+      rename(curDir, tmpCkptDir);
+      if(!curDir.mkdir())
+        throw new IOException("Cannot create directory " + curDir);
+    }
+  }
+
+  /**
+   * Load checkpoint from local files only if the memory state is empty.<br>
+   * Set new checkpoint time received from the name-node.<br>
+   * Move <code>lastcheckpoint.tmp</code> to <code>previous.checkpoint</code>.
+   * @throws IOException
+   */
+  void loadCheckpoint(CheckpointSignature sig) throws IOException {
+    // load current image and journal if it is not in memory already
+    if(!editLog.isOpen())
+      editLog.open();
+
+    FSDirectory fsDir = getFSNamesystem().dir;
+    if(fsDir.isEmpty()) {
+      Iterator<StorageDirectory> itImage = dirIterator(NameNodeDirType.IMAGE);
+      Iterator<StorageDirectory> itEdits = dirIterator(NameNodeDirType.EDITS);
+      if(!itImage.hasNext() || ! itEdits.hasNext())
+        throw new IOException("Could not locate checkpoint directories");
+      StorageDirectory sdName = itImage.next();
+      StorageDirectory sdEdits = itEdits.next();
+      synchronized(getFSDirectoryRootLock()) { // load image under rootDir lock
+        loadFSImage(FSImage.getImageFile(sdName, NameNodeFile.IMAGE));
+      }
+      loadFSEdits(sdEdits);
+    }
+
+    // set storage fields
+    setStorageInfo(sig);
+    checkpointTime = sig.checkpointTime;
+  }
+
+  /**
+   * Save meta-data into fsimage files.
+   * and create empty edits.
+   */
+  void saveCheckpoint() throws IOException {
+    // save image into fsimage.ckpt and purge edits file
+    for (Iterator<StorageDirectory> it = 
+                           dirIterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      NameNodeDirType dirType = (NameNodeDirType)sd.getStorageDirType();
+      if (dirType.isOfType(NameNodeDirType.IMAGE))
+        saveFSImage(getImageFile(sd, NameNodeFile.IMAGE_NEW));
+      if (dirType.isOfType(NameNodeDirType.EDITS))
+        editLog.createEditLogFile(getImageFile(sd, NameNodeFile.EDITS));
+    }
+
+    ckptState = CheckpointStates.UPLOAD_DONE;
+    renameCheckpoint();
+  }
+
+  private Object getFSDirectoryRootLock() {
+    return getFSNamesystem().dir.rootDir;
+  }
+
+  static File getJSpoolDir(StorageDirectory sd) {
+    return new File(sd.getRoot(), STORAGE_JSPOOL_DIR);
+  }
+
+  static File getJSpoolFile(StorageDirectory sd) {
+    return new File(getJSpoolDir(sd), STORAGE_JSPOOL_FILE);
+  }
+
+  /**
+   * Journal writer journals new meta-data state.
+   * <ol>
+   * <li> If Journal Spool state is OFF then journal records (edits)
+   * are applied directly to meta-data state in memory and are written 
+   * to the edits file(s).</li>
+   * <li> If Journal Spool state is INPROGRESS then records are only 
+   * written to edits.new file, which is called Spooling.</li>
+   * <li> Journal Spool state WAIT blocks journaling until the
+   * Journal Spool reader finalizes merging of the spooled data and
+   * switches to applying journal to memory.</li>
+   * </ol>
+   * @param length length of data.
+   * @param data serialized journal records.
+   * @throws IOException
+   * @see #convergeJournalSpool()
+   */
+  synchronized void journal(int length, byte[] data) throws IOException {
+    assert backupInputStream.length() == 0 : "backup input stream is not empty";
+    try {
+      switch(jsState) {
+        case WAIT:
+        case OFF:
+          // wait until spooling is off
+          waitSpoolEnd();
+          // update NameSpace in memory
+          backupInputStream.setBytes(data);
+          editLog.loadEditRecords(getLayoutVersion(),
+                    backupInputStream.getDataInputStream(), true);
+          getFSNamesystem().dir.updateCountForINodeWithQuota(); // inefficient!
+          break;
+        case INPROGRESS:
+          break;
+      }
+      // write to files
+      editLog.logEdit(length, data);
+      editLog.logSync();
+    } finally {
+      backupInputStream.clear();
+    }
+  }
+
+  private synchronized void waitSpoolEnd() {
+    while(jsState == JSpoolState.WAIT) {
+      try {
+        wait();
+      } catch (InterruptedException  e) {}
+    }
+    // now spooling should be off, verifying just in case
+    assert jsState == JSpoolState.OFF : "Unexpected JSpool state: " + jsState;
+  }
+
+  /**
+   * Start journal spool.
+   * Switch to writing into edits.new instead of edits.
+   * 
+   * edits.new for spooling is in separate directory "spool" rather than in
+   * "current" because the two directories should be independent.
+   * While spooling a checkpoint can happen and current will first
+   * move to lastcheckpoint.tmp and then to previous.checkpoint
+   * spool/edits.new will remain in place during that.
+   */
+  synchronized void startJournalSpool(NamenodeRegistration nnReg)
+  throws IOException {
+    switch(jsState) {
+      case OFF:
+        break;
+      case INPROGRESS:
+        return;
+      case WAIT:
+        waitSpoolEnd();
+    }
+
+    // create journal spool directories
+    for(Iterator<StorageDirectory> it = 
+                          dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      File jsDir = getJSpoolDir(sd);
+      if (!jsDir.exists() && !jsDir.mkdirs()) {
+        throw new IOException("Mkdirs failed to create "
+                              + jsDir.getCanonicalPath());
+      }
+      // create edit file if missing
+      File eFile = getEditFile(sd);
+      if(!eFile.exists()) {
+        editLog.createEditLogFile(eFile);
+      }
+    }
+
+    if(!editLog.isOpen())
+      editLog.open();
+
+    // create streams pointing to the journal spool files
+    // subsequent journal records will go directly to the spool
+    editLog.divertFileStreams(STORAGE_JSPOOL_DIR + "/" + STORAGE_JSPOOL_FILE);
+    setCheckpointState(CheckpointStates.ROLLED_EDITS);
+
+    // set up spooling
+    if(backupInputStream == null)
+      backupInputStream = new EditLogBackupInputStream(nnReg.getAddress());
+    jsState = JSpoolState.INPROGRESS;
+  }
+
+  synchronized void setCheckpointTime(int length, byte[] data)
+  throws IOException {
+    assert backupInputStream.length() == 0 : "backup input stream is not empty";
+    try {
+      // unpack new checkpoint time
+      backupInputStream.setBytes(data);
+      DataInputStream in = backupInputStream.getDataInputStream();
+      byte op = in.readByte();
+      assert op == NamenodeProtocol.JA_CHECKPOINT_TIME;
+      LongWritable lw = new LongWritable();
+      lw.readFields(in);
+      setCheckpointTime(lw.get());
+    } finally {
+      backupInputStream.clear();
+    }
+  }
+
+  /**
+   * Merge Journal Spool to memory.<p>
+   * Journal Spool reader reads journal records from edits.new.
+   * When it reaches the end of the file it sets {@link JSpoolState} to WAIT.
+   * This blocks journaling (see {@link #journal(int,byte[])}.
+   * The reader
+   * <ul>
+   * <li> reads remaining journal records if any,</li>
+   * <li> renames edits.new to edits,</li>
+   * <li> sets {@link JSpoolState} to OFF,</li> 
+   * <li> and notifies the journaling thread.</li>
+   * </ul>
+   * Journaling resumes with applying new journal records to the memory state,
+   * and writing them into edits file(s).
+   */
+  void convergeJournalSpool() throws IOException {
+    Iterator<StorageDirectory> itEdits = dirIterator(NameNodeDirType.EDITS);
+    if(! itEdits.hasNext())
+      throw new IOException("Could not locate checkpoint directories");
+    StorageDirectory sdEdits = itEdits.next();
+    int numEdits = 0;
+    File jSpoolFile = getJSpoolFile(sdEdits);
+    long startTime = FSNamesystem.now();
+    if(jSpoolFile.exists()) {
+      // load edits.new
+      EditLogFileInputStream edits = new EditLogFileInputStream(jSpoolFile);
+      DataInputStream in = edits.getDataInputStream();
+      numEdits += editLog.loadFSEdits(in, false);
+  
+      // first time reached the end of spool
+      jsState = JSpoolState.WAIT;
+      numEdits += editLog.loadEditRecords(getLayoutVersion(), in, true);
+      getFSNamesystem().dir.updateCountForINodeWithQuota();
+      edits.close();
+    }
+
+    FSImage.LOG.info("Edits file " + jSpoolFile.getCanonicalPath() 
+        + " of size " + jSpoolFile.length() + " edits # " + numEdits 
+        + " loaded in " + (FSNamesystem.now()-startTime)/1000 + " seconds.");
+
+    // rename spool edits.new to edits making it in sync with the active node
+    // subsequent journal records will go directly to edits
+    editLog.revertFileStreams(STORAGE_JSPOOL_DIR + "/" + STORAGE_JSPOOL_FILE);
+
+    // write version file
+    resetVersion(false);
+
+    // wake up journal writer
+    synchronized(this) {
+      jsState = JSpoolState.OFF;
+      notifyAll();
+    }
+
+    // Rename lastcheckpoint.tmp to previous.checkpoint
+    for(StorageDirectory sd : storageDirs) {
+      File tmpCkptDir = sd.getLastCheckpointTmp();
+      File prevCkptDir = sd.getPreviousCheckpoint();
+      // delete previous directory
+      if (prevCkptDir.exists())
+        deleteDir(prevCkptDir);
+      // rename tmp to previous
+      if (tmpCkptDir.exists())
+        rename(tmpCkptDir, prevCkptDir);
+    }
+  }
+}

+ 1375 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java

@@ -0,0 +1,1375 @@
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
+import org.apache.hadoop.hdfs.server.namenode.UnderReplicatedBlocks.BlockIterator;
+import org.apache.hadoop.security.AccessTokenHandler;
+
+/**
+ * Keeps information related to the blocks stored in the Hadoop cluster.
+ * This class is a helper class for {@link FSNamesystem} and requires several
+ * methods to be called with lock held on {@link FSNamesystem}.
+ */
+public class BlockManager {
+  private final FSNamesystem namesystem;
+
+  long pendingReplicationBlocksCount = 0L, corruptReplicaBlocksCount,
+  underReplicatedBlocksCount = 0L, scheduledReplicationBlocksCount = 0L;
+
+  //
+  // Mapping: Block -> { INode, datanodes, self ref }
+  // Updated only in response to client-sent information.
+  //
+  BlocksMap blocksMap = new BlocksMap();
+
+  //
+  // Store blocks-->datanodedescriptor(s) map of corrupt replicas
+  //
+  CorruptReplicasMap corruptReplicas = new CorruptReplicasMap();
+
+  //
+  // Keeps a Collection for every named machine containing
+  // blocks that have recently been invalidated and are thought to live
+  // on the machine in question.
+  // Mapping: StorageID -> ArrayList<Block>
+  //
+  Map<String, Collection<Block>> recentInvalidateSets =
+    new TreeMap<String, Collection<Block>>();
+
+  //
+  // Keeps a TreeSet for every named node. Each treeset contains
+  // a list of the blocks that are "extra" at that location. We'll
+  // eventually remove these extras.
+  // Mapping: StorageID -> TreeSet<Block>
+  //
+  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();
+  private PendingReplicationBlocks pendingReplications;
+
+  //  The maximum number of replicas allowed for a block
+  int maxReplication;
+  //  How many outgoing replication streams a given node should have at one time
+  int maxReplicationStreams;
+  // Minimum copies needed or else write is disallowed
+  int minReplication;
+  // Default number of replicas
+  int defaultReplication;
+
+  /**
+   * Last block index used for replication work.
+   */
+  private int replIndex = 0;
+  private long missingBlocksInCurIter = 0;
+  private long missingBlocksInPrevIter = 0;
+  Random r = new Random();
+
+  // for block replicas placement
+  ReplicationTargetChooser replicator;
+
+  BlockManager(FSNamesystem fsn, Configuration conf) throws IOException {
+    namesystem = fsn;
+    pendingReplications = new PendingReplicationBlocks(
+        conf.getInt("dfs.replication.pending.timeout.sec",
+                    -1) * 1000L);
+    setConfigurationParameters(conf);
+  }
+
+  void setConfigurationParameters(Configuration conf) throws IOException {
+    this.replicator = new ReplicationTargetChooser(
+                          conf.getBoolean("dfs.replication.considerLoad", true),
+                          namesystem,
+                          namesystem.clusterMap);
+
+    this.defaultReplication = conf.getInt("dfs.replication", 3);
+    this.maxReplication = conf.getInt("dfs.replication.max", 512);
+    this.minReplication = conf.getInt("dfs.replication.min", 1);
+    if (minReplication <= 0)
+      throw new IOException(
+                            "Unexpected configuration parameters: dfs.replication.min = "
+                            + minReplication
+                            + " must be greater than 0");
+    if (maxReplication >= (int)Short.MAX_VALUE)
+      throw new IOException(
+                            "Unexpected configuration parameters: dfs.replication.max = "
+                            + maxReplication + " must be less than " + (Short.MAX_VALUE));
+    if (maxReplication < minReplication)
+      throw new IOException(
+                            "Unexpected configuration parameters: dfs.replication.min = "
+                            + minReplication
+                            + " must be less than dfs.replication.max = "
+                            + maxReplication);
+    this.maxReplicationStreams = conf.getInt("dfs.max-repl-streams", 2);
+    FSNamesystem.LOG.info("defaultReplication = " + defaultReplication);
+    FSNamesystem.LOG.info("maxReplication = " + maxReplication);
+    FSNamesystem.LOG.info("minReplication = " + minReplication);
+    FSNamesystem.LOG.info("maxReplicationStreams = " + maxReplicationStreams);
+  }
+
+  void activate() {
+    pendingReplications.start();
+  }
+
+  void close() {
+    if (pendingReplications != null) pendingReplications.stop();
+  }
+
+  void metaSave(PrintWriter out) {
+    //
+    // Dump contents of neededReplication
+    //
+    synchronized (neededReplications) {
+      out.println("Metasave: Blocks waiting for replication: "
+          + neededReplications.size());
+      for (Block block : neededReplications) {
+        List<DatanodeDescriptor> containingNodes =
+                                          new ArrayList<DatanodeDescriptor>();
+        NumberReplicas numReplicas = new NumberReplicas();
+        // source node returned is not used
+        chooseSourceDatanode(block, containingNodes, numReplicas);
+        int usableReplicas = numReplicas.liveReplicas() +
+                             numReplicas.decommissionedReplicas();
+        // l: == live:, d: == decommissioned c: == corrupt e: == excess
+        out.print(block + ((usableReplicas > 0)? "" : " MISSING") + 
+                  " (replicas:" +
+                  " l: " + numReplicas.liveReplicas() +
+                  " d: " + numReplicas.decommissionedReplicas() +
+                  " c: " + numReplicas.corruptReplicas() +
+                  " e: " + numReplicas.excessReplicas() + ") "); 
+
+        Collection<DatanodeDescriptor> corruptNodes = 
+                                      corruptReplicas.getNodes(block);
+        
+        for (Iterator<DatanodeDescriptor> jt = blocksMap.nodeIterator(block);
+             jt.hasNext();) {
+          DatanodeDescriptor node = jt.next();
+          String state = "";
+          if (corruptNodes != null && corruptNodes.contains(node)) {
+            state = "(corrupt)";
+          } else if (node.isDecommissioned() || 
+              node.isDecommissionInProgress()) {
+            state = "(decommissioned)";
+          }          
+          out.print(" " + node + state + " : ");
+        }
+        out.println("");
+      }
+    }
+
+    //
+    // Dump blocks from pendingReplication
+    //
+    pendingReplications.metaSave(out);
+
+    //
+    // Dump blocks that are waiting to be deleted
+    //
+    dumpRecentInvalidateSets(out);
+  }
+
+  /**
+   * @param block
+   * @return true if the block has minimum replicas
+   */
+  boolean checkMinReplication(Block block) {
+    return (blocksMap.numNodes(block) >= minReplication);
+  }
+
+  /**
+   * Get all valid locations of the block & add the block to results
+   * return the length of the added block; 0 if the block is not added
+   */
+  ArrayList<String> addBlock(Block block) {
+    ArrayList<String> machineSet =
+      new ArrayList<String>(blocksMap.numNodes(block));
+    for(Iterator<DatanodeDescriptor> it =
+      blocksMap.nodeIterator(block); it.hasNext();) {
+      String storageID = it.next().getStorageID();
+      // filter invalidate replicas
+      Collection<Block> blocks = recentInvalidateSets.get(storageID);
+      if(blocks==null || !blocks.contains(block)) {
+        machineSet.add(storageID);
+      }
+    }
+    return machineSet;
+  }
+
+
+  List<LocatedBlock> getBlockLocations(Block[] blocks, long offset,
+      long length, int nrBlocksToReturn) throws IOException {
+    int curBlk = 0;
+    long curPos = 0, blkSize = 0;
+    int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
+    for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
+      blkSize = blocks[curBlk].getNumBytes();
+      assert blkSize > 0 : "Block of size 0";
+      if (curPos + blkSize > offset) {
+        break;
+      }
+      curPos += blkSize;
+    }
+
+    if (nrBlocks > 0 && curBlk == nrBlocks)   // offset >= end of file
+      return null;
+
+    long endOff = offset + length;
+    List<LocatedBlock> results = new ArrayList<LocatedBlock>(blocks.length);
+    do {
+      // get block locations
+      int numNodes = blocksMap.numNodes(blocks[curBlk]);
+      int numCorruptNodes = countNodes(blocks[curBlk]).corruptReplicas();
+      int numCorruptReplicas = corruptReplicas
+          .numCorruptReplicas(blocks[curBlk]);
+      if (numCorruptNodes != numCorruptReplicas) {
+        FSNamesystem.LOG.warn("Inconsistent number of corrupt replicas for "
+            + blocks[curBlk] + "blockMap has " + numCorruptNodes
+            + " but corrupt replicas map has " + numCorruptReplicas);
+      }
+      boolean blockCorrupt = (numCorruptNodes == numNodes);
+      int numMachineSet = blockCorrupt ? numNodes
+          : (numNodes - numCorruptNodes);
+      DatanodeDescriptor[] machineSet = new DatanodeDescriptor[numMachineSet];
+      if (numMachineSet > 0) {
+        numNodes = 0;
+        for (Iterator<DatanodeDescriptor> it = blocksMap
+            .nodeIterator(blocks[curBlk]); it.hasNext();) {
+          DatanodeDescriptor dn = it.next();
+          boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(
+              blocks[curBlk], dn);
+          if (blockCorrupt || (!blockCorrupt && !replicaCorrupt))
+            machineSet[numNodes++] = dn;
+        }
+      }
+      LocatedBlock b = new LocatedBlock(blocks[curBlk], machineSet, curPos,
+          blockCorrupt);
+      if (namesystem.isAccessTokenEnabled) {
+        b.setAccessToken(namesystem.accessTokenHandler.generateToken(b.getBlock()
+            .getBlockId(), EnumSet.of(AccessTokenHandler.AccessMode.READ)));
+      }
+      results.add(b);
+      curPos += blocks[curBlk].getNumBytes();
+      curBlk++;
+    } while (curPos < endOff && curBlk < blocks.length
+        && results.size() < nrBlocksToReturn);
+    return results;
+  }
+
+  /**
+   * Check whether the replication parameter is within the range
+   * determined by system configuration.
+   */
+   void verifyReplication(String src,
+                          short replication,
+                          String clientName) throws IOException {
+
+    if (replication >= minReplication && replication <= maxReplication) {
+      //common case. avoid building 'text'
+      return;
+    }
+
+    String text = "file " + src + ((clientName != null) ? " on client "
+      + clientName : "") + ".\n" + "Requested replication " + replication;
+
+    if (replication > maxReplication)
+      throw new IOException(text + " exceeds maximum " + maxReplication);
+
+    if (replication < minReplication)
+      throw new IOException(text + " is less than the required minimum " +
+                            minReplication);
+  }
+
+  void removeFromInvalidates(String datanodeId, Block block) {
+    Collection<Block> v = recentInvalidateSets.get(datanodeId);
+    if (v != null && v.remove(block) && v.isEmpty()) {
+      recentInvalidateSets.remove(datanodeId);
+    }
+  }
+
+  /**
+   * Adds block to list of blocks which will be invalidated on specified
+   * datanode and log the move
+   *
+   * @param b block
+   * @param dn datanode
+   */
+  void addToInvalidates(Block b, DatanodeInfo dn) {
+    Collection<Block> invalidateSet = recentInvalidateSets
+        .get(dn.getStorageID());
+    if (invalidateSet == null) {
+      invalidateSet = new HashSet<Block>();
+      recentInvalidateSets.put(dn.getStorageID(), invalidateSet);
+    }
+    if (invalidateSet.add(b)) {
+      NameNode.stateChangeLog.info("BLOCK* NameSystem.addToInvalidates: "
+          + b.getBlockName() + " is added to invalidSet of " + dn.getName());
+    }
+  }
+
+  /**
+   * Adds block to list of blocks which will be invalidated on all its
+   * datanodes.
+   */
+  private void addToInvalidates(Block b) {
+    for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(b); it
+        .hasNext();) {
+      DatanodeDescriptor node = it.next();
+      addToInvalidates(b, node);
+    }
+  }
+
+  /**
+   * dumps the contents of recentInvalidateSets
+   */
+  private void dumpRecentInvalidateSets(PrintWriter out) {
+    int size = recentInvalidateSets.values().size();
+    out.println("Metasave: Blocks waiting deletion from "+size+" datanodes.");
+    if (size == 0) {
+      return;
+    }
+    for(Map.Entry<String,Collection<Block>> entry : recentInvalidateSets.entrySet()) {
+      Collection<Block> blocks = entry.getValue();
+      if (blocks.size() > 0) {
+        out.println(namesystem.getDatanode(entry.getKey()).getName() + blocks);
+      }
+    }
+  }
+
+  void markBlockAsCorrupt(Block blk, DatanodeInfo dn) throws IOException {
+    DatanodeDescriptor node = namesystem.getDatanode(dn);
+    if (node == null) {
+      throw new IOException("Cannot mark block" + blk.getBlockName()
+          + " as corrupt because datanode " + dn.getName()
+          + " does not exist. ");
+    }
+
+    final BlockInfo storedBlockInfo = blocksMap.getStoredBlock(blk);
+    if (storedBlockInfo == null) {
+      // Check if the replica is in the blockMap, if not
+      // ignore the request for now. This could happen when BlockScanner
+      // thread of Datanode reports bad block before Block reports are sent
+      // by the Datanode on startup
+      NameNode.stateChangeLog.info("BLOCK NameSystem.markBlockAsCorrupt: "
+          + "block " + blk + " could not be marked "
+          + "as corrupt as it does not exists in " + "blocksMap");
+    } else {
+      INodeFile inode = storedBlockInfo.getINode();
+      if (inode == null) {
+        NameNode.stateChangeLog.info("BLOCK NameSystem.markBlockAsCorrupt: "
+            + "block " + blk + " could not be marked "
+            + "as corrupt as it does not belong to " + "any file");
+        addToInvalidates(storedBlockInfo, node);
+        return;
+      }
+      // Add this replica to corruptReplicas Map
+      corruptReplicas.addToCorruptReplicasMap(storedBlockInfo, node);
+      if (countNodes(storedBlockInfo).liveReplicas() > inode.getReplication()) {
+        // the block is over-replicated so invalidate the replicas immediately
+        invalidateBlock(storedBlockInfo, node);
+      } else {
+        // add the block to neededReplication
+        updateNeededReplications(storedBlockInfo, -1, 0);
+      }
+    }
+  }
+
+  /**
+   * Invalidates the given block on the given datanode.
+   */
+  private void invalidateBlock(Block blk, DatanodeInfo dn)
+      throws IOException {
+    NameNode.stateChangeLog.info("DIR* NameSystem.invalidateBlock: " + blk
+        + " on " + dn.getName());
+    DatanodeDescriptor node = namesystem.getDatanode(dn);
+    if (node == null) {
+      throw new IOException("Cannot invalidate block " + blk
+          + " because datanode " + dn.getName() + " does not exist.");
+    }
+
+    // Check how many copies we have of the block. If we have at least one
+    // copy on a live node, then we can delete it.
+    int count = countNodes(blk).liveReplicas();
+    if (count > 1) {
+      addToInvalidates(blk, dn);
+      removeStoredBlock(blk, node);
+      NameNode.stateChangeLog.debug("BLOCK* NameSystem.invalidateBlocks: "
+          + blk + " on " + dn.getName() + " listed for deletion.");
+    } else {
+      NameNode.stateChangeLog.info("BLOCK* NameSystem.invalidateBlocks: "
+          + blk + " on " + dn.getName()
+          + " is the only copy and was not deleted.");
+    }
+  }
+
+  void updateState() {
+    pendingReplicationBlocksCount = pendingReplications.size();
+    underReplicatedBlocksCount = neededReplications.size();
+    corruptReplicaBlocksCount = corruptReplicas.size();
+  }
+
+  /**
+   * Schedule blocks for deletion at datanodes
+   * @param nodesToProcess number of datanodes to schedule deletion work
+   * @return total number of block for deletion
+   */
+  int computeInvalidateWork(int nodesToProcess) {
+    int numOfNodes = recentInvalidateSets.size();
+    nodesToProcess = Math.min(numOfNodes, nodesToProcess);
+
+    // TODO should using recentInvalidateSets be synchronized?
+    // get an array of the keys
+    ArrayList<String> keyArray =
+      new ArrayList<String>(recentInvalidateSets.keySet());
+
+    // randomly pick up <i>nodesToProcess</i> nodes
+    // and put them at [0, nodesToProcess)
+    int remainingNodes = numOfNodes - nodesToProcess;
+    if (nodesToProcess < remainingNodes) {
+      for(int i=0; i<nodesToProcess; i++) {
+        int keyIndex = r.nextInt(numOfNodes-i)+i;
+        Collections.swap(keyArray, keyIndex, i); // swap to front
+      }
+    } else {
+      for(int i=0; i<remainingNodes; i++) {
+        int keyIndex = r.nextInt(numOfNodes-i);
+        Collections.swap(keyArray, keyIndex, numOfNodes-i-1); // swap to end
+      }
+    }
+
+    int blockCnt = 0;
+    for(int nodeCnt = 0; nodeCnt < nodesToProcess; nodeCnt++ ) {
+      blockCnt += invalidateWorkForOneNode(keyArray.get(nodeCnt));
+    }
+    return blockCnt;
+  }
+
+  /**
+   * Scan blocks in {@link #neededReplications} and assign replication
+   * work to data-nodes they belong to.
+   *
+   * The number of process blocks equals either twice the number of live
+   * data-nodes or the number of under-replicated blocks whichever is less.
+   *
+   * @return number of blocks scheduled for replication during this iteration.
+   */
+  int computeReplicationWork(int blocksToProcess) throws IOException {
+    // Choose the blocks to be replicated
+    List<List<Block>> blocksToReplicate =
+      chooseUnderReplicatedBlocks(blocksToProcess);
+
+    // replicate blocks
+    int scheduledReplicationCount = 0;
+    for (int i=0; i<blocksToReplicate.size(); i++) {
+      for(Block block : blocksToReplicate.get(i)) {
+        if (computeReplicationWorkForBlock(block, i)) {
+          scheduledReplicationCount++;
+        }
+      }
+    }
+    return scheduledReplicationCount;
+  }
+
+  /**
+   * Get a list of block lists to be replicated The index of block lists
+   * represents the
+   *
+   * @param blocksToProcess
+   * @return Return a list of block lists to be replicated. The block list index
+   *         represents its replication priority.
+   */
+  private List<List<Block>> chooseUnderReplicatedBlocks(int blocksToProcess) {
+    // initialize data structure for the return value
+    List<List<Block>> blocksToReplicate = new ArrayList<List<Block>>(
+        UnderReplicatedBlocks.LEVEL);
+    synchronized (namesystem) {
+      for (int i = 0; i < UnderReplicatedBlocks.LEVEL; i++) {
+        blocksToReplicate.add(new ArrayList<Block>());
+      }
+
+      synchronized (neededReplications) {
+        if (neededReplications.size() == 0) {
+          missingBlocksInCurIter = 0;
+          missingBlocksInPrevIter = 0;
+          return blocksToReplicate;
+        }
+
+        // Go through all blocks that need replications.
+        BlockIterator neededReplicationsIterator = neededReplications
+            .iterator();
+        // skip to the first unprocessed block, which is at replIndex
+        for (int i = 0; i < replIndex && neededReplicationsIterator.hasNext(); i++) {
+          neededReplicationsIterator.next();
+        }
+        // # of blocks to process equals either twice the number of live
+        // data-nodes or the number of under-replicated blocks whichever is less
+        blocksToProcess = Math.min(blocksToProcess, neededReplications.size());
+
+        for (int blkCnt = 0; blkCnt < blocksToProcess; blkCnt++, replIndex++) {
+          if (!neededReplicationsIterator.hasNext()) {
+            // start from the beginning
+            replIndex = 0;
+            missingBlocksInPrevIter = missingBlocksInCurIter;
+            missingBlocksInCurIter = 0;
+            blocksToProcess = Math.min(blocksToProcess, neededReplications
+                .size());
+            if (blkCnt >= blocksToProcess)
+              break;
+            neededReplicationsIterator = neededReplications.iterator();
+            assert neededReplicationsIterator.hasNext() : "neededReplications should not be empty.";
+          }
+
+          Block block = neededReplicationsIterator.next();
+          int priority = neededReplicationsIterator.getPriority();
+          if (priority < 0 || priority >= blocksToReplicate.size()) {
+            FSNamesystem.LOG.warn("Unexpected replication priority: "
+                + priority + " " + block);
+          } else {
+            blocksToReplicate.get(priority).add(block);
+          }
+        } // end for
+      } // end synchronized neededReplication
+    } // end synchronized namesystem
+
+    return blocksToReplicate;
+  }
+
+  /** Replicate a block
+   *
+   * @param block block to be replicated
+   * @param priority a hint of its priority in the neededReplication queue
+   * @return if the block gets replicated or not
+   */
+  private boolean computeReplicationWorkForBlock(Block block, int priority) {
+    int requiredReplication, numEffectiveReplicas;
+    List<DatanodeDescriptor> containingNodes;
+    DatanodeDescriptor srcNode;
+
+    synchronized (namesystem) {
+      synchronized (neededReplications) {
+        // block should belong to a file
+        INodeFile fileINode = blocksMap.getINode(block);
+        // abandoned block or block reopened for append
+        if(fileINode == null || fileINode.isUnderConstruction()) {
+          neededReplications.remove(block, priority); // remove from neededReplications
+          replIndex--;
+          return false;
+        }
+        requiredReplication = fileINode.getReplication();
+
+        // get a source data-node
+        containingNodes = new ArrayList<DatanodeDescriptor>();
+        NumberReplicas numReplicas = new NumberReplicas();
+        srcNode = chooseSourceDatanode(block, containingNodes, numReplicas);
+        if ((numReplicas.liveReplicas() + numReplicas.decommissionedReplicas())
+            <= 0) {
+          missingBlocksInCurIter++;
+        }
+        if(srcNode == null) // block can not be replicated from any node
+          return false;
+
+        // do not schedule more if enough replicas is already pending
+        numEffectiveReplicas = numReplicas.liveReplicas() +
+                                pendingReplications.getNumReplicas(block);
+        if(numEffectiveReplicas >= requiredReplication) {
+          neededReplications.remove(block, priority); // remove from neededReplications
+          replIndex--;
+          NameNode.stateChangeLog.info("BLOCK* "
+              + "Removing block " + block
+              + " from neededReplications as it has enough replicas.");
+          return false;
+        }
+      }
+    }
+
+    // choose replication targets: NOT HOLDING THE GLOBAL LOCK
+    DatanodeDescriptor targets[] = replicator.chooseTarget(
+        requiredReplication - numEffectiveReplicas,
+        srcNode, containingNodes, null, block.getNumBytes());
+    if(targets.length == 0)
+      return false;
+
+    synchronized (namesystem) {
+      synchronized (neededReplications) {
+        // Recheck since global lock was released
+        // block should belong to a file
+        INodeFile fileINode = blocksMap.getINode(block);
+        // abandoned block or block reopened for append
+        if(fileINode == null || fileINode.isUnderConstruction()) {
+          neededReplications.remove(block, priority); // remove from neededReplications
+          replIndex--;
+          return false;
+        }
+        requiredReplication = fileINode.getReplication();
+
+        // do not schedule more if enough replicas is already pending
+        NumberReplicas numReplicas = countNodes(block);
+        numEffectiveReplicas = numReplicas.liveReplicas() +
+        pendingReplications.getNumReplicas(block);
+        if(numEffectiveReplicas >= requiredReplication) {
+          neededReplications.remove(block, priority); // remove from neededReplications
+          replIndex--;
+          NameNode.stateChangeLog.info("BLOCK* "
+              + "Removing block " + block
+              + " from neededReplications as it has enough replicas.");
+          return false;
+        }
+
+        // Add block to the to be replicated list
+        srcNode.addBlockToBeReplicated(block, targets);
+
+        for (DatanodeDescriptor dn : targets) {
+          dn.incBlocksScheduled();
+        }
+
+        // Move the block-replication into a "pending" state.
+        // The reason we use 'pending' is so we can retry
+        // replications that fail after an appropriate amount of time.
+        pendingReplications.add(block, targets.length);
+        NameNode.stateChangeLog.debug(
+            "BLOCK* block " + block
+            + " is moved from neededReplications to pendingReplications");
+
+        // remove from neededReplications
+        if(numEffectiveReplicas + targets.length >= requiredReplication) {
+          neededReplications.remove(block, priority); // remove from neededReplications
+          replIndex--;
+        }
+        if (NameNode.stateChangeLog.isInfoEnabled()) {
+          StringBuffer targetList = new StringBuffer("datanode(s)");
+          for (int k = 0; k < targets.length; k++) {
+            targetList.append(' ');
+            targetList.append(targets[k].getName());
+          }
+          NameNode.stateChangeLog.info(
+                    "BLOCK* ask "
+                    + srcNode.getName() + " to replicate "
+                    + block + " to " + targetList);
+          NameNode.stateChangeLog.debug(
+                    "BLOCK* neededReplications = " + neededReplications.size()
+                    + " pendingReplications = " + pendingReplications.size());
+        }
+      }
+    }
+
+    return true;
+  }
+
+  /**
+   * Parse the data-nodes the block belongs to and choose one,
+   * which will be the replication source.
+   *
+   * We prefer nodes that are in DECOMMISSION_INPROGRESS state to other nodes
+   * since the former do not have write traffic and hence are less busy.
+   * We do not use already decommissioned nodes as a source.
+   * Otherwise we choose a random node among those that did not reach their
+   * replication limit.
+   *
+   * In addition form a list of all nodes containing the block
+   * and calculate its replication numbers.
+   */
+  private DatanodeDescriptor chooseSourceDatanode(
+                                    Block block,
+                                    List<DatanodeDescriptor> containingNodes,
+                                    NumberReplicas numReplicas) {
+    containingNodes.clear();
+    DatanodeDescriptor srcNode = null;
+    int live = 0;
+    int decommissioned = 0;
+    int corrupt = 0;
+    int excess = 0;
+    Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
+    Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
+    while(it.hasNext()) {
+      DatanodeDescriptor node = it.next();
+      Collection<Block> excessBlocks =
+        excessReplicateMap.get(node.getStorageID());
+      if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
+        corrupt++;
+      else if (node.isDecommissionInProgress() || node.isDecommissioned())
+        decommissioned++;
+      else if (excessBlocks != null && excessBlocks.contains(block)) {
+        excess++;
+      } else {
+        live++;
+      }
+      containingNodes.add(node);
+      // Check if this replica is corrupt
+      // If so, do not select the node as src node
+      if ((nodesCorrupt != null) && nodesCorrupt.contains(node))
+        continue;
+      if(node.getNumberOfBlocksToBeReplicated() >= maxReplicationStreams)
+        continue; // already reached replication limit
+      // the block must not be scheduled for removal on srcNode
+      if(excessBlocks != null && excessBlocks.contains(block))
+        continue;
+      // never use already decommissioned nodes
+      if(node.isDecommissioned())
+        continue;
+      // we prefer nodes that are in DECOMMISSION_INPROGRESS state
+      if(node.isDecommissionInProgress() || srcNode == null) {
+        srcNode = node;
+        continue;
+      }
+      if(srcNode.isDecommissionInProgress())
+        continue;
+      // switch to a different node randomly
+      // this to prevent from deterministically selecting the same node even
+      // if the node failed to replicate the block on previous iterations
+      if(r.nextBoolean())
+        srcNode = node;
+    }
+    if(numReplicas != null)
+      numReplicas.initialize(live, decommissioned, corrupt, excess);
+    return srcNode;
+  }
+
+  /**
+   * If there were any replication requests that timed out, reap them
+   * and put them back into the neededReplication queue
+   */
+  void processPendingReplications() {
+    Block[] timedOutItems = pendingReplications.getTimedOutBlocks();
+    if (timedOutItems != null) {
+      synchronized (namesystem) {
+        for (int i = 0; i < timedOutItems.length; i++) {
+          NumberReplicas num = countNodes(timedOutItems[i]);
+          neededReplications.add(timedOutItems[i],
+                                 num.liveReplicas(),
+                                 num.decommissionedReplicas(),
+                                 getReplication(timedOutItems[i]));
+        }
+      }
+      /* If we know the target datanodes where the replication timedout,
+       * we could invoke decBlocksScheduled() on it. Its ok for now.
+       */
+    }
+  }
+
+  /**
+   * The given node is reporting all its blocks.  Use this info to
+   * update the (machine-->blocklist) and (block-->machinelist) tables.
+   */
+  public void processReport(DatanodeDescriptor node,
+                            BlockListAsLongs report) throws IOException {
+    //
+    // Modify the (block-->datanode) map, according to the difference
+    // between the old and new block report.
+    //
+    Collection<Block> toAdd = new LinkedList<Block>();
+    Collection<Block> toRemove = new LinkedList<Block>();
+    Collection<Block> toInvalidate = new LinkedList<Block>();
+    node.reportDiff(blocksMap, report, toAdd, toRemove, toInvalidate);
+
+    for (Block b : toRemove) {
+      removeStoredBlock(b, node);
+    }
+    for (Block b : toAdd) {
+      addStoredBlock(b, node, null);
+    }
+    for (Block b : toInvalidate) {
+      NameNode.stateChangeLog.info("BLOCK* NameSystem.processReport: block "
+          + b + " on " + node.getName() + " size " + b.getNumBytes()
+          + " does not belong to any file.");
+      addToInvalidates(b, node);
+    }
+  }
+
+  /**
+   * Modify (block-->datanode) map. Remove block from set of needed replications
+   * if this takes care of the problem.
+   *
+   * @return the block that is stored in blockMap.
+   */
+  private Block addStoredBlock(Block block, DatanodeDescriptor node,
+      DatanodeDescriptor delNodeHint) {
+    BlockInfo storedBlock = blocksMap.getStoredBlock(block);
+    if (storedBlock == null || storedBlock.getINode() == null) {
+      // If this block does not belong to anyfile, then we are done.
+      NameNode.stateChangeLog.info("BLOCK* NameSystem.addStoredBlock: "
+          + "addStoredBlock request received for " + block + " on "
+          + node.getName() + " size " + block.getNumBytes()
+          + " But it does not belong to any file.");
+      // we could add this block to invalidate set of this datanode.
+      // it will happen in next block report otherwise.
+      return block;
+    }
+
+    // add block to the data-node
+    boolean added = node.addBlock(storedBlock);
+
+    assert storedBlock != null : "Block must be stored by now";
+
+    if (block != storedBlock) {
+      if (block.getNumBytes() >= 0) {
+        long cursize = storedBlock.getNumBytes();
+        if (cursize == 0) {
+          storedBlock.setNumBytes(block.getNumBytes());
+        } else if (cursize != block.getNumBytes()) {
+          FSNamesystem.LOG.warn("Inconsistent size for block " + block
+              + " reported from " + node.getName() + " current size is "
+              + cursize + " reported size is " + block.getNumBytes());
+          try {
+            if (cursize > block.getNumBytes()) {
+              // new replica is smaller in size than existing block.
+              // Mark the new replica as corrupt.
+              FSNamesystem.LOG.warn("Mark new replica "
+                  + block + " from " + node.getName() + " as corrupt "
+                  + "because length is shorter than existing ones");
+              markBlockAsCorrupt(block, node);
+            } else {
+              // new replica is larger in size than existing block.
+              // Mark pre-existing replicas as corrupt.
+              int numNodes = blocksMap.numNodes(block);
+              int count = 0;
+              DatanodeDescriptor nodes[] = new DatanodeDescriptor[numNodes];
+              Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
+              for (; it != null && it.hasNext();) {
+                DatanodeDescriptor dd = it.next();
+                if (!dd.equals(node)) {
+                  nodes[count++] = dd;
+                }
+              }
+              for (int j = 0; j < count; j++) {
+                FSNamesystem.LOG.warn("Mark existing replica "
+                        + block + " from " + node.getName() + " as corrupt "
+                        + "because its length is shorter than the new one");
+                markBlockAsCorrupt(block, nodes[j]);
+              }
+              //
+              // change the size of block in blocksMap
+              //
+              storedBlock = blocksMap.getStoredBlock(block); // extra look up!
+              if (storedBlock == null) {
+                FSNamesystem.LOG.warn("Block " + block + " reported from "
+                    + node.getName()
+                    + " does not exist in blockMap. Surprise! Surprise!");
+              } else {
+                storedBlock.setNumBytes(block.getNumBytes());
+              }
+            }
+          } catch (IOException e) {
+            FSNamesystem.LOG.warn("Error in deleting bad block " + block + e);
+          }
+        }
+
+        // Updated space consumed if required.
+        INodeFile file = (storedBlock != null) ? storedBlock.getINode() : null;
+        long diff = (file == null) ? 0
+            : (file.getPreferredBlockSize() - storedBlock.getNumBytes());
+
+        if (diff > 0 && file.isUnderConstruction()
+            && cursize < storedBlock.getNumBytes()) {
+          try {
+            String path = /* For finding parents */
+            namesystem.leaseManager.findPath((INodeFileUnderConstruction) file);
+            namesystem.dir.updateSpaceConsumed(path, 0, -diff
+                * file.getReplication());
+          } catch (IOException e) {
+            FSNamesystem.LOG
+                .warn("Unexpected exception while updating disk space : "
+                    + e.getMessage());
+          }
+        }
+      }
+      block = storedBlock;
+    }
+    assert storedBlock == block : "Block must be stored by now";
+
+    int curReplicaDelta = 0;
+
+    if (added) {
+      curReplicaDelta = 1;
+      //
+      // At startup time, because too many new blocks come in
+      // they take up lots of space in the log file.
+      // So, we log only when namenode is out of safemode.
+      //
+      if (!namesystem.isInSafeMode()) {
+        NameNode.stateChangeLog.info("BLOCK* NameSystem.addStoredBlock: "
+            + "blockMap updated: " + node.getName() + " is added to " + block
+            + " size " + block.getNumBytes());
+      }
+    } else {
+      NameNode.stateChangeLog.warn("BLOCK* NameSystem.addStoredBlock: "
+          + "Redundant addStoredBlock request received for " + block + " on "
+          + node.getName() + " size " + block.getNumBytes());
+    }
+
+    // filter out containingNodes that are marked for decommission.
+    NumberReplicas num = countNodes(storedBlock);
+    int numLiveReplicas = num.liveReplicas();
+    int numCurrentReplica = numLiveReplicas
+        + pendingReplications.getNumReplicas(block);
+
+    // check whether safe replication is reached for the block
+    namesystem.incrementSafeBlockCount(numCurrentReplica);
+
+    //
+    // if file is being actively written to, then do not check
+    // replication-factor here. It will be checked when the file is closed.
+    //
+    INodeFile fileINode = null;
+    fileINode = storedBlock.getINode();
+    if (fileINode.isUnderConstruction()) {
+      return block;
+    }
+
+    // do not handle mis-replicated blocks during startup
+    if (namesystem.isInSafeMode())
+      return block;
+
+    // handle underReplication/overReplication
+    short fileReplication = fileINode.getReplication();
+    if (numCurrentReplica >= fileReplication) {
+      neededReplications.remove(block, numCurrentReplica,
+          num.decommissionedReplicas, fileReplication);
+    } else {
+      updateNeededReplications(block, curReplicaDelta, 0);
+    }
+    if (numCurrentReplica > fileReplication) {
+      processOverReplicatedBlock(block, fileReplication, node, delNodeHint);
+    }
+    // If the file replication has reached desired value
+    // we can remove any corrupt replicas the block may have
+    int corruptReplicasCount = corruptReplicas.numCorruptReplicas(block);
+    int numCorruptNodes = num.corruptReplicas();
+    if (numCorruptNodes != corruptReplicasCount) {
+      FSNamesystem.LOG.warn("Inconsistent number of corrupt replicas for "
+          + block + "blockMap has " + numCorruptNodes
+          + " but corrupt replicas map has " + corruptReplicasCount);
+    }
+    if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication))
+      invalidateCorruptReplicas(block);
+    return block;
+  }
+
+  /**
+   * Invalidate corrupt replicas.
+   * <p>
+   * This will remove the replicas from the block's location list, add them to
+   * {@link #recentInvalidateSets} so that they could be further deleted from
+   * the respective data-nodes, and remove the block from corruptReplicasMap.
+   * <p>
+   * This method should be called when the block has sufficient number of live
+   * replicas.
+   *
+   * @param blk
+   *          Block whose corrupt replicas need to be invalidated
+   */
+  private void invalidateCorruptReplicas(Block blk) {
+    Collection<DatanodeDescriptor> nodes = corruptReplicas.getNodes(blk);
+    boolean gotException = false;
+    if (nodes == null)
+      return;
+    for (Iterator<DatanodeDescriptor> it = nodes.iterator(); it.hasNext();) {
+      DatanodeDescriptor node = it.next();
+      try {
+        invalidateBlock(blk, node);
+      } catch (IOException e) {
+        NameNode.stateChangeLog.info("NameNode.invalidateCorruptReplicas "
+            + "error in deleting bad block " + blk + " on " + node + e);
+        gotException = true;
+      }
+    }
+    // Remove the block from corruptReplicasMap
+    if (!gotException)
+      corruptReplicas.removeFromCorruptReplicasMap(blk);
+  }
+
+  /**
+   * 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() {
+    long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0;
+    synchronized (namesystem) {
+      neededReplications.clear();
+      for (BlocksMap.BlockInfo block : blocksMap.getBlocks()) {
+        INodeFile fileINode = block.getINode();
+        if (fileINode == null) {
+          // block does not belong to any file
+          nrInvalid++;
+          addToInvalidates(block);
+          continue;
+        }
+        // calculate current replication
+        short expectedReplication = fileINode.getReplication();
+        NumberReplicas num = countNodes(block);
+        int numCurrentReplica = num.liveReplicas();
+        // add to under-replicated queue if need to be
+        if (neededReplications.add(block, numCurrentReplica, num
+            .decommissionedReplicas(), expectedReplication)) {
+          nrUnderReplicated++;
+        }
+
+        if (numCurrentReplica > expectedReplication) {
+          // over-replicated block
+          nrOverReplicated++;
+          processOverReplicatedBlock(block, expectedReplication, null, null);
+        }
+      }
+    }
+    FSNamesystem.LOG.info("Total number of blocks = " + blocksMap.size());
+    FSNamesystem.LOG.info("Number of invalid blocks = " + nrInvalid);
+    FSNamesystem.LOG.info("Number of under-replicated blocks = " + nrUnderReplicated);
+    FSNamesystem.LOG.info("Number of  over-replicated blocks = " + nrOverReplicated);
+  }
+
+  /**
+   * Find how many of the containing nodes are "extra", if any. If there are any
+   * extras, call chooseExcessReplicates() to mark them in the
+   * excessReplicateMap.
+   */
+  void processOverReplicatedBlock(Block block, short replication,
+      DatanodeDescriptor addedNode, DatanodeDescriptor delNodeHint) {
+    if (addedNode == delNodeHint) {
+      delNodeHint = null;
+    }
+    Collection<DatanodeDescriptor> nonExcess = new ArrayList<DatanodeDescriptor>();
+    Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
+        .getNodes(block);
+    for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block); it
+        .hasNext();) {
+      DatanodeDescriptor cur = it.next();
+      Collection<Block> excessBlocks = excessReplicateMap.get(cur
+          .getStorageID());
+      if (excessBlocks == null || !excessBlocks.contains(block)) {
+        if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
+          // exclude corrupt replicas
+          if (corruptNodes == null || !corruptNodes.contains(cur)) {
+            nonExcess.add(cur);
+          }
+        }
+      }
+    }
+    namesystem.chooseExcessReplicates(nonExcess, block, replication, addedNode,
+        delNodeHint);
+  }
+
+  void addToExcessReplicate(DatanodeInfo dn, Block block) {
+    Collection<Block> excessBlocks = excessReplicateMap.get(dn.getStorageID());
+    if (excessBlocks == null) {
+      excessBlocks = new TreeSet<Block>();
+      excessReplicateMap.put(dn.getStorageID(), excessBlocks);
+    }
+    excessBlocks.add(block);
+    NameNode.stateChangeLog.debug("BLOCK* NameSystem.chooseExcessReplicates: "
+        + "(" + dn.getName() + ", " + block
+        + ") is added to excessReplicateMap");
+  }
+
+  /**
+   * Modify (block-->datanode) map. Possibly generate replication tasks, if the
+   * removed block is still valid.
+   */
+  void removeStoredBlock(Block block, DatanodeDescriptor node) {
+    NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
+        + block + " from " + node.getName());
+    synchronized (namesystem) {
+      if (!blocksMap.removeNode(block, node)) {
+        NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
+            + block + " has already been removed from node " + node);
+        return;
+      }
+
+      //
+      // It's possible that the block was removed because of a datanode
+      // failure. If the block is still valid, check if replication is
+      // necessary. In that case, put block on a possibly-will-
+      // be-replicated list.
+      //
+      INode fileINode = blocksMap.getINode(block);
+      if (fileINode != null) {
+        namesystem.decrementSafeBlockCount(block);
+        updateNeededReplications(block, -1, 0);
+      }
+
+      //
+      // We've removed a block from a node, so it's definitely no longer
+      // in "excess" there.
+      //
+      Collection<Block> excessBlocks = excessReplicateMap.get(node
+          .getStorageID());
+      if (excessBlocks != null) {
+        excessBlocks.remove(block);
+        NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
+            + block + " is removed from excessBlocks");
+        if (excessBlocks.size() == 0) {
+          excessReplicateMap.remove(node.getStorageID());
+        }
+      }
+
+      // Remove the replica from corruptReplicas
+      corruptReplicas.removeFromCorruptReplicasMap(block, node);
+    }
+  }
+
+  /**
+   * The given node is reporting that it received a certain block.
+   */
+  void addBlock(DatanodeDescriptor node, Block block, String delHint)
+      throws IOException {
+    // decrement number of blocks scheduled to this datanode.
+    node.decBlocksScheduled();
+
+    // get the deletion hint node
+    DatanodeDescriptor delHintNode = null;
+    if (delHint != null && delHint.length() != 0) {
+      delHintNode = namesystem.getDatanode(delHint);
+      if (delHintNode == null) {
+        NameNode.stateChangeLog.warn("BLOCK* NameSystem.blockReceived: "
+            + block + " is expected to be removed from an unrecorded node "
+            + delHint);
+      }
+    }
+
+    //
+    // Modify the blocks->datanode map and node's map.
+    //
+    pendingReplications.remove(block);
+    addStoredBlock(block, node, delHintNode);
+  }
+
+  /**
+   * Return the number of nodes that are live and decommissioned.
+   */
+  NumberReplicas countNodes(Block b) {
+    int count = 0;
+    int live = 0;
+    int corrupt = 0;
+    int excess = 0;
+    Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(b);
+    Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
+    while (nodeIter.hasNext()) {
+      DatanodeDescriptor node = nodeIter.next();
+      if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) {
+        corrupt++;
+      } else if (node.isDecommissionInProgress() || node.isDecommissioned()) {
+        count++;
+      } else {
+        Collection<Block> blocksExcess = excessReplicateMap.get(node
+            .getStorageID());
+        if (blocksExcess != null && blocksExcess.contains(b)) {
+          excess++;
+        } else {
+          live++;
+        }
+      }
+    }
+    return new NumberReplicas(live, count, corrupt, excess);
+  }
+
+  /**
+   * 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) {
+    boolean status = false;
+    for(final Iterator<Block> i = srcNode.getBlockIterator(); i.hasNext(); ) {
+      final Block block = i.next();
+      INode fileINode = blocksMap.getINode(block);
+
+      if (fileINode != null) {
+        NumberReplicas num = countNodes(block);
+        int curReplicas = num.liveReplicas();
+        int curExpectedReplicas = getReplication(block);
+        if (curExpectedReplicas > curReplicas) {
+          status = true;
+          if (!neededReplications.contains(block) &&
+            pendingReplications.getNumReplicas(block) == 0) {
+            //
+            // These blocks have been reported from the datanode
+            // after the startDecommission method has been executed. These
+            // blocks were in flight when the decommission was started.
+            //
+            neededReplications.add(block,
+                                   curReplicas,
+                                   num.decommissionedReplicas(),
+                                   curExpectedReplicas);
+          }
+        }
+      }
+    }
+    return status;
+  }
+
+  int getActiveBlockCount() {
+    int activeBlocks = blocksMap.size();
+    for(Iterator<Collection<Block>> it =
+          recentInvalidateSets.values().iterator(); it.hasNext();) {
+      activeBlocks -= it.next().size();
+    }
+    return activeBlocks;
+  }
+
+  DatanodeDescriptor[] getNodes(Block block) {
+    DatanodeDescriptor[] nodes =
+      new DatanodeDescriptor[blocksMap.numNodes(block)];
+    Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
+    for (int i = 0; it != null && it.hasNext(); i++) {
+      nodes[i] = it.next();
+    }
+    return nodes;
+  }
+
+  int getTotalBlocks() {
+    return blocksMap.size();
+  }
+
+  void removeBlock(Block block) {
+    blocksMap.removeINode(block);
+    corruptReplicas.removeFromCorruptReplicasMap(block);
+    addToInvalidates(block);
+  }
+
+  BlockInfo getStoredBlock(Block block) {
+    return blocksMap.getStoredBlock(block);
+  }
+
+  /* updates a block in under replication queue */
+  void updateNeededReplications(Block block, int curReplicasDelta,
+      int expectedReplicasDelta) {
+    synchronized (namesystem) {
+      NumberReplicas repl = countNodes(block);
+      int curExpectedReplicas = getReplication(block);
+      neededReplications.update(block, repl.liveReplicas(), repl
+          .decommissionedReplicas(), curExpectedReplicas, curReplicasDelta,
+          expectedReplicasDelta);
+    }
+  }
+
+  void checkReplication(Block block, int numExpectedReplicas) {
+    // filter out containingNodes that are marked for decommission.
+    NumberReplicas number = countNodes(block);
+    if (number.liveReplicas() < numExpectedReplicas) {
+      neededReplications.add(block,
+                             number.liveReplicas(),
+                             number.decommissionedReplicas,
+                             numExpectedReplicas);
+    }
+  }
+
+  /* get replication factor of a block */
+  private int getReplication(Block block) {
+    INodeFile fileINode = blocksMap.getINode(block);
+    if (fileINode == null) { // block does not belong to any file
+      return 0;
+    }
+    assert !fileINode.isDirectory() : "Block cannot belong to a directory.";
+    return fileINode.getReplication();
+  }
+
+  /**
+   * Remove a datanode from the invalidatesSet
+   * @param n datanode
+   */
+  void removeFromInvalidates(DatanodeInfo n) {
+    recentInvalidateSets.remove(n.getStorageID());
+  }
+
+  /**
+   * Get blocks to invalidate for <i>nodeId</i>
+   * in {@link #recentInvalidateSets}.
+   *
+   * @return number of blocks scheduled for removal during this iteration.
+   */
+  private int invalidateWorkForOneNode(String nodeId) {
+    synchronized (namesystem) {
+      // blocks should not be replicated or removed if safe mode is on
+      if (namesystem.isInSafeMode())
+        return 0;
+      // get blocks to invalidate for the nodeId
+      assert nodeId != null;
+      DatanodeDescriptor dn = namesystem.getDatanode(nodeId);
+      if (dn == null) {
+        recentInvalidateSets.remove(nodeId);
+        return 0;
+      }
+
+      Collection<Block> invalidateSet = recentInvalidateSets.get(nodeId);
+      if (invalidateSet == null)
+        return 0;
+
+      ArrayList<Block> blocksToInvalidate = new ArrayList<Block>(
+          namesystem.blockInvalidateLimit);
+
+      // # blocks that can be sent in one message is limited
+      Iterator<Block> it = invalidateSet.iterator();
+      for (int blkCount = 0; blkCount < namesystem.blockInvalidateLimit
+          && it.hasNext(); blkCount++) {
+        blocksToInvalidate.add(it.next());
+        it.remove();
+      }
+
+      // If we send everything in this message, remove this node entry
+      if (!it.hasNext())
+        recentInvalidateSets.remove(nodeId);
+
+      dn.addBlocksToBeInvalidated(blocksToInvalidate);
+
+      if (NameNode.stateChangeLog.isInfoEnabled()) {
+        StringBuffer blockList = new StringBuffer();
+        for (Block blk : blocksToInvalidate) {
+          blockList.append(' ');
+          blockList.append(blk);
+        }
+        NameNode.stateChangeLog.info("BLOCK* ask " + dn.getName()
+            + " to delete " + blockList);
+      }
+      return blocksToInvalidate.size();
+    }
+  }
+
+  long getMissingBlocksCount() {
+    // not locking
+    return Math.max(missingBlocksInPrevIter, missingBlocksInCurIter);
+  }
+
+  BlockInfo addINode(Block block, INodeFile iNode) {
+    return blocksMap.addINode(block, iNode);
+  }
+
+  void removeINode(Block block) {
+    blocksMap.removeINode(block);
+  }
+
+  INodeFile getINode(Block b) {
+    return blocksMap.getINode(b);
+  }
+
+  void removeFromCorruptReplicasMap(Block block) {
+    corruptReplicas.removeFromCorruptReplicasMap(block);
+  }
+
+  int numCorruptReplicas(Block block) {
+    return corruptReplicas.numCorruptReplicas(block);
+  }
+
+  void removeBlockFromMap(BlockInfo blockInfo) {
+    blocksMap.removeBlock(blockInfo);
+  }
+}

+ 424 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java

@@ -0,0 +1,424 @@
+/**
+ * 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 java.util.*;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/**
+ * This class maintains the map from a block to its metadata.
+ * block's metadata currently includes INode it belongs to and
+ * the datanodes that store the block.
+ */
+class BlocksMap {
+        
+  /**
+   * Internal class for block metadata.
+   */
+  static class BlockInfo extends Block {
+    private INodeFile          inode;
+
+    /**
+     * This array contains triplets of references.
+     * For each i-th data-node the block belongs to
+     * triplets[3*i] is the reference to the DatanodeDescriptor
+     * and triplets[3*i+1] and triplets[3*i+2] are references 
+     * to the previous and the next blocks, respectively, in the 
+     * list of blocks belonging to this data-node.
+     */
+    private Object[] triplets;
+
+    BlockInfo(Block blk, int replication) {
+      super(blk);
+      this.triplets = new Object[3*replication];
+      this.inode = null;
+    }
+
+    INodeFile getINode() {
+      return inode;
+    }
+
+    DatanodeDescriptor getDatanode(int index) {
+      assert this.triplets != null : "BlockInfo is not initialized";
+      assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
+      DatanodeDescriptor node = (DatanodeDescriptor)triplets[index*3];
+      assert node == null || 
+          DatanodeDescriptor.class.getName().equals(node.getClass().getName()) : 
+                "DatanodeDescriptor is expected at " + index*3;
+      return node;
+    }
+
+    BlockInfo getPrevious(int index) {
+      assert this.triplets != null : "BlockInfo is not initialized";
+      assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
+      BlockInfo info = (BlockInfo)triplets[index*3+1];
+      assert info == null || 
+          BlockInfo.class.getName().equals(info.getClass().getName()) : 
+                "BlockInfo is expected at " + index*3;
+      return info;
+    }
+
+    BlockInfo getNext(int index) {
+      assert this.triplets != null : "BlockInfo is not initialized";
+      assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
+      BlockInfo info = (BlockInfo)triplets[index*3+2];
+      assert info == null || 
+          BlockInfo.class.getName().equals(info.getClass().getName()) : 
+                "BlockInfo is expected at " + index*3;
+      return info;
+    }
+
+    void setDatanode(int index, DatanodeDescriptor node) {
+      assert this.triplets != null : "BlockInfo is not initialized";
+      assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
+      triplets[index*3] = node;
+    }
+
+    void setPrevious(int index, BlockInfo to) {
+      assert this.triplets != null : "BlockInfo is not initialized";
+      assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
+      triplets[index*3+1] = to;
+    }
+
+    void setNext(int index, BlockInfo to) {
+      assert this.triplets != null : "BlockInfo is not initialized";
+      assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
+      triplets[index*3+2] = to;
+    }
+
+    private int getCapacity() {
+      assert this.triplets != null : "BlockInfo is not initialized";
+      assert triplets.length % 3 == 0 : "Malformed BlockInfo";
+      return triplets.length / 3;
+    }
+
+    /**
+     * Ensure that there is enough  space to include num more triplets.
+     *      * @return first free triplet index.
+     */
+    private int ensureCapacity(int num) {
+      assert this.triplets != null : "BlockInfo is not initialized";
+      int last = numNodes();
+      if(triplets.length >= (last+num)*3)
+        return last;
+      /* Not enough space left. Create a new array. Should normally 
+       * happen only when replication is manually increased by the user. */
+      Object[] old = triplets;
+      triplets = new Object[(last+num)*3];
+      for(int i=0; i < last*3; i++) {
+        triplets[i] = old[i];
+      }
+      return last;
+    }
+
+    /**
+     * Count the number of data-nodes the block belongs to.
+     */
+    int numNodes() {
+      assert this.triplets != null : "BlockInfo is not initialized";
+      assert triplets.length % 3 == 0 : "Malformed BlockInfo";
+      for(int idx = getCapacity()-1; idx >= 0; idx--) {
+        if(getDatanode(idx) != null)
+          return idx+1;
+      }
+      return 0;
+    }
+
+    /**
+     * Add data-node this block belongs to.
+     */
+    boolean addNode(DatanodeDescriptor node) {
+      if(findDatanode(node) >= 0) // the node is already there
+        return false;
+      // find the last null node
+      int lastNode = ensureCapacity(1);
+      setDatanode(lastNode, node);
+      setNext(lastNode, null);
+      setPrevious(lastNode, null);
+      return true;
+    }
+
+    /**
+     * Remove data-node from the block.
+     */
+    boolean removeNode(DatanodeDescriptor node) {
+      int dnIndex = findDatanode(node);
+      if(dnIndex < 0) // the node is not found
+        return false;
+      assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
+        "Block is still in the list and must be removed first.";
+      // find the last not null node
+      int lastNode = numNodes()-1; 
+      // replace current node triplet by the lastNode one 
+      setDatanode(dnIndex, getDatanode(lastNode));
+      setNext(dnIndex, getNext(lastNode)); 
+      setPrevious(dnIndex, getPrevious(lastNode)); 
+      // set the last triplet to null
+      setDatanode(lastNode, null);
+      setNext(lastNode, null); 
+      setPrevious(lastNode, null); 
+      return true;
+    }
+
+    /**
+     * Find specified DatanodeDescriptor.
+     * @param dn
+     * @return index or -1 if not found.
+     */
+    int findDatanode(DatanodeDescriptor dn) {
+      int len = getCapacity();
+      for(int idx = 0; idx < len; idx++) {
+        DatanodeDescriptor cur = getDatanode(idx);
+        if(cur == dn)
+          return idx;
+        if(cur == null)
+          break;
+      }
+      return -1;
+    }
+
+    /**
+     * Insert this block into the head of the list of blocks 
+     * related to the specified DatanodeDescriptor.
+     * 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) {
+      int dnIndex = this.findDatanode(dn);
+      assert dnIndex >= 0 : "Data node is not found: current";
+      assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
+              "Block is already in the list and cannot be inserted.";
+      this.setPrevious(dnIndex, null);
+      this.setNext(dnIndex, head);
+      if(head != null)
+        head.setPrevious(head.findDatanode(dn), this);
+      return this;
+    }
+
+    /**
+     * Remove this block from the list of blocks 
+     * related to the specified DatanodeDescriptor.
+     * If this block is the head of the list then return the next block as 
+     * the new head.
+     * @return the new head of the list or null if the list becomes
+     * empy after deletion.
+     */
+    BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
+      if(head == null)
+        return null;
+      int dnIndex = this.findDatanode(dn);
+      if(dnIndex < 0) // this block is not on the data-node list
+        return head;
+
+      BlockInfo next = this.getNext(dnIndex);
+      BlockInfo prev = this.getPrevious(dnIndex);
+      this.setNext(dnIndex, null);
+      this.setPrevious(dnIndex, null);
+      if(prev != null)
+        prev.setNext(prev.findDatanode(dn), next);
+      if(next != null)
+        next.setPrevious(next.findDatanode(dn), prev);
+      if(this == head)  // removing the head
+        head = next;
+      return head;
+    }
+
+    int listCount(DatanodeDescriptor dn) {
+      int count = 0;
+      for(BlockInfo cur = this; cur != null;
+            cur = cur.getNext(cur.findDatanode(dn)))
+        count++;
+      return count;
+    }
+
+    boolean listIsConsistent(DatanodeDescriptor dn) {
+      // going forward
+      int count = 0;
+      BlockInfo next, nextPrev;
+      BlockInfo cur = this;
+      while(cur != null) {
+        next = cur.getNext(cur.findDatanode(dn));
+        if(next != null) {
+          nextPrev = next.getPrevious(next.findDatanode(dn));
+          if(cur != nextPrev) {
+            System.out.println("Inconsistent list: cur->next->prev != cur");
+            return false;
+          }
+        }
+        cur = next;
+        count++;
+      }
+      return true;
+    }
+  }
+
+  private static class NodeIterator implements Iterator<DatanodeDescriptor> {
+    private BlockInfo blockInfo;
+    private int nextIdx = 0;
+      
+    NodeIterator(BlockInfo blkInfo) {
+      this.blockInfo = blkInfo;
+    }
+
+    public boolean hasNext() {
+      return blockInfo != null && nextIdx < blockInfo.getCapacity()
+              && blockInfo.getDatanode(nextIdx) != null;
+    }
+
+    public DatanodeDescriptor next() {
+      return blockInfo.getDatanode(nextIdx++);
+    }
+
+    public void remove()  {
+      throw new UnsupportedOperationException("Sorry. can't remove.");
+    }
+  }
+
+  private Map<Block, BlockInfo> map = new HashMap<Block, BlockInfo>();
+
+  /**
+   * Add BlockInfo if mapping does not exist.
+   */
+  private BlockInfo checkBlockInfo(Block b, int replication) {
+    BlockInfo info = map.get(b);
+    if (info == null) {
+      info = new BlockInfo(b, replication);
+      map.put(info, info);
+    }
+    return info;
+  }
+
+  INodeFile getINode(Block b) {
+    BlockInfo info = map.get(b);
+    return (info != null) ? info.inode : null;
+  }
+
+  /**
+   * Add block b belonging to the specified file inode to the map.
+   */
+  BlockInfo addINode(Block b, INodeFile iNode) {
+    BlockInfo info = checkBlockInfo(b, iNode.getReplication());
+    info.inode = iNode;
+    return info;
+  }
+
+  /**
+   * Remove INode reference from block b.
+   * If it does not belong to any file and data-nodes,
+   * then remove the block from the block map.
+   */
+  void removeINode(Block b) {
+    BlockInfo info = map.get(b);
+    if (info != null) {
+      info.inode = null;
+      if (info.getDatanode(0) == null) {  // no datanodes left
+        map.remove(b);  // remove block from the map
+      }
+    }
+  }
+
+  /**
+   * Remove the block from the block map;
+   * remove it from all data-node lists it belongs to;
+   * and remove all data-node locations associated with the block.
+   */
+  void removeBlock(BlockInfo blockInfo) {
+    if (blockInfo == null)
+      return;
+    blockInfo.inode = null;
+    for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
+      DatanodeDescriptor dn = blockInfo.getDatanode(idx);
+      dn.removeBlock(blockInfo); // remove from the list and wipe the location
+    }
+    map.remove(blockInfo);  // remove block from the map
+  }
+
+  /** Returns the block object it it exists in the map. */
+  BlockInfo getStoredBlock(Block b) {
+    return map.get(b);
+  }
+
+  /** Returned Iterator does not support. */
+  Iterator<DatanodeDescriptor> nodeIterator(Block b) {
+    return new NodeIterator(map.get(b));
+  }
+
+  /** counts number of containing nodes. Better than using iterator. */
+  int numNodes(Block b) {
+    BlockInfo info = map.get(b);
+    return info == null ? 0 : info.numNodes();
+  }
+
+  /** returns true if the node does not already exists and is added.
+   * false if the node already exists.*/
+  boolean addNode(Block b, DatanodeDescriptor node, int replication) {
+    // insert into the map if not there yet
+    BlockInfo info = checkBlockInfo(b, replication);
+    // add block to the data-node list and the node to the block info
+    return node.addBlock(info);
+  }
+
+  /**
+   * Remove data-node reference from the block.
+   * Remove the block from the block map
+   * only if it does not belong to any file and data-nodes.
+   */
+  boolean removeNode(Block b, DatanodeDescriptor node) {
+    BlockInfo info = map.get(b);
+    if (info == null)
+      return false;
+
+    // remove block from the data-node list and the node from the block info
+    boolean removed = node.removeBlock(info);
+
+    if (info.getDatanode(0) == null     // no datanodes left
+              && info.inode == null) {  // does not belong to a file
+      map.remove(b);  // remove block from the map
+    }
+    return removed;
+  }
+
+  int size() {
+    return map.size();
+  }
+
+  Collection<BlockInfo> getBlocks() {
+    return map.values();
+  }
+  /**
+   * Check if the block exists in map
+   */
+  boolean contains(Block block) {
+    return map.containsKey(block);
+  }
+  
+  /**
+   * Check if the replica at the given datanode exists in map
+   */
+  boolean contains(Block block, DatanodeDescriptor datanode) {
+    BlockInfo info = map.get(block);
+    if (info == null)
+      return false;
+    
+    if (-1 == info.findDatanode(datanode))
+      return false;
+    
+    return true;
+  }
+}

+ 117 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java

@@ -0,0 +1,117 @@
+/**
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * A unique signature intended to identify checkpoint transactions.
+ */
+public class CheckpointSignature extends StorageInfo 
+                      implements WritableComparable<CheckpointSignature> {
+  private static final String FIELD_SEPARATOR = ":";
+  long editsTime = -1L;
+  long checkpointTime = -1L;
+
+  public CheckpointSignature() {}
+
+  CheckpointSignature(FSImage fsImage) {
+    super(fsImage);
+    editsTime = fsImage.getEditLog().getFsEditTime();
+    checkpointTime = fsImage.getCheckpointTime();
+  }
+
+  CheckpointSignature(String str) {
+    String[] fields = str.split(FIELD_SEPARATOR);
+    assert fields.length == 5 : "Must be 5 fields in CheckpointSignature";
+    layoutVersion = Integer.valueOf(fields[0]);
+    namespaceID = Integer.valueOf(fields[1]);
+    cTime = Long.valueOf(fields[2]);
+    editsTime = Long.valueOf(fields[3]);
+    checkpointTime = Long.valueOf(fields[4]);
+  }
+
+  public String toString() {
+    return String.valueOf(layoutVersion) + FIELD_SEPARATOR
+         + String.valueOf(namespaceID) + FIELD_SEPARATOR
+         + String.valueOf(cTime) + FIELD_SEPARATOR
+         + String.valueOf(editsTime) + FIELD_SEPARATOR
+         + String.valueOf(checkpointTime);
+  }
+
+  void validateStorageInfo(FSImage si) throws IOException {
+    if(layoutVersion != si.layoutVersion
+        || namespaceID != si.namespaceID || cTime != si.cTime
+        || checkpointTime != si.checkpointTime) {
+      // checkpointTime can change when the image is saved - do not compare
+      throw new IOException("Inconsistent checkpoint fields.\n"
+          + "LV = " + layoutVersion + " namespaceID = " + namespaceID
+          + " cTime = " + cTime + "; checkpointTime = " + checkpointTime 
+          + ".\nExpecting respectively: "
+          + si.layoutVersion + "; " + si.namespaceID + "; " + si.cTime
+          + "; " + si.checkpointTime);
+    }
+  }
+
+  //
+  // Comparable interface
+  //
+  public int compareTo(CheckpointSignature o) {
+    return 
+      (layoutVersion < o.layoutVersion) ? -1 : 
+                  (layoutVersion > o.layoutVersion) ? 1 :
+      (namespaceID < o.namespaceID) ? -1 : (namespaceID > o.namespaceID) ? 1 :
+      (cTime < o.cTime) ? -1 : (cTime > o.cTime) ? 1 :
+      (editsTime < o.editsTime) ? -1 : (editsTime > o.editsTime) ? 1 :
+      (checkpointTime < o.checkpointTime) ? -1 : 
+                  (checkpointTime > o.checkpointTime) ? 1 : 0;
+  }
+
+  public boolean equals(Object o) {
+    if (!(o instanceof CheckpointSignature)) {
+      return false;
+    }
+    return compareTo((CheckpointSignature)o) == 0;
+  }
+
+  public int hashCode() {
+    return layoutVersion ^ namespaceID ^
+            (int)(cTime ^ editsTime ^ checkpointTime);
+  }
+
+  /////////////////////////////////////////////////
+  // Writable
+  /////////////////////////////////////////////////
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    out.writeLong(editsTime);
+    out.writeLong(checkpointTime);
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    editsTime = in.readLong();
+    checkpointTime = in.readLong();
+  }
+}

+ 255 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java

@@ -0,0 +1,255 @@
+/**
+ * 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 java.io.IOException;
+import java.io.File;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.namenode.FSImage.CheckpointStates;
+import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
+import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeFile;
+import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.HttpServer;
+
+/**
+ * The Checkpointer is responsible for supporting periodic checkpoints 
+ * of the HDFS metadata.
+ *
+ * The Checkpointer is a daemon that periodically wakes up
+ * up (determined by the schedule specified in the configuration),
+ * triggers a periodic checkpoint and then goes back to sleep.
+ * 
+ * The start of a checkpoint is triggered by one of the two factors:
+ * (1) time or (2) the size of the edits file.
+ */
+class Checkpointer implements Runnable {
+  public static final Log LOG = 
+    LogFactory.getLog(Checkpointer.class.getName());
+
+  private BackupNode backupNode;
+  volatile boolean shouldRun;
+  private long checkpointPeriod;	// in seconds
+  private long checkpointSize;    // size (in MB) of current Edit Log
+
+  private BackupStorage getFSImage() {
+    return (BackupStorage)backupNode.getFSImage();
+  }
+
+  private NamenodeProtocol getNamenode(){
+    return backupNode.namenode;
+  }
+
+  /**
+   * Create a connection to the primary namenode.
+   */
+  Checkpointer(Configuration conf, BackupNode bnNode)  throws IOException {
+    this.backupNode = bnNode;
+    try {
+      initialize(conf);
+    } catch(IOException e) {
+      shutdown();
+      throw e;
+    }
+  }
+
+  /**
+   * Initialize checkpoint.
+   */
+  private void initialize(Configuration conf) throws IOException {
+    // Create connection to the namenode.
+    shouldRun = true;
+
+    // Initialize other scheduling parameters from the configuration
+    checkpointPeriod = conf.getLong("fs.checkpoint.period", 3600);
+    checkpointSize = conf.getLong("fs.checkpoint.size", 4194304);
+
+    HttpServer httpServer = backupNode.httpServer;
+    httpServer.setAttribute("name.system.image", getFSImage());
+    httpServer.setAttribute("name.conf", conf);
+    httpServer.addServlet("getimage", "/getimage", GetImageServlet.class);
+
+    LOG.info("Checkpoint Period : " + checkpointPeriod + " secs " +
+             "(" + checkpointPeriod/60 + " min)");
+    LOG.info("Log Size Trigger  : " + checkpointSize + " bytes " +
+             "(" + checkpointSize/1024 + " KB)");
+  }
+
+  /**
+   * Shut down the checkpointer.
+   */
+  void shutdown() {
+    shouldRun = false;
+    backupNode.stop();
+  }
+
+  //
+  // The main work loop
+  //
+  public void run() {
+    // Check the size of the edit log once every 5 minutes.
+    long periodMSec = 5 * 60;   // 5 minutes
+    if(checkpointPeriod < periodMSec) {
+      periodMSec = checkpointPeriod;
+    }
+    periodMSec *= 1000;
+
+    long lastCheckpointTime = 0;
+    if(!backupNode.shouldCheckpointAtStartup())
+      lastCheckpointTime = FSNamesystem.now();
+    while(shouldRun) {
+      try {
+        long now = FSNamesystem.now();
+        boolean shouldCheckpoint = false;
+        if(now >= lastCheckpointTime + periodMSec) {
+          shouldCheckpoint = true;
+        } else {
+          long size = getJournalSize();
+          if(size >= checkpointSize)
+            shouldCheckpoint = true;
+        }
+        if(shouldCheckpoint) {
+          doCheckpoint();
+          lastCheckpointTime = now;
+        }
+      } catch(IOException e) {
+        LOG.error("Exception in doCheckpoint: ", e);
+      } catch(Throwable e) {
+        LOG.error("Throwable Exception in doCheckpoint: ", e);
+        Runtime.getRuntime().exit(-1);
+      }
+      try {
+        Thread.sleep(periodMSec);
+      } catch(InterruptedException ie) {
+        // do nothing
+      }
+    }
+  }
+
+  private long getJournalSize() throws IOException {
+    // If BACKUP node has been loaded
+    // get edits size from the local file. ACTIVE has the same.
+    if(backupNode.isRole(NamenodeRole.BACKUP)
+        && getFSImage().getEditLog().isOpen())
+      return backupNode.journalSize();
+    // Go to the ACTIVE node for its size
+    return getNamenode().journalSize(backupNode.getRegistration());
+  }
+
+  /**
+   * Download <code>fsimage</code> and <code>edits</code>
+   * files from the remote name-node.
+   */
+  private void downloadCheckpoint(CheckpointSignature sig) throws IOException {
+    // Retrieve image file
+    String fileid = "getimage=1";
+    Collection<File> list = getFSImage().getFiles(NameNodeFile.IMAGE,
+        NameNodeDirType.IMAGE);
+    File[] files = list.toArray(new File[list.size()]);
+    assert files.length > 0 : "No checkpoint targets.";
+    String nnHttpAddr = backupNode.nnHttpAddress;
+    TransferFsImage.getFileClient(nnHttpAddr, fileid, files);
+    LOG.info("Downloaded file " + files[0].getName() + " size " +
+             files[0].length() + " bytes.");
+
+    // Retrieve edits file
+    fileid = "getedit=1";
+    list = getFSImage().getFiles(NameNodeFile.EDITS, NameNodeDirType.EDITS);
+    files = list.toArray(new File[list.size()]);
+    assert files.length > 0 : "No checkpoint targets.";
+    TransferFsImage.getFileClient(nnHttpAddr, fileid, files);
+    LOG.info("Downloaded file " + files[0].getName() + " size " +
+        files[0].length() + " bytes.");
+  }
+
+  /**
+   * Copy the new image into remote name-node.
+   */
+  private void uploadCheckpoint(CheckpointSignature sig) throws IOException {
+    InetSocketAddress httpSocAddr = backupNode.getHttpAddress();
+    int httpPort = httpSocAddr.getPort();
+    String fileid = "putimage=1&port=" + httpPort +
+      "&machine=" +
+      InetAddress.getLocalHost().getHostAddress() +
+      "&token=" + sig.toString();
+    LOG.info("Posted URL " + backupNode.nnHttpAddress + fileid);
+    TransferFsImage.getFileClient(backupNode.nnHttpAddress, fileid, (File[])null);
+  }
+
+  /**
+   * Create a new checkpoint
+   */
+  void doCheckpoint() throws IOException {
+    long startTime = FSNamesystem.now();
+    NamenodeCommand cmd = 
+      getNamenode().startCheckpoint(backupNode.getRegistration());
+    CheckpointCommand cpCmd = null;
+    switch(cmd.getAction()) {
+      case NamenodeProtocol.ACT_SHUTDOWN:
+        shutdown();
+        throw new IOException("Name-node " + backupNode.nnRpcAddress
+                                           + " requested shutdown.");
+      case NamenodeProtocol.ACT_CHECKPOINT:
+        cpCmd = (CheckpointCommand)cmd;
+        break;
+      default:
+        throw new IOException("Unsupported NamenodeCommand: "+cmd.getAction());
+    }
+
+    CheckpointSignature sig = cpCmd.getSignature();
+    assert FSConstants.LAYOUT_VERSION == sig.getLayoutVersion() :
+      "Signature should have current layout version. Expected: "
+      + FSConstants.LAYOUT_VERSION + " actual "+ sig.getLayoutVersion();
+    assert !backupNode.isRole(NamenodeRole.CHECKPOINT) ||
+      cpCmd.isImageObsolete() : "checkpoint node should always download image.";
+    backupNode.setCheckpointState(CheckpointStates.UPLOAD_START);
+    if(cpCmd.isImageObsolete()) {
+      // First reset storage on disk and memory state
+      backupNode.resetNamespace();
+      downloadCheckpoint(sig);
+    }
+
+    BackupStorage bnImage = getFSImage();
+    bnImage.loadCheckpoint(sig);
+    sig.validateStorageInfo(bnImage);
+    bnImage.saveCheckpoint();
+
+    if(cpCmd.needToReturnImage())
+      uploadCheckpoint(sig);
+
+    getNamenode().endCheckpoint(backupNode.getRegistration(), sig);
+
+    bnImage.convergeJournalSpool();
+    backupNode.setRegistration(); // keep registration up to date
+    if(backupNode.isRole(NamenodeRole.CHECKPOINT))
+        getFSImage().getEditLog().close();
+    LOG.info("Checkpoint completed in "
+        + (FSNamesystem.now() - startTime)/1000 + " seconds."
+        +	" New Image Size: " + bnImage.getFsImageName().length());
+  }
+}

+ 137 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/CorruptReplicasMap.java

@@ -0,0 +1,137 @@
+/**
+ * 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 org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.ipc.Server;
+
+import java.util.*;
+
+/**
+ * Stores information about all corrupt blocks in the File System.
+ * A Block is considered corrupt only if all of its replicas are
+ * corrupt. While reporting replicas of a Block, we hide any corrupt
+ * copies. These copies are removed once Block is found to have 
+ * expected number of good replicas.
+ * Mapping: Block -> TreeSet<DatanodeDescriptor> 
+ */
+
+public class CorruptReplicasMap{
+
+  private Map<Block, Collection<DatanodeDescriptor>> corruptReplicasMap =
+    new TreeMap<Block, Collection<DatanodeDescriptor>>();
+  
+  /**
+   * Mark the block belonging to datanode as corrupt.
+   *
+   * @param blk Block to be added to CorruptReplicasMap
+   * @param dn DatanodeDescriptor which holds the corrupt replica
+   */
+  public void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn) {
+    Collection<DatanodeDescriptor> nodes = getNodes(blk);
+    if (nodes == null) {
+      nodes = new TreeSet<DatanodeDescriptor>();
+      corruptReplicasMap.put(blk, nodes);
+    }
+    if (!nodes.contains(dn)) {
+      nodes.add(dn);
+      NameNode.stateChangeLog.info("BLOCK NameSystem.addToCorruptReplicasMap: "+
+                                   blk.getBlockName() +
+                                   " added as corrupt on " + dn.getName() +
+                                   " by " + Server.getRemoteIp());
+    } else {
+      NameNode.stateChangeLog.info("BLOCK NameSystem.addToCorruptReplicasMap: "+
+                                   "duplicate requested for " + 
+                                   blk.getBlockName() + " to add as corrupt " +
+                                   "on " + dn.getName() +
+                                   " by " + Server.getRemoteIp());
+    }
+    if (NameNode.getNameNodeMetrics() != null) {
+      NameNode.getNameNodeMetrics().numBlocksCorrupted.set(
+        corruptReplicasMap.size());
+    }
+  }
+
+  /**
+   * Remove Block from CorruptBlocksMap
+   *
+   * @param blk Block to be removed
+   */
+  void removeFromCorruptReplicasMap(Block blk) {
+    if (corruptReplicasMap != null) {
+      corruptReplicasMap.remove(blk);
+      if (NameNode.getNameNodeMetrics() != null) {
+        NameNode.getNameNodeMetrics().numBlocksCorrupted.set(
+          corruptReplicasMap.size());
+      }
+    }
+  }
+
+  /**
+   * Remove the block at the given datanode from CorruptBlockMap
+   * @param blk block to be removed
+   * @param datanode datanode where the block is located
+   * @return true if the removal is successful; 
+             false if the replica is not in the map
+   */ 
+  boolean removeFromCorruptReplicasMap(Block blk, DatanodeDescriptor datanode) {
+    Collection<DatanodeDescriptor> datanodes = corruptReplicasMap.get(blk);
+    if (datanodes==null)
+      return false;
+    if (datanodes.remove(datanode)) { // remove the replicas
+      if (datanodes.isEmpty()) {
+        // remove the block if there is no more corrupted replicas
+        corruptReplicasMap.remove(blk);
+      }
+      return true;
+    }
+    return false;
+  }
+    
+
+  /**
+   * Get Nodes which have corrupt replicas of Block
+   * 
+   * @param blk Block for which nodes are requested
+   * @return collection of nodes. Null if does not exists
+   */
+  Collection<DatanodeDescriptor> getNodes(Block blk) {
+    return corruptReplicasMap.get(blk);
+  }
+
+  /**
+   * Check if replica belonging to Datanode is corrupt
+   *
+   * @param blk Block to check
+   * @param node DatanodeDescriptor which holds the replica
+   * @return true if replica is corrupt, false if does not exists in this map
+   */
+  boolean isReplicaCorrupt(Block blk, DatanodeDescriptor node) {
+    Collection<DatanodeDescriptor> nodes = getNodes(blk);
+    return ((nodes != null) && (nodes.contains(node)));
+  }
+
+  public int numCorruptReplicas(Block blk) {
+    Collection<DatanodeDescriptor> nodes = getNodes(blk);
+    return (nodes == null) ? 0 : nodes.size();
+  }
+  
+  public int size() {
+    return corruptReplicasMap.size();
+  }
+}

+ 466 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java

@@ -0,0 +1,466 @@
+/**
+ * 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 java.io.DataInput;
+import java.io.IOException;
+import java.util.*;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.DeprecatedUTF8;
+import org.apache.hadoop.io.WritableUtils;
+
+/**************************************************
+ * DatanodeDescriptor tracks stats on a given DataNode,
+ * such as available storage capacity, last update time, etc.,
+ * and maintains a set of blocks stored on the datanode. 
+ *
+ * This data structure is a data structure that is internal
+ * to the namenode. It is *not* sent over-the-wire to the Client
+ * or the Datnodes. Neither is it stored persistently in the
+ * fsImage.
+
+ **************************************************/
+public class DatanodeDescriptor extends DatanodeInfo {
+  /** Block and targets pair */
+  public static class BlockTargetPair {
+    public final Block block;
+    public final DatanodeDescriptor[] targets;    
+
+    BlockTargetPair(Block block, DatanodeDescriptor[] targets) {
+      this.block = block;
+      this.targets = targets;
+    }
+  }
+
+  /** A BlockTargetPair queue. */
+  private static class BlockQueue {
+    private final Queue<BlockTargetPair> blockq = new LinkedList<BlockTargetPair>();
+
+    /** Size of the queue */
+    synchronized int size() {return blockq.size();}
+
+    /** Enqueue */
+    synchronized boolean offer(Block block, DatanodeDescriptor[] targets) { 
+      return blockq.offer(new BlockTargetPair(block, targets));
+    }
+
+    /** Dequeue */
+    synchronized List<BlockTargetPair> poll(int numBlocks) {
+      if (numBlocks <= 0 || blockq.isEmpty()) {
+        return null;
+      }
+
+      List<BlockTargetPair> results = new ArrayList<BlockTargetPair>();
+      for(; !blockq.isEmpty() && numBlocks > 0; numBlocks--) {
+        results.add(blockq.poll());
+      }
+      return results;
+    }
+  }
+
+  private volatile BlockInfo blockList = null;
+  // isAlive == heartbeats.contains(this)
+  // This is an optimization, because contains takes O(n) time on Arraylist
+  protected boolean isAlive = false;
+  protected boolean needKeyUpdate = false;
+
+  /** A queue of blocks to be replicated by this datanode */
+  private BlockQueue replicateBlocks = new BlockQueue();
+  /** A queue of blocks to be recovered by this datanode */
+  private BlockQueue recoverBlocks = new BlockQueue();
+  /** A set of blocks to be invalidated by this datanode */
+  private Set<Block> invalidateBlocks = new TreeSet<Block>();
+
+  /* Variables for maintaning number of blocks scheduled to be written to
+   * this datanode. This count is approximate and might be slightly higger
+   * in case of errors (e.g. datanode does not report if an error occurs 
+   * while writing the block).
+   */
+  private int currApproxBlocksScheduled = 0;
+  private int prevApproxBlocksScheduled = 0;
+  private long lastBlocksScheduledRollTime = 0;
+  private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
+  
+  /** Default constructor */
+  public DatanodeDescriptor() {}
+  
+  /** DatanodeDescriptor constructor
+   * @param nodeID id of the data node
+   */
+  public DatanodeDescriptor(DatanodeID nodeID) {
+    this(nodeID, 0L, 0L, 0L, 0);
+  }
+
+  /** DatanodeDescriptor constructor
+   * 
+   * @param nodeID id of the data node
+   * @param networkLocation location of the data node in network
+   */
+  public DatanodeDescriptor(DatanodeID nodeID, 
+                            String networkLocation) {
+    this(nodeID, networkLocation, null);
+  }
+  
+  /** DatanodeDescriptor constructor
+   * 
+   * @param nodeID id of the data node
+   * @param networkLocation location of the data node in network
+   * @param hostName it could be different from host specified for DatanodeID
+   */
+  public DatanodeDescriptor(DatanodeID nodeID, 
+                            String networkLocation,
+                            String hostName) {
+    this(nodeID, networkLocation, hostName, 0L, 0L, 0L, 0);
+  }
+  
+  /** DatanodeDescriptor constructor
+   * 
+   * @param nodeID id of the data node
+   * @param capacity capacity of the data node
+   * @param dfsUsed space used by the data node
+   * @param remaining remaing capacity of the data node
+   * @param xceiverCount # of data transfers at the data node
+   */
+  public DatanodeDescriptor(DatanodeID nodeID, 
+                            long capacity,
+                            long dfsUsed,
+                            long remaining,
+                            int xceiverCount) {
+    super(nodeID);
+    updateHeartbeat(capacity, dfsUsed, remaining, xceiverCount);
+  }
+
+  /** DatanodeDescriptor constructor
+   * 
+   * @param nodeID id of the data node
+   * @param networkLocation location of the data node in network
+   * @param capacity capacity of the data node, including space used by non-dfs
+   * @param dfsUsed the used space by dfs datanode
+   * @param remaining remaing capacity of the data node
+   * @param xceiverCount # of data transfers at the data node
+   */
+  public DatanodeDescriptor(DatanodeID nodeID,
+                            String networkLocation,
+                            String hostName,
+                            long capacity,
+                            long dfsUsed,
+                            long remaining,
+                            int xceiverCount) {
+    super(nodeID, networkLocation, hostName);
+    updateHeartbeat(capacity, dfsUsed, remaining, xceiverCount);
+  }
+
+  /**
+   * Add data-node to the block.
+   * Add block to the head of the list of blocks belonging to the data-node.
+   */
+  boolean addBlock(BlockInfo b) {
+    if(!b.addNode(this))
+      return false;
+    // add to the head of the data-node list
+    blockList = b.listInsert(blockList, this);
+    return true;
+  }
+  
+  /**
+   * Remove block from the list of blocks belonging to the data-node.
+   * Remove data-node from the block.
+   */
+  boolean removeBlock(BlockInfo b) {
+    blockList = b.listRemove(blockList, this);
+    return b.removeNode(this);
+  }
+
+  /**
+   * Move block to the head of the list of blocks belonging to the data-node.
+   */
+  void moveBlockToHead(BlockInfo b) {
+    blockList = b.listRemove(blockList, this);
+    blockList = b.listInsert(blockList, this);
+  }
+
+  void resetBlocks() {
+    this.capacity = 0;
+    this.remaining = 0;
+    this.dfsUsed = 0;
+    this.xceiverCount = 0;
+    this.blockList = null;
+    this.invalidateBlocks.clear();
+  }
+
+  public int numBlocks() {
+    return blockList == null ? 0 : blockList.listCount(this);
+  }
+
+  /**
+   */
+  void updateHeartbeat(long capacity, long dfsUsed, long remaining,
+      int xceiverCount) {
+    this.capacity = capacity;
+    this.dfsUsed = dfsUsed;
+    this.remaining = remaining;
+    this.lastUpdate = System.currentTimeMillis();
+    this.xceiverCount = xceiverCount;
+    rollBlocksScheduled(lastUpdate);
+  }
+
+  /**
+   * Iterates over the list of blocks belonging to the data-node.
+   */
+  static private class BlockIterator implements Iterator<Block> {
+    private BlockInfo current;
+    private DatanodeDescriptor node;
+      
+    BlockIterator(BlockInfo head, DatanodeDescriptor dn) {
+      this.current = head;
+      this.node = dn;
+    }
+
+    public boolean hasNext() {
+      return current != null;
+    }
+
+    public BlockInfo next() {
+      BlockInfo res = current;
+      current = current.getNext(current.findDatanode(node));
+      return res;
+    }
+
+    public void remove()  {
+      throw new UnsupportedOperationException("Sorry. can't remove.");
+    }
+  }
+
+  Iterator<Block> getBlockIterator() {
+    return new BlockIterator(this.blockList, this);
+  }
+  
+  /**
+   * Store block replication work.
+   */
+  void addBlockToBeReplicated(Block block, DatanodeDescriptor[] targets) {
+    assert(block != null && targets != null && targets.length > 0);
+    replicateBlocks.offer(block, targets);
+  }
+
+  /**
+   * Store block recovery work.
+   */
+  void addBlockToBeRecovered(Block block, DatanodeDescriptor[] targets) {
+    assert(block != null && targets != null && targets.length > 0);
+    recoverBlocks.offer(block, targets);
+  }
+
+  /**
+   * Store block invalidation work.
+   */
+  void addBlocksToBeInvalidated(List<Block> blocklist) {
+    assert(blocklist != null && blocklist.size() > 0);
+    synchronized (invalidateBlocks) {
+      for(Block blk : blocklist) {
+        invalidateBlocks.add(blk);
+      }
+    }
+  }
+
+  /**
+   * The number of work items that are pending to be replicated
+   */
+  int getNumberOfBlocksToBeReplicated() {
+    return replicateBlocks.size();
+  }
+
+  /**
+   * The number of block invalidation items that are pending to 
+   * be sent to the datanode
+   */
+  int getNumberOfBlocksToBeInvalidated() {
+    synchronized (invalidateBlocks) {
+      return invalidateBlocks.size();
+    }
+  }
+  
+  BlockCommand getReplicationCommand(int maxTransfers) {
+    List<BlockTargetPair> blocktargetlist = replicateBlocks.poll(maxTransfers);
+    return blocktargetlist == null? null:
+        new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blocktargetlist);
+  }
+
+  BlockCommand getLeaseRecoveryCommand(int maxTransfers) {
+    List<BlockTargetPair> blocktargetlist = recoverBlocks.poll(maxTransfers);
+    return blocktargetlist == null? null:
+        new BlockCommand(DatanodeProtocol.DNA_RECOVERBLOCK, blocktargetlist);
+  }
+
+  /**
+   * Remove the specified number of blocks to be invalidated
+   */
+  BlockCommand getInvalidateBlocks(int maxblocks) {
+    Block[] deleteList = getBlockArray(invalidateBlocks, maxblocks); 
+    return deleteList == null? 
+        null: new BlockCommand(DatanodeProtocol.DNA_INVALIDATE, deleteList);
+  }
+
+  static private Block[] getBlockArray(Collection<Block> blocks, int max) {
+    Block[] blockarray = null;
+    synchronized(blocks) {
+      int available = blocks.size();
+      int n = available;
+      if (max > 0 && n > 0) {
+        if (max < n) {
+          n = max;
+        }
+        // allocate the properly sized block array ... 
+        blockarray = new Block[n];
+
+        // iterate tree collecting n blocks... 
+        Iterator<Block> e = blocks.iterator();
+        int blockCount = 0;
+
+        while (blockCount < n && e.hasNext()) {
+          // insert into array ... 
+          blockarray[blockCount++] = e.next();
+
+          // remove from tree via iterator, if we are removing 
+          // less than total available blocks
+          if (n < available){
+            e.remove();
+          }
+        }
+        assert(blockarray.length == n);
+        
+        // now if the number of blocks removed equals available blocks,
+        // them remove all blocks in one fell swoop via clear
+        if (n == available) { 
+          blocks.clear();
+        }
+      }
+    }
+    return blockarray;
+  }
+
+  void reportDiff(BlocksMap blocksMap,
+                  BlockListAsLongs newReport,
+                  Collection<Block> toAdd,    // add to DatanodeDescriptor
+                  Collection<Block> toRemove, // remove from DatanodeDescriptor
+                  Collection<Block> toInvalidate) { // should be removed from DN
+    // place a deilimiter in the list which separates blocks 
+    // that have been reported from those that have not
+    BlockInfo delimiter = new BlockInfo(new Block(), 1);
+    boolean added = this.addBlock(delimiter);
+    assert added : "Delimiting block cannot be present in the node";
+    if(newReport == null)
+      newReport = new BlockListAsLongs( new long[0]);
+    // scan the report and collect newly reported blocks
+    // Note we are taking special precaution to limit tmp blocks allocated
+    // as part this block report - which why block list is stored as longs
+    Block iblk = new Block(); // a fixed new'ed block to be reused with index i
+    for (int i = 0; i < newReport.getNumberOfBlocks(); ++i) {
+      iblk.set(newReport.getBlockId(i), newReport.getBlockLen(i), 
+               newReport.getBlockGenStamp(i));
+      BlockInfo storedBlock = blocksMap.getStoredBlock(iblk);
+      if(storedBlock == null) {
+        // If block is not in blocksMap it does not belong to any file
+        toInvalidate.add(new Block(iblk));
+        continue;
+      }
+      if(storedBlock.findDatanode(this) < 0) {// Known block, but not on the DN
+        // if the size differs from what is in the blockmap, then return
+        // the new block. addStoredBlock will then pick up the right size of this
+        // block and will update the block object in the BlocksMap
+        if (storedBlock.getNumBytes() != iblk.getNumBytes()) {
+          toAdd.add(new Block(iblk));
+        } else {
+          toAdd.add(storedBlock);
+        }
+        continue;
+      }
+      // move block to the head of the list
+      this.moveBlockToHead(storedBlock);
+    }
+    // collect blocks that have not been reported
+    // all of them are next to the delimiter
+    Iterator<Block> it = new BlockIterator(delimiter.getNext(0), this);
+    while(it.hasNext())
+      toRemove.add(it.next());
+    this.removeBlock(delimiter);
+  }
+
+  /** Serialization for FSEditLog */
+  void readFieldsFromFSEditLog(DataInput in) throws IOException {
+    this.name = DeprecatedUTF8.readString(in);
+    this.storageID = DeprecatedUTF8.readString(in);
+    this.infoPort = in.readShort() & 0x0000ffff;
+
+    this.capacity = in.readLong();
+    this.dfsUsed = in.readLong();
+    this.remaining = in.readLong();
+    this.lastUpdate = in.readLong();
+    this.xceiverCount = in.readInt();
+    this.location = Text.readString(in);
+    this.hostName = Text.readString(in);
+    setAdminState(WritableUtils.readEnum(in, AdminStates.class));
+  }
+  
+  /**
+   * @return Approximate number of blocks currently scheduled to be written 
+   * to this datanode.
+   */
+  public int getBlocksScheduled() {
+    return currApproxBlocksScheduled + prevApproxBlocksScheduled;
+  }
+  
+  /**
+   * Increments counter for number of blocks scheduled. 
+   */
+  void incBlocksScheduled() {
+    currApproxBlocksScheduled++;
+  }
+  
+  /**
+   * Decrements counter for number of blocks scheduled.
+   */
+  void decBlocksScheduled() {
+    if (prevApproxBlocksScheduled > 0) {
+      prevApproxBlocksScheduled--;
+    } else if (currApproxBlocksScheduled > 0) {
+      currApproxBlocksScheduled--;
+    } 
+    // its ok if both counters are zero.
+  }
+  
+  /**
+   * Adjusts curr and prev number of blocks scheduled every few minutes.
+   */
+  private void rollBlocksScheduled(long now) {
+    if ((now - lastBlocksScheduledRollTime) > 
+        BLOCKS_SCHEDULED_ROLL_INTERVAL) {
+      prevApproxBlocksScheduled = currApproxBlocksScheduled;
+      currApproxBlocksScheduled = 0;
+      lastBlocksScheduledRollTime = now;
+    }
+  }
+}

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

@@ -0,0 +1,93 @@
+/**
+ * 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 java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.CyclicIteration;
+
+/**
+ * Manage node decommissioning.
+ */
+class DecommissionManager {
+  static final Log LOG = LogFactory.getLog(DecommissionManager.class);
+
+  private final FSNamesystem fsnamesystem;
+
+  DecommissionManager(FSNamesystem namesystem) {
+    this.fsnamesystem = namesystem;
+  }
+
+  /** Periodically check decommission status. */
+  class Monitor implements Runnable {
+    /** recheckInterval is how often namenode checks
+     *  if a node has finished decommission
+     */
+    private final long recheckInterval;
+    /** The number of decommission nodes to check for each interval */
+    private final int numNodesPerCheck;
+    /** firstkey can be initialized to anything. */
+    private String firstkey = "";
+
+    Monitor(int recheckIntervalInSecond, int numNodesPerCheck) {
+      this.recheckInterval = recheckIntervalInSecond * 1000L;
+      this.numNodesPerCheck = numNodesPerCheck;
+    }
+
+    /**
+     * Check decommission status of numNodesPerCheck nodes
+     * for every recheckInterval milliseconds.
+     */
+    public void run() {
+      for(; fsnamesystem.isRunning(); ) {
+        synchronized(fsnamesystem) {
+          check();
+        }
+  
+        try {
+          Thread.sleep(recheckInterval);
+        } catch (InterruptedException ie) {
+          LOG.warn(this.getClass().getSimpleName() + " interrupted: " + ie);
+        }
+      }
+    }
+    
+    private void check() {
+      int count = 0;
+      for(Map.Entry<String, DatanodeDescriptor> entry
+          : new CyclicIteration<String, DatanodeDescriptor>(
+              fsnamesystem.datanodeMap, firstkey)) {
+        final DatanodeDescriptor d = entry.getValue();
+        firstkey = entry.getKey();
+
+        if (d.isDecommissionInProgress()) {
+          try {
+            fsnamesystem.checkDecommissionStateInternal(d);
+          } catch(Exception e) {
+            LOG.warn("entry=" + entry, e);
+          }
+          if (++count == numNodesPerCheck) {
+            return;
+          }
+        }
+      }
+    }
+  }
+}

+ 98 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java

@@ -0,0 +1,98 @@
+/**
+ * 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 java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * A base class for the servlets in DFS.
+ */
+abstract class DfsServlet extends HttpServlet {
+  /** For java.io.Serializable */
+  private static final long serialVersionUID = 1L;
+
+  static final Log LOG = LogFactory.getLog(DfsServlet.class.getCanonicalName());
+
+  /** Get {@link UserGroupInformation} from request */
+  protected UnixUserGroupInformation getUGI(HttpServletRequest request) {
+    String ugi = request.getParameter("ugi");
+    try {
+      return new UnixUserGroupInformation(ugi.split(","));
+    }
+    catch(Exception e) {
+      LOG.warn("Invalid ugi (= " + ugi + ")");
+    }
+    return JspHelper.webUGI;
+  }
+
+  /**
+   * Create a {@link NameNode} proxy from the current {@link ServletContext}. 
+   */
+  protected ClientProtocol createNameNodeProxy(UnixUserGroupInformation ugi
+      ) throws IOException {
+    ServletContext context = getServletContext();
+    InetSocketAddress nnAddr = (InetSocketAddress)context.getAttribute("name.node.address");
+    Configuration conf = new Configuration(
+        (Configuration)context.getAttribute("name.conf"));
+    UnixUserGroupInformation.saveToConf(conf,
+        UnixUserGroupInformation.UGI_PROPERTY_NAME, ugi);
+    return DFSClient.createNamenode(nnAddr, conf);
+  }
+
+  /** Create a URI for redirecting request */
+  protected URI createRedirectUri(String servletpath, UserGroupInformation ugi,
+      DatanodeID host, HttpServletRequest request) throws URISyntaxException {
+    final String hostname = host instanceof DatanodeInfo?
+        ((DatanodeInfo)host).getHostName(): host.getHost();
+    final String scheme = request.getScheme();
+    final int port = "https".equals(scheme)?
+        (Integer)getServletContext().getAttribute("datanode.https.port")
+        : host.getInfoPort();
+    final String filename = request.getPathInfo();
+    return new URI(scheme, null, hostname, port, servletpath,
+        "filename=" + filename + "&ugi=" + ugi, null);
+  }
+
+  /** Get filename from the request */
+  protected String getFilename(HttpServletRequest request,
+      HttpServletResponse response) throws IOException {
+    final String filename = request.getParameter("filename");
+    if (filename == null || filename.length() == 0) {
+      throw new IOException("Invalid filename");
+    }
+    return filename;
+  }
+}

+ 118 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java

@@ -0,0 +1,118 @@
+/**
+ * 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 java.io.DataInputStream;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+
+/**
+ * An implementation of the abstract class {@link EditLogInputStream},
+ * which is used to updates HDFS meta-data state on a backup node.
+ * 
+ * @see org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol#journal
+ * (org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration,
+ *  int, int, byte[])
+ */
+class EditLogBackupInputStream extends EditLogInputStream {
+  String address; // sender address 
+  private ByteBufferInputStream inner;
+  private DataInputStream in;
+
+  /**
+   * A ByteArrayInputStream, which lets modify the underlying byte array.
+   */
+  private static class ByteBufferInputStream extends ByteArrayInputStream {
+    ByteBufferInputStream() {
+      super(new byte[0]);
+    }
+
+    byte[] getData() {
+      return super.buf;
+    }
+
+    void setData(byte[] newBytes) {
+      super.buf = newBytes;
+      super.count = newBytes == null ? 0 : newBytes.length;
+      super.mark = 0;
+      reset();
+    }
+
+    /**
+     * Number of bytes read from the stream so far.
+     */
+    int length() {
+      return count;
+    }
+  }
+
+  EditLogBackupInputStream(String name) throws IOException {
+    address = name;
+    inner = new ByteBufferInputStream();
+    in = new DataInputStream(inner);
+  }
+
+  @Override // JournalStream
+  public String getName() {
+    return address;
+  }
+
+  @Override // JournalStream
+  public JournalType getType() {
+    return JournalType.BACKUP;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return in.available();
+  }
+
+  @Override
+  public int read() throws IOException {
+    return in.read();
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    return in.read(b, off, len);
+  }
+
+  @Override
+  public void close() throws IOException {
+    in.close();
+  }
+
+  @Override
+  long length() throws IOException {
+    // file size + size of both buffers
+    return inner.length();
+  }
+
+  DataInputStream getDataInputStream() {
+    return in;
+  }
+
+  void setBytes(byte[] newBytes) throws IOException {
+    inner.setData(newBytes);
+    in.reset();
+  }
+
+  void clear() throws IOException {
+    setBytes(null);
+  }
+}

+ 207 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java

@@ -0,0 +1,207 @@
+/**
+ * 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 java.io.DataOutputStream;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+
+/**
+ * An implementation of the abstract class {@link EditLogOutputStream},
+ * which streams edits to a backup node.
+ * 
+ * @see org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol#journal
+ * (org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration,
+ *  int, int, byte[])
+ */
+class EditLogBackupOutputStream extends EditLogOutputStream {
+  static int DEFAULT_BUFFER_SIZE = 256;
+
+  private NamenodeProtocol backupNode;          // RPC proxy to backup node
+  private NamenodeRegistration bnRegistration;  // backup node registration
+  private NamenodeRegistration nnRegistration;  // active node registration
+  private ArrayList<JournalRecord> bufCurrent;  // current buffer for writing
+  private ArrayList<JournalRecord> bufReady;    // buffer ready for flushing
+  private DataOutputBuffer out;     // serialized output sent to backup node
+
+  static class JournalRecord {
+    byte op;
+    Writable[] args;
+
+    JournalRecord(byte op, Writable ... writables) {
+      this.op = op;
+      this.args = writables;
+    }
+
+    void write(DataOutputStream out) throws IOException {
+      out.write(op);
+      if(args == null)
+        return;
+      for(Writable w : args)
+        w.write(out);
+    }
+  }
+
+  EditLogBackupOutputStream(NamenodeRegistration bnReg, // backup node
+                            NamenodeRegistration nnReg) // active name-node
+  throws IOException {
+    super();
+    this.bnRegistration = bnReg;
+    this.nnRegistration = nnReg;
+    InetSocketAddress bnAddress =
+      NetUtils.createSocketAddr(bnRegistration.getAddress());
+    Storage.LOG.info("EditLogBackupOutputStream connects to: " + bnAddress);
+    try {
+      this.backupNode =
+        (NamenodeProtocol) RPC.getProxy(NamenodeProtocol.class,
+            NamenodeProtocol.versionID, bnAddress, new Configuration());
+    } catch(IOException e) {
+      Storage.LOG.error("Error connecting to: " + bnAddress, e);
+      throw e;
+    }
+    this.bufCurrent = new ArrayList<JournalRecord>();
+    this.bufReady = new ArrayList<JournalRecord>();
+    this.out = new DataOutputBuffer(DEFAULT_BUFFER_SIZE);
+  }
+
+  @Override // JournalStream
+  public String getName() {
+    return bnRegistration.getAddress();
+  }
+
+  @Override // JournalStream
+  public JournalType getType() {
+    return JournalType.BACKUP;
+  }
+
+  @Override // EditLogOutputStream
+  public void write(int b) throws IOException {
+    throw new IOException("Not implemented");
+  }
+
+  @Override // EditLogOutputStream
+  void write(byte op, Writable ... writables) throws IOException {
+    bufCurrent.add(new JournalRecord(op, writables));
+  }
+
+  /**
+   * There is no persistent storage. Just clear the buffers.
+   */
+  @Override // EditLogOutputStream
+  void create() throws IOException {
+    bufCurrent.clear();
+    assert bufReady.size() == 0 : "previous data is not flushed yet";
+  }
+
+  @Override // EditLogOutputStream
+  public void close() throws IOException {
+    // close should have been called after all pending transactions 
+    // have been flushed & synced.
+    int size = bufCurrent.size();
+    if (size != 0) {
+      throw new IOException("BackupEditStream has " + size +
+                          " records still to be flushed and cannot be closed.");
+    } 
+    RPC.stopProxy(backupNode); // stop the RPC threads
+    bufCurrent = bufReady = null;
+  }
+
+  @Override // EditLogOutputStream
+  void setReadyToFlush() throws IOException {
+    assert bufReady.size() == 0 : "previous data is not flushed yet";
+    ArrayList<JournalRecord>  tmp = bufReady;
+    bufReady = bufCurrent;
+    bufCurrent = tmp;
+  }
+
+  @Override // EditLogOutputStream
+  protected void flushAndSync() throws IOException {
+    assert out.size() == 0 : "Output buffer is not empty";
+    int bufReadySize = bufReady.size();
+    for(int idx = 0; idx < bufReadySize; idx++) {
+      JournalRecord jRec = null;
+      for(; idx < bufReadySize; idx++) {
+        jRec = bufReady.get(idx);
+        if(jRec.op >= FSEditLog.OP_JSPOOL_START)
+          break;  // special operation should be sent in a separate call to BN
+        jRec.write(out);
+      }
+      if(out.size() > 0)
+        send(NamenodeProtocol.JA_JOURNAL);
+      if(idx == bufReadySize)
+        break;
+      // operation like start journal spool or increment checkpoint time
+      // is a separate call to BN
+      jRec.write(out);
+      send(jRec.op);
+    }
+    bufReady.clear();         // erase all data in the buffer
+    out.reset();              // reset buffer to the start position
+  }
+
+  /**
+   * There is no persistent storage. Therefore length is 0.<p>
+   * Length is used to check when it is large enough to start a checkpoint.
+   * This criteria should not be used for backup streams.
+   */
+  @Override // EditLogOutputStream
+  long length() throws IOException {
+    return 0;
+  }
+
+  private void send(int ja) throws IOException {
+    try {
+      int length = out.getLength();
+      out.write(FSEditLog.OP_INVALID);
+      backupNode.journal(nnRegistration, ja, length, out.getData());
+    } finally {
+      out.reset();
+    }
+  }
+
+  /**
+   * Get backup node registration.
+   */
+  NamenodeRegistration getRegistration() {
+    return bnRegistration;
+  }
+
+  /**
+   * Verify that the backup node is alive.
+   */
+  boolean isAlive() {
+    try {
+      send(NamenodeProtocol.JA_IS_ALIVE);
+    } catch(IOException ei) {
+      Storage.LOG.info(bnRegistration.getRole() + " "
+                      + bnRegistration.getAddress() + " is not alive. ", ei);
+      return false;
+    }
+    return true;
+  }
+}

+ 73 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java

@@ -0,0 +1,73 @@
+/**
+ * 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 java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+/**
+ * An implementation of the abstract class {@link EditLogInputStream}, which
+ * reads edits from a local file.
+ */
+class EditLogFileInputStream extends EditLogInputStream {
+  private File file;
+  private FileInputStream fStream;
+
+  EditLogFileInputStream(File name) throws IOException {
+    file = name;
+    fStream = new FileInputStream(name);
+  }
+
+  @Override // JournalStream
+  public String getName() {
+    return file.getPath();
+  }
+
+  @Override // JournalStream
+  public JournalType getType() {
+    return JournalType.FILE;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return fStream.available();
+  }
+
+  @Override
+  public int read() throws IOException {
+    return fStream.read();
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    return fStream.read(b, off, len);
+  }
+
+  @Override
+  public void close() throws IOException {
+    fStream.close();
+  }
+
+  @Override
+  long length() throws IOException {
+    // file size + size of both buffers
+    return file.length();
+  }
+}

+ 187 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java

@@ -0,0 +1,187 @@
+/**
+ * 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 java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * An implementation of the abstract class {@link EditLogOutputStream}, which
+ * stores edits in a local file.
+ */
+class EditLogFileOutputStream extends EditLogOutputStream {
+  private static int EDITS_FILE_HEADER_SIZE_BYTES = Integer.SIZE / Byte.SIZE;
+
+  private File file;
+  private FileOutputStream fp; // file stream for storing edit logs
+  private FileChannel fc; // channel of the file stream for sync
+  private DataOutputBuffer bufCurrent; // current buffer for writing
+  private DataOutputBuffer bufReady; // buffer ready for flushing
+  static ByteBuffer fill = ByteBuffer.allocateDirect(512); // preallocation
+
+  /**
+   * Creates output buffers and file object.
+   * 
+   * @param name
+   *          File name to store edit log
+   * @param size
+   *          Size of flush buffer
+   * @throws IOException
+   */
+  EditLogFileOutputStream(File name, int size) throws IOException {
+    super();
+    file = name;
+    bufCurrent = new DataOutputBuffer(size);
+    bufReady = new DataOutputBuffer(size);
+    RandomAccessFile rp = new RandomAccessFile(name, "rw");
+    fp = new FileOutputStream(rp.getFD()); // open for append
+    fc = rp.getChannel();
+    fc.position(fc.size());
+  }
+
+  @Override // JournalStream
+  public String getName() {
+    return file.getPath();
+  }
+
+  @Override // JournalStream
+  public JournalType getType() {
+    return JournalType.FILE;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void write(int b) throws IOException {
+    bufCurrent.write(b);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  void write(byte op, Writable... writables) throws IOException {
+    write(op);
+    for (Writable w : writables) {
+      w.write(bufCurrent);
+    }
+  }
+
+  /**
+   * Create empty edits logs file.
+   */
+  @Override
+  void create() throws IOException {
+    fc.truncate(0);
+    fc.position(0);
+    bufCurrent.writeInt(FSConstants.LAYOUT_VERSION);
+    setReadyToFlush();
+    flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    // 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.");
+    }
+    bufCurrent.close();
+    bufReady.close();
+
+    // remove the last INVALID marker from transaction log.
+    fc.truncate(fc.position());
+    fp.close();
+
+    bufCurrent = bufReady = null;
+  }
+
+  /**
+   * All data that has been written to the stream so far will be flushed. New
+   * data can be still written to the stream while flushing is performed.
+   */
+  @Override
+  void setReadyToFlush() throws IOException {
+    assert bufReady.size() == 0 : "previous data is not flushed yet";
+    write(FSEditLog.OP_INVALID); // insert end-of-file marker
+    DataOutputBuffer tmp = bufReady;
+    bufReady = bufCurrent;
+    bufCurrent = tmp;
+  }
+
+  /**
+   * Flush ready buffer to persistent store. currentBuffer is not flushed as it
+   * accumulates new log records while readyBuffer will be flushed and synced.
+   */
+  @Override
+  protected void flushAndSync() throws IOException {
+    preallocate(); // preallocate file if necessary
+    bufReady.writeTo(fp); // write data to file
+    bufReady.reset(); // erase all data in the buffer
+    fc.force(false); // metadata updates not needed because of preallocation
+    fc.position(fc.position() - 1); // skip back the end-of-file marker
+  }
+
+  /**
+   * Return the size of the current edit log including buffered data.
+   */
+  @Override
+  long length() throws IOException {
+    // file size - header size + size of both buffers
+    return fc.size() - EDITS_FILE_HEADER_SIZE_BYTES + bufReady.size()
+        + bufCurrent.size();
+  }
+
+  // allocate a big chunk of data
+  private void preallocate() throws IOException {
+    long position = fc.position();
+    if (position + 4096 >= fc.size()) {
+      FSNamesystem.LOG.debug("Preallocating Edit log, current size "
+          + fc.size());
+      long newsize = position + 1024 * 1024; // 1MB
+      fill.position(0);
+      int written = fc.write(fill, newsize);
+      FSNamesystem.LOG.debug("Edit log size is now " + fc.size() + " written "
+          + written + " bytes " + " at offset " + newsize);
+    }
+  }
+
+  /**
+   * Operations like OP_JSPOOL_START and OP_CHECKPOINT_TIME should not be
+   * written into edits file.
+   */
+  @Override
+  boolean isOperationSupported(byte op) {
+    return op < FSEditLog.OP_JSPOOL_START - 1;
+  }
+
+  /**
+   * Returns the file associated with this stream.
+   */
+  File getFile() {
+    return file;
+  }
+}

+ 57 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java

@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * A generic abstract class to support reading edits log data from 
+ * persistent storage.
+ * 
+ * It should stream bytes from the storage exactly as they were written
+ * into the #{@link EditLogOutputStream}.
+ */
+abstract class EditLogInputStream extends InputStream
+implements JournalStream {
+  /** {@inheritDoc} */
+  public abstract int available() throws IOException;
+
+  /** {@inheritDoc} */
+  public abstract int read() throws IOException;
+
+  /** {@inheritDoc} */
+  public abstract int read(byte[] b, int off, int len) throws IOException;
+
+  /** {@inheritDoc} */
+  public abstract void close() throws IOException;
+
+  /**
+   * Return the size of the current edits log.
+   */
+  abstract long length() throws IOException;
+
+  /**
+   * Return DataInputStream based on this edit stream.
+   */
+  DataInputStream getDataInputStream() {
+    return new DataInputStream(new BufferedInputStream(this));
+  }
+}

+ 116 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java

@@ -0,0 +1,116 @@
+/**
+ * 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 java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * A generic abstract class to support journaling of edits logs into 
+ * a persistent storage.
+ */
+abstract class EditLogOutputStream extends OutputStream 
+implements JournalStream {
+  // these are statistics counters
+  private long numSync;        // number of sync(s) to disk
+  private long totalTimeSync;  // total time to sync
+
+  EditLogOutputStream() throws IOException {
+    numSync = totalTimeSync = 0;
+  }
+
+  /** {@inheritDoc} */
+  abstract public void write(int b) throws IOException;
+
+  /**
+   * Write edits log record into the stream.
+   * The record is represented by operation name and
+   * an array of Writable arguments.
+   * 
+   * @param op operation
+   * @param writables array of Writable arguments
+   * @throws IOException
+   */
+  abstract void write(byte op, Writable ... writables) throws IOException;
+
+  /**
+   * Create and initialize underlying persistent edits log storage.
+   * 
+   * @throws IOException
+   */
+  abstract void create() throws IOException;
+
+  /** {@inheritDoc} */
+  abstract public void close() throws IOException;
+
+  /**
+   * All data that has been written to the stream so far will be flushed.
+   * New data can be still written to the stream while flushing is performed.
+   */
+  abstract void setReadyToFlush() throws IOException;
+
+  /**
+   * Flush and sync all data that is ready to be flush 
+   * {@link #setReadyToFlush()} into underlying persistent store.
+   * @throws IOException
+   */
+  abstract protected void flushAndSync() throws IOException;
+
+  /**
+   * Flush data to persistent store.
+   * Collect sync metrics.
+   */
+  public void flush() throws IOException {
+    numSync++;
+    long start = FSNamesystem.now();
+    flushAndSync();
+    long end = FSNamesystem.now();
+    totalTimeSync += (end - start);
+  }
+
+  /**
+   * Return the size of the current edits log.
+   * Length is used to check when it is large enough to start a checkpoint.
+   */
+  abstract long length() throws IOException;
+
+  boolean isOperationSupported(byte op) {
+    return true;
+  }
+
+  /**
+   * Return total time spent in {@link #flushAndSync()}
+   */
+  long getTotalSyncTime() {
+    return totalTimeSync;
+  }
+
+  /**
+   * Return number of calls to {@link #flushAndSync()}
+   */
+  long getNumSync() {
+    return numSync;
+  }
+
+  @Override // Object
+  public String toString() {
+    return getName();
+  }
+}

+ 1289 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -0,0 +1,1289 @@
+/**
+ * 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 java.io.*;
+import java.util.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.metrics.MetricsRecord;
+import org.apache.hadoop.metrics.MetricsUtil;
+import org.apache.hadoop.metrics.MetricsContext;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
+
+/*************************************************
+ * FSDirectory stores the filesystem directory state.
+ * It handles writing/loading values to disk, and logging
+ * changes as we go.
+ *
+ * It keeps the filename->blockset mapping always-current
+ * and logged to disk.
+ * 
+ *************************************************/
+class FSDirectory implements Closeable {
+
+  INodeDirectoryWithQuota rootDir;
+  FSImage fsImage;  
+  private volatile boolean ready = false;
+  // Metrics record
+  private MetricsRecord directoryMetrics = null;
+
+  /** Access an existing dfs name directory. */
+  FSDirectory(FSNamesystem ns, Configuration conf) {
+    this(new FSImage(), ns, conf);
+    if(conf.getBoolean("dfs.name.dir.restore", false)) {
+      NameNode.LOG.info("set FSImage.restoreFailedStorage");
+      fsImage.setRestoreFailedStorage(true);
+    }
+    fsImage.setCheckpointDirectories(FSImage.getCheckpointDirs(conf, null),
+                                FSImage.getCheckpointEditsDirs(conf, null));
+  }
+
+  FSDirectory(FSImage fsImage, FSNamesystem ns, Configuration conf) {
+    fsImage.setFSNamesystem(ns);
+    rootDir = new INodeDirectoryWithQuota(INodeDirectory.ROOT_NAME,
+        ns.createFsOwnerPermissions(new FsPermission((short)0755)),
+        Integer.MAX_VALUE, -1);
+    this.fsImage = fsImage;
+    initialize(conf);
+  }
+    
+  private FSNamesystem getFSNamesystem() {
+    return fsImage.getFSNamesystem();
+  }
+
+  private BlockManager getBlockManager() {
+    return getFSNamesystem().blockManager;
+  }
+
+  private void initialize(Configuration conf) {
+    MetricsContext metricsContext = MetricsUtil.getContext("dfs");
+    directoryMetrics = MetricsUtil.createRecord(metricsContext, "FSDirectory");
+    directoryMetrics.setTag("sessionId", conf.get("session.id"));
+  }
+
+  void loadFSImage(Collection<File> dataDirs,
+                   Collection<File> editsDirs,
+                   StartupOption startOpt) throws IOException {
+    // format before starting up if requested
+    if (startOpt == StartupOption.FORMAT) {
+      fsImage.setStorageDirectories(dataDirs, editsDirs);
+      fsImage.format();
+      startOpt = StartupOption.REGULAR;
+    }
+    try {
+      if (fsImage.recoverTransitionRead(dataDirs, editsDirs, startOpt)) {
+        fsImage.saveFSImage();
+      }
+      FSEditLog editLog = fsImage.getEditLog();
+      assert editLog != null : "editLog must be initialized";
+      fsImage.setCheckpointDirectories(null, null);
+    } catch(IOException e) {
+      fsImage.close();
+      throw e;
+    }
+    synchronized (this) {
+      this.ready = true;
+      this.notifyAll();
+    }
+  }
+
+  private void incrDeletedFileCount(int count) {
+    directoryMetrics.incrMetric("files_deleted", count);
+    directoryMetrics.update();
+  }
+    
+  /**
+   * Shutdown the filestore
+   */
+  public void close() throws IOException {
+    fsImage.close();
+  }
+
+  /**
+   * Block until the object is ready to be used.
+   */
+  void waitForReady() {
+    if (!ready) {
+      synchronized (this) {
+        while (!ready) {
+          try {
+            this.wait(5000);
+          } catch (InterruptedException ie) {
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Add the given filename to the fs.
+   */
+  INodeFileUnderConstruction addFile(String path, 
+                PermissionStatus permissions,
+                short replication,
+                long preferredBlockSize,
+                String clientName,
+                String clientMachine,
+                DatanodeDescriptor clientNode,
+                long generationStamp) 
+                throws IOException {
+    waitForReady();
+
+    // Always do an implicit mkdirs for parent directory tree.
+    long modTime = FSNamesystem.now();
+    if (!mkdirs(new Path(path).getParent().toString(), permissions, true,
+        modTime)) {
+      return null;
+    }
+    INodeFileUnderConstruction newNode = new INodeFileUnderConstruction(
+                                 permissions,replication,
+                                 preferredBlockSize, modTime, clientName, 
+                                 clientMachine, clientNode);
+    synchronized (rootDir) {
+      newNode = addNode(path, newNode, -1, false);
+    }
+    if (newNode == null) {
+      NameNode.stateChangeLog.info("DIR* FSDirectory.addFile: "
+                                   +"failed to add "+path
+                                   +" to the file system");
+      return null;
+    }
+    // add create file record to log, record new generation stamp
+    fsImage.getEditLog().logOpenFile(path, newNode);
+
+    NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
+                                  +path+" is added to the file system");
+    return newNode;
+  }
+
+  /**
+   */
+  INode unprotectedAddFile( String path, 
+                            PermissionStatus permissions,
+                            Block[] blocks, 
+                            short replication,
+                            long modificationTime,
+                            long atime,
+                            long preferredBlockSize) {
+    INode newNode;
+    long diskspace = -1; // unknown
+    if (blocks == null)
+      newNode = new INodeDirectory(permissions, modificationTime);
+    else {
+      newNode = new INodeFile(permissions, blocks.length, replication,
+                              modificationTime, atime, preferredBlockSize);
+      diskspace = ((INodeFile)newNode).diskspaceConsumed(blocks);
+    }
+    synchronized (rootDir) {
+      try {
+        newNode = addNode(path, newNode, diskspace, false);
+        if(newNode != null && blocks != null) {
+          int nrBlocks = blocks.length;
+          // Add file->block mapping
+          INodeFile newF = (INodeFile)newNode;
+          for (int i = 0; i < nrBlocks; i++) {
+            newF.setBlock(i, getBlockManager().addINode(blocks[i], newF));
+          }
+        }
+      } catch (IOException e) {
+        return null;
+      }
+      return newNode;
+    }
+  }
+
+  INodeDirectory addToParent( String src,
+                              INodeDirectory parentINode,
+                              PermissionStatus permissions,
+                              Block[] blocks, 
+                              short replication,
+                              long modificationTime,
+                              long atime,
+                              long nsQuota,
+                              long dsQuota,
+                              long preferredBlockSize) {
+    // NOTE: This does not update space counts for parents
+    // create new inode
+    INode newNode;
+    if (blocks == null) {
+      if (nsQuota >= 0 || dsQuota >= 0) {
+        newNode = new INodeDirectoryWithQuota(
+            permissions, modificationTime, nsQuota, dsQuota);
+      } else {
+        newNode = new INodeDirectory(permissions, modificationTime);
+      }
+    } else 
+      newNode = new INodeFile(permissions, blocks.length, replication,
+                              modificationTime, atime, preferredBlockSize);
+    // add new node to the parent
+    INodeDirectory newParent = null;
+    synchronized (rootDir) {
+      try {
+        newParent = rootDir.addToParent(src, newNode, parentINode, false);
+      } catch (FileNotFoundException e) {
+        return null;
+      }
+      if(newParent == null)
+        return null;
+      if(blocks != null) {
+        int nrBlocks = blocks.length;
+        // Add file->block mapping
+        INodeFile newF = (INodeFile)newNode;
+        for (int i = 0; i < nrBlocks; i++) {
+          newF.setBlock(i, getBlockManager().addINode(blocks[i], newF));
+        }
+      }
+    }
+    return newParent;
+  }
+
+  /**
+   * Add a block to the file. Returns a reference to the added block.
+   */
+  Block addBlock(String path, INode[] inodes, Block block
+      ) throws QuotaExceededException  {
+    waitForReady();
+
+    synchronized (rootDir) {
+      INodeFile fileNode = (INodeFile) inodes[inodes.length-1];
+
+      // check quota limits and updated space consumed
+      updateCount(inodes, inodes.length-1, 0, 
+                  fileNode.getPreferredBlockSize()*fileNode.getReplication());
+      
+      // associate the new list of blocks with this file
+      getBlockManager().addINode(block, fileNode);
+      BlockInfo blockInfo = getBlockManager().getStoredBlock(block);
+      fileNode.addBlock(blockInfo);
+
+      NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
+                                    + path + " with " + block
+                                    + " block is added to the in-memory "
+                                    + "file system");
+    }
+    return block;
+  }
+
+  /**
+   * Persist the block list for the inode.
+   */
+  void persistBlocks(String path, INodeFileUnderConstruction file) {
+    waitForReady();
+
+    synchronized (rootDir) {
+      fsImage.getEditLog().logOpenFile(path, file);
+      NameNode.stateChangeLog.debug("DIR* FSDirectory.persistBlocks: "
+                                    +path+" with "+ file.getBlocks().length 
+                                    +" blocks is persisted to the file system");
+    }
+  }
+
+  /**
+   * Close file.
+   */
+  void closeFile(String path, INodeFile file) {
+    waitForReady();
+    synchronized (rootDir) {
+      // file is closed
+      fsImage.getEditLog().logCloseFile(path, file);
+      if (NameNode.stateChangeLog.isDebugEnabled()) {
+        NameNode.stateChangeLog.debug("DIR* FSDirectory.closeFile: "
+                                    +path+" with "+ file.getBlocks().length 
+                                    +" blocks is persisted to the file system");
+      }
+    }
+  }
+
+  /**
+   * Remove a block to the file.
+   */
+  boolean removeBlock(String path, INodeFileUnderConstruction fileNode, 
+                      Block block) throws IOException {
+    waitForReady();
+
+    synchronized (rootDir) {
+      // modify file-> block and blocksMap
+      fileNode.removeBlock(block);
+      getBlockManager().removeINode(block);
+      // If block is removed from blocksMap remove it from corruptReplicasMap
+      getBlockManager().removeFromCorruptReplicasMap(block);
+
+      // write modified block locations to log
+      fsImage.getEditLog().logOpenFile(path, fileNode);
+      NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
+                                    +path+" with "+block
+                                    +" block is added to the file system");
+    }
+    return true;
+  }
+
+  /**
+   * @see #unprotectedRenameTo(String, String, long)
+   */
+  boolean renameTo(String src, String dst) throws QuotaExceededException {
+    if (NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("DIR* FSDirectory.renameTo: "
+                                  +src+" to "+dst);
+    }
+    waitForReady();
+    long now = FSNamesystem.now();
+    if (!unprotectedRenameTo(src, dst, now))
+      return false;
+    fsImage.getEditLog().logRename(src, dst, now);
+    return true;
+  }
+
+  /** Change a path name
+   * 
+   * @param src source path
+   * @param dst destination path
+   * @return true if rename succeeds; false otherwise
+   * @throws QuotaExceededException if the operation violates any quota limit
+   */
+  boolean unprotectedRenameTo(String src, String dst, long timestamp) 
+  throws QuotaExceededException {
+    synchronized (rootDir) {
+      INode[] srcInodes = rootDir.getExistingPathINodes(src);
+
+      // check the validation of the source
+      if (srcInodes[srcInodes.length-1] == null) {
+        NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
+                                     +"failed to rename "+src+" to "+dst+ " because source does not exist");
+        return false;
+      } else if (srcInodes.length == 1) {
+        NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
+            +"failed to rename "+src+" to "+dst+ " because source is the root");
+        return false;
+      }
+      if (isDir(dst)) {
+        dst += Path.SEPARATOR + new Path(src).getName();
+      }
+      
+      // remove source
+      INode srcChild = null;
+      try {
+        srcChild = removeChild(srcInodes, srcInodes.length-1);
+      } catch (IOException e) {
+        // srcChild == null; go to next if statement
+      }
+      if (srcChild == null) {
+        NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
+            +"failed to rename "+src+" to "+dst+ " because the source can not be removed");
+        return false;
+      }
+
+      String srcChildName = srcChild.getLocalName();
+      
+      // check the validity of the destination
+      INode dstChild = null;
+      QuotaExceededException failureByQuota = null;
+
+      byte[][] dstComponents = INode.getPathComponents(dst);
+      INode[] dstInodes = new INode[dstComponents.length];
+      rootDir.getExistingPathINodes(dstComponents, dstInodes);
+      if (dstInodes[dstInodes.length-1] != null) { //check if destination exists
+        NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
+                                     +"failed to rename "+src+" to "+dst+ 
+                                     " because destination exists");
+      } else if (dstInodes[dstInodes.length-2] == null) { // check if its parent exists
+        NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
+            +"failed to rename "+src+" to "+dst+ 
+            " because destination's parent does not exists");
+      }
+      else {
+        // add to the destination
+        srcChild.setLocalName(dstComponents[dstInodes.length-1]);
+        try {
+          // add it to the namespace
+          dstChild = addChild(dstInodes, dstInodes.length-1, srcChild, false);
+        } catch (QuotaExceededException qe) {
+          failureByQuota = qe;
+        }
+      }
+      if (dstChild != null) {
+        if (NameNode.stateChangeLog.isDebugEnabled()) {
+          NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedRenameTo: "
+            +src+" is renamed to "+dst);
+        }
+
+        // update modification time of dst and the parent of src
+        srcInodes[srcInodes.length-2].setModificationTime(timestamp);
+        dstInodes[dstInodes.length-2].setModificationTime(timestamp);
+        return true;
+      } else {
+        NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
+            +"failed to rename "+src+" to "+dst);
+        try {
+          // put it back
+          srcChild.setLocalName(srcChildName);
+          addChild(srcInodes, srcInodes.length-1, srcChild, false);
+        } catch (IOException ignored) {}
+        if (failureByQuota != null) {
+          throw failureByQuota;
+        } else {
+          return false;
+        }
+      }
+    }
+  }
+
+  /**
+   * Set file replication
+   * 
+   * @param src file name
+   * @param replication new replication
+   * @param oldReplication old replication - output parameter
+   * @return array of file blocks
+   * @throws QuotaExceededException
+   */
+  Block[] setReplication(String src, 
+                         short replication,
+                         int[] oldReplication
+                         ) throws QuotaExceededException {
+    waitForReady();
+    Block[] fileBlocks = unprotectedSetReplication(src, replication, oldReplication);
+    if (fileBlocks != null)  // log replication change
+      fsImage.getEditLog().logSetReplication(src, replication);
+    return fileBlocks;
+  }
+
+  Block[] unprotectedSetReplication( String src, 
+                                     short replication,
+                                     int[] oldReplication
+                                     ) throws QuotaExceededException {
+    if (oldReplication == null)
+      oldReplication = new int[1];
+    oldReplication[0] = -1;
+    Block[] fileBlocks = null;
+    synchronized(rootDir) {
+      INode[] inodes = rootDir.getExistingPathINodes(src);
+      INode inode = inodes[inodes.length - 1];
+      if (inode == null)
+        return null;
+      if (inode.isDirectory())
+        return null;
+      INodeFile fileNode = (INodeFile)inode;
+      oldReplication[0] = fileNode.getReplication();
+
+      // check disk quota
+      long dsDelta = (replication - oldReplication[0]) *
+           (fileNode.diskspaceConsumed()/oldReplication[0]);
+      updateCount(inodes, inodes.length-1, 0, dsDelta);
+
+      fileNode.setReplication(replication);
+      fileBlocks = fileNode.getBlocks();
+    }
+    return fileBlocks;
+  }
+
+  /**
+   * Get the blocksize of a file
+   * @param filename the filename
+   * @return the number of bytes 
+   * @throws IOException if it is a directory or does not exist.
+   */
+  long getPreferredBlockSize(String filename) throws IOException {
+    synchronized (rootDir) {
+      INode fileNode = rootDir.getNode(filename);
+      if (fileNode == null) {
+        throw new FileNotFoundException("File does not exist: " + filename);
+      }
+      if (fileNode.isDirectory()) {
+        throw new IOException("Getting block size of a directory: " + 
+                              filename);
+      }
+      return ((INodeFile)fileNode).getPreferredBlockSize();
+    }
+  }
+
+  boolean exists(String src) {
+    src = normalizePath(src);
+    synchronized(rootDir) {
+      INode inode = rootDir.getNode(src);
+      if (inode == null) {
+         return false;
+      }
+      return inode.isDirectory()? true: ((INodeFile)inode).getBlocks() != null;
+    }
+  }
+
+  void setPermission(String src, FsPermission permission
+      ) throws FileNotFoundException {
+    unprotectedSetPermission(src, permission);
+    fsImage.getEditLog().logSetPermissions(src, permission);
+  }
+
+  void unprotectedSetPermission(String src, FsPermission permissions
+      ) throws FileNotFoundException {
+    synchronized(rootDir) {
+        INode inode = rootDir.getNode(src);
+        if(inode == null)
+            throw new FileNotFoundException("File does not exist: " + src);
+        inode.setPermission(permissions);
+    }
+  }
+
+  void setOwner(String src, String username, String groupname
+      ) throws FileNotFoundException {
+    unprotectedSetOwner(src, username, groupname);
+    fsImage.getEditLog().logSetOwner(src, username, groupname);
+  }
+
+  void unprotectedSetOwner(String src, String username, String groupname
+      ) throws FileNotFoundException {
+    synchronized(rootDir) {
+      INode inode = rootDir.getNode(src);
+      if(inode == null)
+          throw new FileNotFoundException("File does not exist: " + src);
+      if (username != null) {
+        inode.setUser(username);
+      }
+      if (groupname != null) {
+        inode.setGroup(groupname);
+      }
+    }
+  }
+    
+  /**
+   * Remove the file from management, return blocks
+   */
+  INode delete(String src) {
+    if (NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("DIR* FSDirectory.delete: "+src);
+    }
+    waitForReady();
+    long now = FSNamesystem.now();
+    INode deletedNode = unprotectedDelete(src, now);
+    if (deletedNode != null) {
+      fsImage.getEditLog().logDelete(src, now);
+    }
+    return deletedNode;
+  }
+  
+  /** Return if a directory is empty or not **/
+  boolean isDirEmpty(String src) {
+	   boolean dirNotEmpty = true;
+    if (!isDir(src)) {
+      return true;
+    }
+    synchronized(rootDir) {
+      INode targetNode = rootDir.getNode(src);
+      assert targetNode != null : "should be taken care in isDir() above";
+      if (((INodeDirectory)targetNode).getChildren().size() != 0) {
+        dirNotEmpty = false;
+      }
+    }
+    return dirNotEmpty;
+  }
+
+  boolean isEmpty() {
+    return isDirEmpty("/");
+  }
+
+  /**
+   * Delete a path from the name space
+   * Update the count at each ancestor directory with quota
+   * @param src a string representation of a path to an inode
+   * @param modificationTime the time the inode is removed
+   * @param deletedBlocks the place holder for the blocks to be removed
+   * @return if the deletion succeeds
+   */ 
+  INode unprotectedDelete(String src, long modificationTime) {
+    src = normalizePath(src);
+
+    synchronized (rootDir) {
+      INode[] inodes =  rootDir.getExistingPathINodes(src);
+      INode targetNode = inodes[inodes.length-1];
+
+      if (targetNode == null) { // non-existent src
+        NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
+            +"failed to remove "+src+" because it does not exist");
+        return null;
+      } else if (inodes.length == 1) { // src is the root
+        NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedDelete: " +
+            "failed to remove " + src +
+            " because the root is not allowed to be deleted");
+        return null;
+      } else {
+        try {
+          // Remove the node from the namespace
+          removeChild(inodes, inodes.length-1);
+          // set the parent's modification time
+          inodes[inodes.length-2].setModificationTime(modificationTime);
+          // GC all the blocks underneath the node.
+          ArrayList<Block> v = new ArrayList<Block>();
+          int filesRemoved = targetNode.collectSubtreeBlocksAndClear(v);
+          incrDeletedFileCount(filesRemoved);
+          getFSNamesystem().removePathAndBlocks(src, v);
+          if (NameNode.stateChangeLog.isDebugEnabled()) {
+            NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
+              +src+" is removed");
+          }
+          return targetNode;
+        } catch(QuotaExceededException e) {
+          NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedDelete: " +
+              "failed to remove " + src + " because " + e.getMessage());
+          return null;
+        }
+      }
+    }
+  }
+
+  /**
+   * Replaces the specified inode with the specified one.
+   */
+  void replaceNode(String path, INodeFile oldnode, INodeFile newnode) 
+                                                   throws IOException {
+    replaceNode(path, oldnode, newnode, true);
+  }
+  
+  /**
+   * @see #replaceNode(String, INodeFile, INodeFile)
+   */
+  private void replaceNode(String path, INodeFile oldnode, INodeFile newnode,
+                           boolean updateDiskspace) throws IOException {    
+    synchronized (rootDir) {
+      long dsOld = oldnode.diskspaceConsumed();
+      
+      //
+      // Remove the node from the namespace 
+      //
+      if (!oldnode.removeNode()) {
+        NameNode.stateChangeLog.warn("DIR* FSDirectory.replaceNode: " +
+                                     "failed to remove " + path);
+        throw new IOException("FSDirectory.replaceNode: " +
+                              "failed to remove " + path);
+      } 
+      
+      /* Currently oldnode and newnode are assumed to contain the same
+       * blocks. Otherwise, blocks need to be removed from the blocksMap.
+       */
+      
+      rootDir.addNode(path, newnode); 
+
+      //check if disk space needs to be updated.
+      long dsNew = 0;
+      if (updateDiskspace && (dsNew = newnode.diskspaceConsumed()) != dsOld) {
+        try {
+          updateSpaceConsumed(path, 0, dsNew-dsOld);
+        } catch (QuotaExceededException e) {
+          // undo
+          replaceNode(path, newnode, oldnode, false);
+          throw e;
+        }
+      }
+      
+      int index = 0;
+      for (Block b : newnode.getBlocks()) {
+        BlockInfo info = getBlockManager().addINode(b, newnode);
+        newnode.setBlock(index, info); // inode refers to the block in BlocksMap
+        index++;
+      }
+    }
+  }
+
+  /**
+   * Get a listing of files given path 'src'
+   *
+   * This function is admittedly very inefficient right now.  We'll
+   * make it better later.
+   */
+  FileStatus[] getListing(String src) {
+    String srcs = normalizePath(src);
+
+    synchronized (rootDir) {
+      INode targetNode = rootDir.getNode(srcs);
+      if (targetNode == null)
+        return null;
+      if (!targetNode.isDirectory()) {
+        return new FileStatus[]{createFileStatus(srcs, targetNode)};
+      }
+      List<INode> contents = ((INodeDirectory)targetNode).getChildren();
+      FileStatus listing[] = new FileStatus[contents.size()];
+      if(! srcs.endsWith(Path.SEPARATOR))
+        srcs += Path.SEPARATOR;
+      int i = 0;
+      for (INode cur : contents) {
+        listing[i] = createFileStatus(srcs+cur.getLocalName(), cur);
+        i++;
+      }
+      return listing;
+    }
+  }
+
+  /** Get the file info for a specific file.
+   * @param src The string representation of the path to the file
+   * @return object containing information regarding the file
+   *         or null if file not found
+   */
+  FileStatus getFileInfo(String src) {
+    String srcs = normalizePath(src);
+    synchronized (rootDir) {
+      INode targetNode = rootDir.getNode(srcs);
+      if (targetNode == null) {
+        return null;
+      }
+      else {
+        return createFileStatus(srcs, targetNode);
+      }
+    }
+  }
+
+  /**
+   * Get the blocks associated with the file.
+   */
+  Block[] getFileBlocks(String src) {
+    waitForReady();
+    synchronized (rootDir) {
+      INode targetNode = rootDir.getNode(src);
+      if (targetNode == null)
+        return null;
+      if(targetNode.isDirectory())
+        return null;
+      return ((INodeFile)targetNode).getBlocks();
+    }
+  }
+
+  /**
+   * Get {@link INode} associated with the file.
+   */
+  INodeFile getFileINode(String src) {
+    synchronized (rootDir) {
+      INode inode = rootDir.getNode(src);
+      if (inode == null || inode.isDirectory())
+        return null;
+      return (INodeFile)inode;
+    }
+  }
+
+  /**
+   * Retrieve the existing INodes along the given path.
+   * 
+   * @param path the path to explore
+   * @return INodes array containing the existing INodes in the order they
+   *         appear when following the path from the root INode to the
+   *         deepest INodes. The array size will be the number of expected
+   *         components in the path, and non existing components will be
+   *         filled with null
+   *         
+   * @see INodeDirectory#getExistingPathINodes(byte[][], INode[])
+   */
+  INode[] getExistingPathINodes(String path) {
+    synchronized (rootDir){
+      return rootDir.getExistingPathINodes(path);
+    }
+  }
+  
+  /** 
+   * Check whether the filepath could be created
+   */
+  boolean isValidToCreate(String src) {
+    String srcs = normalizePath(src);
+    synchronized (rootDir) {
+      if (srcs.startsWith("/") && 
+          !srcs.endsWith("/") && 
+          rootDir.getNode(srcs) == null) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * Check whether the path specifies a directory
+   */
+  boolean isDir(String src) {
+    synchronized (rootDir) {
+      INode node = rootDir.getNode(normalizePath(src));
+      return node != null && node.isDirectory();
+    }
+  }
+
+  /** Updates namespace and diskspace consumed for all
+   * directories until the parent directory of file represented by path.
+   * 
+   * @param path path for the file.
+   * @param nsDelta the delta change of namespace
+   * @param dsDelta the delta change of diskspace
+   * @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)
+                                         throws QuotaExceededException,
+                                                FileNotFoundException {
+    synchronized (rootDir) {
+      INode[] inodes = rootDir.getExistingPathINodes(path);
+      int len = inodes.length;
+      if (inodes[len - 1] == null) {
+        throw new FileNotFoundException(path + 
+                                        " does not exist under rootDir.");
+      }
+      updateCount(inodes, len-1, nsDelta, dsDelta);
+    }
+  }
+  
+  /** update count of each inode with quota
+   * 
+   * @param inodes an array of inodes on a path
+   * @param numOfINodes the number of inodes to update starting from index 0
+   * @param nsDelta the delta change of namespace
+   * @param dsDelta the delta change of diskspace
+   * @throws QuotaExceededException if the new count violates any quota limit
+   */
+  private void updateCount(INode[] inodes, int numOfINodes, 
+                           long nsDelta, long dsDelta)
+                           throws QuotaExceededException {
+    if (!ready) {
+      //still initializing. do not check or update quotas.
+      return;
+    }
+    if (numOfINodes>inodes.length) {
+      numOfINodes = inodes.length;
+    }
+    // check existing components in the path  
+    int i=0;
+    try {
+      for(; i < numOfINodes; i++) {
+        if (inodes[i].isQuotaSet()) { // a directory with quota
+          INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i]; 
+          node.updateNumItemsInTree(nsDelta, dsDelta);
+        }
+      }
+    } catch (QuotaExceededException e) {
+      e.setPathName(getFullPathName(inodes, i));
+      // undo updates
+      for( ; i-- > 0; ) {
+        try {
+          if (inodes[i].isQuotaSet()) { // a directory with quota
+            INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i]; 
+            node.updateNumItemsInTree(-nsDelta, -dsDelta);
+          }
+        } catch (IOException ingored) {
+        }
+      }
+      throw e;
+    }
+  }
+  
+  /** Return the name of the path represented by inodes at [0, pos] */
+  private static String getFullPathName(INode[] inodes, int pos) {
+    StringBuilder fullPathName = new StringBuilder();
+    for (int i=1; i<=pos; i++) {
+      fullPathName.append(Path.SEPARATOR_CHAR).append(inodes[i].getLocalName());
+    }
+    return fullPathName.toString();
+  }
+  
+  /**
+   * Create a directory 
+   * If ancestor directories do not exist, automatically create them.
+
+   * @param src string representation of the path to the directory
+   * @param permissions the permission of the directory
+   * @param inheritPermission if the permission of the directory should inherit
+   *                          from its parent or not. The automatically created
+   *                          ones always inherit its permission from its parent
+   * @param now creation time
+   * @return true if the operation succeeds false otherwise
+   * @throws FileNotFoundException if an ancestor or itself is a file
+   * @throws QuotaExceededException if directory creation violates 
+   *                                any quota limit
+   */
+  boolean mkdirs(String src, PermissionStatus permissions,
+      boolean inheritPermission, long now)
+      throws FileNotFoundException, QuotaExceededException {
+    src = normalizePath(src);
+    String[] names = INode.getPathNames(src);
+    byte[][] components = INode.getPathComponents(names);
+    INode[] inodes = new INode[components.length];
+
+    synchronized(rootDir) {
+      rootDir.getExistingPathINodes(components, inodes);
+
+      // find the index of the first null in inodes[]
+      StringBuilder pathbuilder = new StringBuilder();
+      int i = 1;
+      for(; i < inodes.length && inodes[i] != null; i++) {
+        pathbuilder.append(Path.SEPARATOR + names[i]);
+        if (!inodes[i].isDirectory()) {
+          throw new FileNotFoundException("Parent path is not a directory: "
+              + pathbuilder);
+        }
+      }
+
+      // create directories beginning from the first null index
+      for(; i < inodes.length; i++) {
+        pathbuilder.append(Path.SEPARATOR + names[i]);
+        String cur = pathbuilder.toString();
+        unprotectedMkdir(inodes, i, components[i], permissions,
+            inheritPermission || i != components.length-1, now);
+        if (inodes[i] == null) {
+          return false;
+        }
+        // Directory creation also count towards FilesCreated
+        // to match count of files_deleted metric. 
+        if (getFSNamesystem() != null)
+          NameNode.getNameNodeMetrics().numFilesCreated.inc();
+        fsImage.getEditLog().logMkDir(cur, inodes[i]);
+        NameNode.stateChangeLog.debug(
+            "DIR* FSDirectory.mkdirs: created directory " + cur);
+      }
+    }
+    return true;
+  }
+
+  /**
+   */
+  INode unprotectedMkdir(String src, PermissionStatus permissions,
+                          long timestamp) throws QuotaExceededException {
+    byte[][] components = INode.getPathComponents(src);
+    INode[] inodes = new INode[components.length];
+    synchronized (rootDir) {
+      rootDir.getExistingPathINodes(components, inodes);
+      unprotectedMkdir(inodes, inodes.length-1, components[inodes.length-1],
+          permissions, false, timestamp);
+      return inodes[inodes.length-1];
+    }
+  }
+
+  /** create a directory at index pos.
+   * The parent path to the directory is at [0, pos-1].
+   * All ancestors exist. Newly created one stored at index pos.
+   */
+  private void unprotectedMkdir(INode[] inodes, int pos,
+      byte[] name, PermissionStatus permission, boolean inheritPermission,
+      long timestamp) throws QuotaExceededException {
+    inodes[pos] = addChild(inodes, pos, 
+        new INodeDirectory(name, permission, timestamp),
+        inheritPermission );
+  }
+  
+  /** Add a node child to the namespace. The full path name of the node is src.
+   * childDiskspace should be -1, if unknown. 
+   * QuotaExceededException is thrown if it violates quota limit */
+  private <T extends INode> T addNode(String src, T child, 
+        long childDiskspace, boolean inheritPermission) 
+  throws QuotaExceededException {
+    byte[][] components = INode.getPathComponents(src);
+    child.setLocalName(components[components.length-1]);
+    INode[] inodes = new INode[components.length];
+    synchronized (rootDir) {
+      rootDir.getExistingPathINodes(components, inodes);
+      return addChild(inodes, inodes.length-1, child, childDiskspace,
+                      inheritPermission);
+    }
+  }
+  
+  /** Add a node child to the inodes at index pos. 
+   * Its ancestors are stored at [0, pos-1]. 
+   * QuotaExceededException is thrown if it violates quota limit */
+  private <T extends INode> T addChild(INode[] pathComponents, int pos, T child,
+      boolean inheritPermission) throws QuotaExceededException {
+    return addChild(pathComponents, pos, child, -1, inheritPermission);
+  }
+  
+  /** Add a node child to the inodes at index pos. 
+   * Its ancestors are stored at [0, pos-1]. 
+   * QuotaExceededException is thrown if it violates quota limit */
+  private <T extends INode> T addChild(INode[] pathComponents, int pos, T child,
+       long childDiskspace, boolean inheritPermission) throws QuotaExceededException {
+    INode.DirCounts counts = new INode.DirCounts();
+    child.spaceConsumedInTree(counts);
+    if (childDiskspace < 0) {
+      childDiskspace = counts.getDsCount();
+    }
+    updateCount(pathComponents, pos, counts.getNsCount(), childDiskspace);
+    T addedNode = ((INodeDirectory)pathComponents[pos-1]).addChild(
+        child, inheritPermission);
+    if (addedNode == null) {
+      updateCount(pathComponents, pos, 
+                  -counts.getNsCount(), -childDiskspace);
+    }
+    return addedNode;
+  }
+  
+  /** Remove an inode at index pos from the namespace.
+   * Its ancestors are stored at [0, pos-1].
+   * Count of each ancestor with quota is also updated.
+   * Return the removed node; null if the removal fails.
+   */
+  private INode removeChild(INode[] pathComponents, int pos)
+  throws QuotaExceededException {
+    INode removedNode = 
+      ((INodeDirectory)pathComponents[pos-1]).removeChild(pathComponents[pos]);
+    if (removedNode != null) {
+      INode.DirCounts counts = new INode.DirCounts();
+      removedNode.spaceConsumedInTree(counts);
+      updateCount(pathComponents, pos,
+                  -counts.getNsCount(), -counts.getDsCount());
+    }
+    return removedNode;
+  }
+  
+  /**
+   */
+  String normalizePath(String src) {
+    if (src.length() > 1 && src.endsWith("/")) {
+      src = src.substring(0, src.length() - 1);
+    }
+    return src;
+  }
+
+  ContentSummary getContentSummary(String src) throws FileNotFoundException {
+    String srcs = normalizePath(src);
+    synchronized (rootDir) {
+      INode targetNode = rootDir.getNode(srcs);
+      if (targetNode == null) {
+        throw new FileNotFoundException("File does not exist: " + srcs);
+      }
+      else {
+        return targetNode.computeContentSummary();
+      }
+    }
+  }
+
+  /** Update the count of each directory with quota in the namespace
+   * A directory's count is defined as the total number inodes in the tree
+   * rooted at the directory.
+   * 
+   * This is an update of existing state of the filesystem and does not
+   * throw QuotaExceededException.
+   */
+  void updateCountForINodeWithQuota() {
+    updateCountForINodeWithQuota(rootDir, new INode.DirCounts(), 
+                                 new ArrayList<INode>(50));
+  }
+  
+  /** 
+   * Update the count of the directory if it has a quota and return the count
+   * 
+   * This does not throw a QuotaExceededException. This is just an update
+   * of of existing state and throwing QuotaExceededException does not help
+   * with fixing the state, if there is a problem.
+   * 
+   * @param dir the root of the tree that represents the directory
+   * @param counters counters for name space and disk space
+   * @param nodesInPath INodes for the each of components in the path.
+   * @return the size of the tree
+   */
+  private static void updateCountForINodeWithQuota(INodeDirectory dir, 
+                                               INode.DirCounts counts,
+                                               ArrayList<INode> nodesInPath) {
+    long parentNamespace = counts.nsCount;
+    long parentDiskspace = counts.dsCount;
+    
+    counts.nsCount = 1L;//for self. should not call node.spaceConsumedInTree()
+    counts.dsCount = 0L;
+    
+    /* We don't need nodesInPath if we could use 'parent' field in 
+     * INode. using 'parent' is not currently recommended. */
+    nodesInPath.add(dir);
+
+    for (INode child : dir.getChildren()) {
+      if (child.isDirectory()) {
+        updateCountForINodeWithQuota((INodeDirectory)child, 
+                                     counts, nodesInPath);
+      } else { // reduce recursive calls
+        counts.nsCount += 1;
+        counts.dsCount += ((INodeFile)child).diskspaceConsumed();
+      }
+    }
+      
+    if (dir.isQuotaSet()) {
+      ((INodeDirectoryWithQuota)dir).setSpaceConsumed(counts.nsCount,
+                                                      counts.dsCount);
+
+      // check if quota is violated for some reason.
+      if ((dir.getNsQuota() >= 0 && counts.nsCount > dir.getNsQuota()) ||
+          (dir.getDsQuota() >= 0 && counts.dsCount > dir.getDsQuota())) {
+
+        // can only happen because of a software bug. the bug should be fixed.
+        StringBuilder path = new StringBuilder(512);
+        for (INode n : nodesInPath) {
+          path.append('/');
+          path.append(n.getLocalName());
+        }
+        
+        NameNode.LOG.warn("Unexpected quota violation in image for " + path + 
+                          " (Namespace quota : " + dir.getNsQuota() +
+                          " consumed : " + counts.nsCount + ")" +
+                          " (Diskspace quota : " + dir.getDsQuota() +
+                          " consumed : " + counts.dsCount + ").");
+      }            
+    }
+      
+    // pop 
+    nodesInPath.remove(nodesInPath.size()-1);
+    
+    counts.nsCount += parentNamespace;
+    counts.dsCount += parentDiskspace;
+  }
+  
+  /**
+   * See {@link ClientProtocol#setQuota(String, long, long)} for the contract.
+   * Sets quota for for a directory.
+   * @returns INodeDirectory if any of the quotas have changed. null other wise.
+   * @throws FileNotFoundException if the path does not exist or is a file
+   * @throws QuotaExceededException if the directory tree size is 
+   *                                greater than the given quota
+   */
+  INodeDirectory unprotectedSetQuota(String src, long nsQuota, long dsQuota) 
+                       throws FileNotFoundException, QuotaExceededException {
+    // sanity check
+    if ((nsQuota < 0 && nsQuota != FSConstants.QUOTA_DONT_SET && 
+         nsQuota < FSConstants.QUOTA_RESET) || 
+        (dsQuota < 0 && dsQuota != FSConstants.QUOTA_DONT_SET && 
+          dsQuota < FSConstants.QUOTA_RESET)) {
+      throw new IllegalArgumentException("Illegal value for nsQuota or " +
+                                         "dsQuota : " + nsQuota + " and " +
+                                         dsQuota);
+    }
+    
+    String srcs = normalizePath(src);
+
+    synchronized(rootDir) {
+      INode[] inodes = rootDir.getExistingPathINodes(src);
+      INode targetNode = inodes[inodes.length-1];
+      if (targetNode == null) {
+        throw new FileNotFoundException("Directory does not exist: " + srcs);
+      } else if (!targetNode.isDirectory()) {
+        throw new FileNotFoundException("Cannot set quota on a file: " + srcs);  
+      } else { // a directory inode
+        INodeDirectory dirNode = (INodeDirectory)targetNode;
+        long oldNsQuota = dirNode.getNsQuota();
+        long oldDsQuota = dirNode.getDsQuota();
+        if (nsQuota == FSConstants.QUOTA_DONT_SET) {
+          nsQuota = oldNsQuota;
+        }
+        if (dsQuota == FSConstants.QUOTA_DONT_SET) {
+          dsQuota = oldDsQuota;
+        }        
+
+        if (dirNode instanceof INodeDirectoryWithQuota) { 
+          // a directory with quota; so set the quota to the new value
+          ((INodeDirectoryWithQuota)dirNode).setQuota(nsQuota, dsQuota);
+        } else {
+          // a non-quota directory; so replace it with a directory with quota
+          INodeDirectoryWithQuota newNode = 
+            new INodeDirectoryWithQuota(nsQuota, dsQuota, dirNode);
+          // non-root directory node; parent != null
+          INodeDirectory parent = (INodeDirectory)inodes[inodes.length-2];
+          dirNode = newNode;
+          parent.replaceChild(newNode);
+        }
+        return (oldNsQuota != nsQuota || oldDsQuota != dsQuota) ? dirNode : null;
+      }
+    }
+  }
+  
+  /**
+   * See {@link ClientProtocol#setQuota(String, long, long)} for the 
+   * contract.
+   * @see #unprotectedSetQuota(String, long, long)
+   */
+  void setQuota(String src, long nsQuota, long dsQuota) 
+                throws FileNotFoundException, QuotaExceededException {
+    synchronized (rootDir) {    
+      INodeDirectory dir = unprotectedSetQuota(src, nsQuota, dsQuota);
+      if (dir != null) {
+        fsImage.getEditLog().logSetQuota(src, dir.getNsQuota(), 
+                                         dir.getDsQuota());
+      }
+    }
+  }
+  
+  long totalInodes() {
+    synchronized (rootDir) {
+      return rootDir.numItemsInTree();
+    }
+  }
+
+  /**
+   * Sets the access time on the file. Logs it in the transaction log
+   */
+  void setTimes(String src, INodeFile inode, long mtime, long atime, boolean force) {
+    if (unprotectedSetTimes(src, inode, mtime, atime, force)) {
+      fsImage.getEditLog().logTimes(src, mtime, atime);
+    }
+  }
+
+  boolean unprotectedSetTimes(String src, long mtime, long atime, boolean force) {
+    synchronized(rootDir) {
+      INodeFile inode = getFileINode(src);
+      return unprotectedSetTimes(src, inode, mtime, atime, force);
+    }
+  }
+
+  private boolean unprotectedSetTimes(String src, INodeFile inode, long mtime,
+                                      long atime, boolean force) {
+    boolean status = false;
+    if (mtime != -1) {
+      inode.setModificationTimeForce(mtime);
+      status = true;
+    }
+    if (atime != -1) {
+      long inodeTime = inode.getAccessTime();
+
+      // if the last access time update was within the last precision interval, then
+      // no need to store access time
+      if (atime <= inodeTime + getFSNamesystem().getAccessTimePrecision() && !force) {
+        status =  false;
+      } else {
+        inode.setAccessTime(atime);
+        status = true;
+      }
+    } 
+    return status;
+  }
+
+  /**
+   * Reset the entire namespace tree.
+   */
+  void reset() {
+    rootDir = new INodeDirectoryWithQuota(INodeDirectory.ROOT_NAME,
+        getFSNamesystem().createFsOwnerPermissions(new FsPermission((short)0755)),
+        Integer.MAX_VALUE, -1);
+  }
+
+  /**
+   * Create FileStatus by file INode 
+   */
+   private static FileStatus createFileStatus(String path, INode node) {
+    // length is zero for directories
+    return new FileStatus(node.isDirectory() ? 0 : node.computeContentSummary().getLength(), 
+        node.isDirectory(), 
+        node.isDirectory() ? 0 : ((INodeFile)node).getReplication(), 
+        node.isDirectory() ? 0 : ((INodeFile)node).getPreferredBlockSize(),
+        node.getModificationTime(),
+        node.getAccessTime(),
+        node.getFsPermission(),
+        node.getUserName(),
+        node.getGroupName(),
+        new Path(path));
+  }
+}

+ 1434 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -0,0 +1,1434 @@
+/**
+ * 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 java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.EOFException;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+
+import org.apache.hadoop.fs.FileStatus;
+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.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
+import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeFile;
+import org.apache.hadoop.hdfs.server.namenode.JournalStream.JournalType;
+import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.DeprecatedUTF8;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+import org.mortbay.log.Log;
+
+/**
+ * FSEditLog maintains a log of the namespace modifications.
+ * 
+ */
+public class FSEditLog {
+  public  static final byte OP_INVALID = -1;
+  private static final byte OP_ADD = 0;
+  private static final byte OP_RENAME = 1;  // rename
+  private static final byte OP_DELETE = 2;  // delete
+  private static final byte OP_MKDIR = 3;   // create directory
+  private static final byte OP_SET_REPLICATION = 4; // set replication
+  //the following two are used only for backward compatibility :
+  @Deprecated private static final byte OP_DATANODE_ADD = 5;
+  @Deprecated private static final byte OP_DATANODE_REMOVE = 6;
+  private static final byte OP_SET_PERMISSIONS = 7;
+  private static final byte OP_SET_OWNER = 8;
+  private static final byte OP_CLOSE = 9;    // close after write
+  private static final byte OP_SET_GENSTAMP = 10;    // store genstamp
+  /* The following two are not used any more. Should be removed once
+   * LAST_UPGRADABLE_LAYOUT_VERSION is -17 or newer. */
+  private static final byte OP_SET_NS_QUOTA = 11; // set namespace quota
+  private static final byte OP_CLEAR_NS_QUOTA = 12; // clear namespace quota
+  private static final byte OP_TIMES = 13; // sets mod & access time on a file
+  private static final byte OP_SET_QUOTA = 14; // sets name and disk quotas.
+  /* 
+   * The following operations are used to control remote edit log streams,
+   * and not logged into file streams.
+   */
+  static final byte OP_JSPOOL_START = // start journal spool
+                                    NamenodeProtocol.JA_JSPOOL_START;
+  static final byte OP_CHECKPOINT_TIME = // incr checkpoint time
+                                    NamenodeProtocol.JA_CHECKPOINT_TIME;
+
+  static final String NO_JOURNAL_STREAMS_WARNING = "!!! WARNING !!!" +
+  		" File system changes are not persistent. No journal streams.";
+
+  private volatile int sizeOutputFlushBuffer = 512*1024;
+
+  private ArrayList<EditLogOutputStream> editStreams = null;
+  private FSImage fsimage = null;
+
+  // a monotonically increasing counter that represents transactionIds.
+  private long txid = 0;
+
+  // stores the last synced transactionId.
+  private long synctxid = 0;
+
+  // the time of printing the statistics to the log file.
+  private long lastPrintTime;
+
+  // is a sync currently running?
+  private boolean isSyncRunning;
+
+  // these are statistics counters.
+  private long numTransactions;        // number of transactions
+  private long numTransactionsBatchedInSync;
+  private long totalTimeTransactions;  // total time for all transactions
+  private NameNodeMetrics metrics;
+
+  private static class TransactionId {
+    public long txid;
+
+    TransactionId(long value) {
+      this.txid = value;
+    }
+  }
+
+  // stores the most current transactionId of this thread.
+  private static final ThreadLocal<TransactionId> myTransactionId = new ThreadLocal<TransactionId>() {
+    protected synchronized TransactionId initialValue() {
+      return new TransactionId(Long.MAX_VALUE);
+    }
+  };
+
+  FSEditLog(FSImage image) {
+    fsimage = image;
+    isSyncRunning = false;
+    metrics = NameNode.getNameNodeMetrics();
+    lastPrintTime = FSNamesystem.now();
+  }
+  
+  private File getEditFile(StorageDirectory sd) {
+    return fsimage.getEditFile(sd);
+  }
+  
+  private File getEditNewFile(StorageDirectory sd) {
+    return fsimage.getEditNewFile(sd);
+  }
+  
+  private int getNumEditsDirs() {
+   return fsimage.getNumStorageDirs(NameNodeDirType.EDITS);
+  }
+
+  synchronized int getNumEditStreams() {
+    return editStreams == null ? 0 : editStreams.size();
+  }
+
+  boolean isOpen() {
+    return getNumEditStreams() > 0;
+  }
+
+  /**
+   * Create empty edit log files.
+   * Initialize the output stream for logging.
+   * 
+   * @throws IOException
+   */
+  synchronized void open() throws IOException {
+    numTransactions = totalTimeTransactions = numTransactionsBatchedInSync = 0;
+    if (editStreams == null)
+      editStreams = new ArrayList<EditLogOutputStream>();
+    
+    ArrayList<StorageDirectory> al = null;
+    for (Iterator<StorageDirectory> it = 
+           fsimage.dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      File eFile = getEditFile(sd);
+      try {
+        addNewEditLogStream(eFile);
+      } catch (IOException e) {
+        FSNamesystem.LOG.warn("Unable to open edit log file " + eFile);
+        // Remove the directory from list of storage directories
+        if(al == null) al = new ArrayList<StorageDirectory>(1);
+        al.add(sd);
+        
+      }
+    }
+    
+    if(al != null) fsimage.processIOError(al, false);
+  }
+  
+  
+  synchronized void addNewEditLogStream(File eFile) throws IOException {
+    EditLogOutputStream eStream = new EditLogFileOutputStream(eFile,
+        sizeOutputFlushBuffer);
+    editStreams.add(eStream);
+  }
+
+  synchronized void createEditLogFile(File name) throws IOException {
+    EditLogOutputStream eStream = new EditLogFileOutputStream(name,
+        sizeOutputFlushBuffer);
+    eStream.create();
+    eStream.close();
+  }
+
+  /**
+   * Create edits.new if non existent.
+   */
+  synchronized void createNewIfMissing() throws IOException {
+ for (Iterator<StorageDirectory> it = 
+       fsimage.dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
+      File newFile = getEditNewFile(it.next());
+      if (!newFile.exists())
+        createEditLogFile(newFile);
+    }
+  }
+  
+  /**
+   * Shutdown the file store.
+   */
+  public synchronized void close() {
+    while (isSyncRunning) {
+      try {
+        wait(1000);
+      } catch (InterruptedException ie) { 
+      }
+    }
+    if (editStreams == null || editStreams.isEmpty()) {
+      return;
+    }
+    printStatistics(true);
+    numTransactions = totalTimeTransactions = numTransactionsBatchedInSync = 0;
+
+    ArrayList<EditLogOutputStream> errorStreams = null;
+    Iterator<EditLogOutputStream> it = getOutputStreamIterator(null);
+    while(it.hasNext()) {
+      EditLogOutputStream eStream = it.next();
+      try {
+        closeStream(eStream);
+      } catch (IOException e) {
+        FSNamesystem.LOG.warn("FSEditLog:close - failed to close stream " 
+            + eStream.getName());
+        if(errorStreams == null)
+          errorStreams = new ArrayList<EditLogOutputStream>(1);
+        errorStreams.add(eStream);
+      }
+    }
+    processIOError(errorStreams, true);
+    editStreams.clear();
+  }
+
+  /**
+   * Close and remove edit log stream.
+   * @param index of the stream
+   */
+  synchronized private void removeStream(int index) {
+    EditLogOutputStream eStream = editStreams.get(index);
+    try {
+      eStream.close();
+    } catch (Exception e) {}
+    editStreams.remove(index);
+  }
+
+  /**
+   * The specified streams have IO errors. Close and remove them.
+   * If propagate is true - close related StorageDirectories.
+   * (is called with propagate value true from everywhere
+   *  except fsimage.processIOError)
+   */
+  synchronized void processIOError(
+      ArrayList<EditLogOutputStream> errorStreams,
+      boolean propagate) {
+    
+    String lsd = fsimage.listStorageDirectories();
+    FSNamesystem.LOG.info("current list of storage dirs:" + lsd);
+    
+    if (errorStreams == null || errorStreams.size() == 0) {
+      return;                       // nothing to do
+    }
+
+    //EditLogOutputStream
+    if (editStreams == null || editStreams.size() <= 1) {
+      FSNamesystem.LOG.fatal(
+      "Fatal Error : All storage directories are inaccessible."); 
+      Runtime.getRuntime().exit(-1);
+    }
+
+    ArrayList<StorageDirectory> al = null;
+    for (EditLogOutputStream eStream : errorStreams) {
+      FSNamesystem.LOG.error("Unable to log edits to " + eStream.getName()
+          + "; removing it");     
+
+      StorageDirectory storageDir;
+      if(propagate && eStream.getType() == JournalType.FILE && //find SD
+          (storageDir = getStorage(eStream)) != null) {
+        FSNamesystem.LOG.info("about to remove corresponding storage:" 
+            + storageDir.getRoot().getAbsolutePath());
+        // remove corresponding storage dir
+        if(al == null) al = new ArrayList<StorageDirectory>(1);
+        al.add(storageDir);
+      }
+      Iterator<EditLogOutputStream> ies = editStreams.iterator();
+      while (ies.hasNext()) {
+        EditLogOutputStream es = ies.next();
+        if (es == eStream) {  
+          try { eStream.close(); } catch (IOException e) {
+            // nothing to do.
+            FSNamesystem.LOG.warn("Failed to close eStream " + eStream.getName()
+                + " before removing it (might be ok)");
+          }
+          ies.remove();
+          break;
+        }
+      } 
+    }
+    
+    // removed failed SDs
+    if(propagate && al != null) fsimage.processIOError(al, false);
+    
+    //for the rest of the streams
+    if(propagate) incrementCheckpointTime();
+    
+    lsd = fsimage.listStorageDirectories();
+    FSNamesystem.LOG.info("at the end current list of storage dirs:" + lsd);
+  }
+
+
+  /**
+   * get an editStream corresponding to a sd
+   * @param es - stream to remove
+   * @return the matching stream
+   */
+  public StorageDirectory getStorage(EditLogOutputStream es) {
+    String parentStorageDir = ((EditLogFileOutputStream)es).getFile()
+    .getParentFile().getParentFile().getAbsolutePath();
+
+    Iterator<StorageDirectory> it = fsimage.dirIterator(); 
+    while (it.hasNext()) {
+      StorageDirectory sd = it.next();
+      FSNamesystem.LOG.info("comparing: " + parentStorageDir + " and " + sd.getRoot().getAbsolutePath()); 
+      if (parentStorageDir.equals(sd.getRoot().getAbsolutePath()))
+        return sd;
+    }
+    return null;
+  }
+  
+  /**
+   * get an editStream corresponding to a sd
+   * @param sd
+   * @return the matching stream
+   */
+  public EditLogOutputStream getEditsStream(StorageDirectory sd) {
+	for (EditLogOutputStream es : editStreams) {
+	  File parentStorageDir = ((EditLogFileOutputStream)es).getFile()
+	  .getParentFile().getParentFile();
+	  if (parentStorageDir.getName().equals(sd.getRoot().getName()))
+		return es;
+	}
+	return null;
+  }
+
+  /**
+   * check if edits.new log exists in the specified stoorage directory
+   */
+  boolean existsNew(StorageDirectory sd) {
+    return getEditNewFile(sd).exists(); 
+  }
+
+  /**
+   * Load an edit log, and apply the changes to the in-memory structure
+   * This is where we apply edits that we've been writing to disk all
+   * along.
+   */
+  int loadFSEdits(EditLogInputStream edits) throws IOException {
+    DataInputStream in = edits.getDataInputStream();
+    long startTime = FSNamesystem.now();
+    int numEdits = loadFSEdits(in, true);
+    FSImage.LOG.info("Edits file " + edits.getName() 
+        + " of size " + edits.length() + " edits # " + numEdits 
+        + " loaded in " + (FSNamesystem.now()-startTime)/1000 + " seconds.");
+    return numEdits;
+  }
+
+  int loadFSEdits(DataInputStream in, boolean closeOnExit) throws IOException {
+    int numEdits = 0;
+    int logVersion = 0;
+
+    try {
+      // Read log file version. Could be missing. 
+      in.mark(4);
+      // If edits log is greater than 2G, available method will return negative
+      // numbers, so we avoid having to call available
+      boolean available = true;
+      try {
+        logVersion = in.readByte();
+      } catch (EOFException e) {
+        available = false;
+      }
+      if (available) {
+        in.reset();
+        logVersion = in.readInt();
+        if (logVersion < FSConstants.LAYOUT_VERSION) // future version
+          throw new IOException(
+                          "Unexpected version of the file system log file: "
+                          + logVersion + ". Current version = " 
+                          + FSConstants.LAYOUT_VERSION + ".");
+      }
+      assert logVersion <= Storage.LAST_UPGRADABLE_LAYOUT_VERSION :
+                            "Unsupported version " + logVersion;
+      numEdits = loadEditRecords(logVersion, in, false);
+    } finally {
+      if(closeOnExit)
+        in.close();
+    }
+    if (logVersion != FSConstants.LAYOUT_VERSION) // other version
+      numEdits++; // save this image asap
+    return numEdits;
+  }
+
+  int loadEditRecords(int logVersion, DataInputStream in,
+                             boolean closeOnExit) throws IOException {
+    FSNamesystem fsNamesys = fsimage.getFSNamesystem();
+    FSDirectory fsDir = fsNamesys.dir;
+    int numEdits = 0;
+    String clientName = null;
+    String clientMachine = null;
+    String path = null;
+    int numOpAdd = 0, numOpClose = 0, numOpDelete = 0,
+        numOpRename = 0, numOpSetRepl = 0, numOpMkDir = 0,
+        numOpSetPerm = 0, numOpSetOwner = 0, numOpSetGenStamp = 0,
+        numOpTimes = 0, numOpOther = 0;
+    try {
+      while (true) {
+        long timestamp = 0;
+        long mtime = 0;
+        long atime = 0;
+        long blockSize = 0;
+        byte opcode = -1;
+        try {
+          in.mark(1);
+          opcode = in.readByte();
+          if (opcode == OP_INVALID) {
+            in.reset(); // reset back to end of file if somebody reads it again
+            break; // no more transactions
+          }
+        } catch (EOFException e) {
+          break; // no more transactions
+        }
+        numEdits++;
+        switch (opcode) {
+        case OP_ADD:
+        case OP_CLOSE: {
+          // versions > 0 support per file replication
+          // get name and replication
+          int length = in.readInt();
+          if (-7 == logVersion && length != 3||
+              -17 < logVersion && logVersion < -7 && length != 4 ||
+              logVersion <= -17 && length != 5) {
+              throw new IOException("Incorrect data format."  +
+                                    " logVersion is " + logVersion +
+                                    " but writables.length is " +
+                                    length + ". ");
+          }
+          path = FSImage.readString(in);
+          short replication = adjustReplication(readShort(in));
+          mtime = readLong(in);
+          if (logVersion <= -17) {
+            atime = readLong(in);
+          }
+          if (logVersion < -7) {
+            blockSize = readLong(in);
+          }
+          // get blocks
+          Block blocks[] = null;
+          if (logVersion <= -14) {
+            blocks = readBlocks(in);
+          } else {
+            BlockTwo oldblk = new BlockTwo();
+            int num = in.readInt();
+            blocks = new Block[num];
+            for (int i = 0; i < num; i++) {
+              oldblk.readFields(in);
+              blocks[i] = new Block(oldblk.blkid, oldblk.len, 
+                                    Block.GRANDFATHER_GENERATION_STAMP);
+            }
+          }
+
+          // Older versions of HDFS does not store the block size in inode.
+          // If the file has more than one block, use the size of the
+          // first block as the blocksize. Otherwise use the default
+          // block size.
+          if (-8 <= logVersion && blockSize == 0) {
+            if (blocks.length > 1) {
+              blockSize = blocks[0].getNumBytes();
+            } else {
+              long first = ((blocks.length == 1)? blocks[0].getNumBytes(): 0);
+              blockSize = Math.max(fsNamesys.getDefaultBlockSize(), first);
+            }
+          }
+           
+          PermissionStatus permissions = fsNamesys.getUpgradePermission();
+          if (logVersion <= -11) {
+            permissions = PermissionStatus.read(in);
+          }
+
+          // clientname, clientMachine and block locations of last block.
+          if (opcode == OP_ADD && logVersion <= -12) {
+            clientName = FSImage.readString(in);
+            clientMachine = FSImage.readString(in);
+            if (-13 <= logVersion) {
+              readDatanodeDescriptorArray(in);
+            }
+          } else {
+            clientName = "";
+            clientMachine = "";
+          }
+
+          // The open lease transaction re-creates a file if necessary.
+          // Delete the file if it already exists.
+          if (FSNamesystem.LOG.isDebugEnabled()) {
+            FSNamesystem.LOG.debug(opcode + ": " + path + 
+                                   " numblocks : " + blocks.length +
+                                   " clientHolder " +  clientName +
+                                   " clientMachine " + clientMachine);
+          }
+
+          fsDir.unprotectedDelete(path, mtime);
+
+          // add to the file tree
+          INodeFile node = (INodeFile)fsDir.unprotectedAddFile(
+                                                    path, permissions,
+                                                    blocks, replication, 
+                                                    mtime, atime, blockSize);
+          if (opcode == OP_ADD) {
+            numOpAdd++;
+            //
+            // Replace current node with a INodeUnderConstruction.
+            // Recreate in-memory lease record.
+            //
+            INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
+                                      node.getLocalNameBytes(),
+                                      node.getReplication(), 
+                                      node.getModificationTime(),
+                                      node.getPreferredBlockSize(),
+                                      node.getBlocks(),
+                                      node.getPermissionStatus(),
+                                      clientName, 
+                                      clientMachine, 
+                                      null);
+            fsDir.replaceNode(path, node, cons);
+            fsNamesys.leaseManager.addLease(cons.clientName, path);
+          }
+          break;
+        } 
+        case OP_SET_REPLICATION: {
+          numOpSetRepl++;
+          path = FSImage.readString(in);
+          short replication = adjustReplication(readShort(in));
+          fsDir.unprotectedSetReplication(path, replication, null);
+          break;
+        } 
+        case OP_RENAME: {
+          numOpRename++;
+          int length = in.readInt();
+          if (length != 3) {
+            throw new IOException("Incorrect data format. " 
+                                  + "Mkdir operation.");
+          }
+          String s = FSImage.readString(in);
+          String d = FSImage.readString(in);
+          timestamp = readLong(in);
+          FileStatus dinfo = fsDir.getFileInfo(d);
+          fsDir.unprotectedRenameTo(s, d, timestamp);
+          fsNamesys.changeLease(s, d, dinfo);
+          break;
+        }
+        case OP_DELETE: {
+          numOpDelete++;
+          int length = in.readInt();
+          if (length != 2) {
+            throw new IOException("Incorrect data format. " 
+                                  + "delete operation.");
+          }
+          path = FSImage.readString(in);
+          timestamp = readLong(in);
+          fsDir.unprotectedDelete(path, timestamp);
+          break;
+        }
+        case OP_MKDIR: {
+          numOpMkDir++;
+          PermissionStatus permissions = fsNamesys.getUpgradePermission();
+          int length = in.readInt();
+          if (-17 < logVersion && length != 2 ||
+              logVersion <= -17 && length != 3) {
+            throw new IOException("Incorrect data format. " 
+                                  + "Mkdir operation.");
+          }
+          path = FSImage.readString(in);
+          timestamp = readLong(in);
+
+          // The disk format stores atimes for directories as well.
+          // However, currently this is not being updated/used because of
+          // performance reasons.
+          if (logVersion <= -17) {
+            atime = readLong(in);
+          }
+
+          if (logVersion <= -11) {
+            permissions = PermissionStatus.read(in);
+          }
+          fsDir.unprotectedMkdir(path, permissions, timestamp);
+          break;
+        }
+        case OP_SET_GENSTAMP: {
+          numOpSetGenStamp++;
+          long lw = in.readLong();
+          fsNamesys.setGenerationStamp(lw);
+          break;
+        } 
+        case OP_DATANODE_ADD: {
+          numOpOther++;
+          FSImage.DatanodeImage nodeimage = new FSImage.DatanodeImage();
+          nodeimage.readFields(in);
+          //Datnodes are not persistent any more.
+          break;
+        }
+        case OP_DATANODE_REMOVE: {
+          numOpOther++;
+          DatanodeID nodeID = new DatanodeID();
+          nodeID.readFields(in);
+          //Datanodes are not persistent any more.
+          break;
+        }
+        case OP_SET_PERMISSIONS: {
+          numOpSetPerm++;
+          if (logVersion > -11)
+            throw new IOException("Unexpected opcode " + opcode
+                                  + " for version " + logVersion);
+          fsDir.unprotectedSetPermission(
+              FSImage.readString(in), FsPermission.read(in));
+          break;
+        }
+        case OP_SET_OWNER: {
+          numOpSetOwner++;
+          if (logVersion > -11)
+            throw new IOException("Unexpected opcode " + opcode
+                                  + " for version " + logVersion);
+          fsDir.unprotectedSetOwner(FSImage.readString(in),
+              FSImage.readString_EmptyAsNull(in),
+              FSImage.readString_EmptyAsNull(in));
+          break;
+        }
+        case OP_SET_NS_QUOTA: {
+          if (logVersion > -16) {
+            throw new IOException("Unexpected opcode " + opcode
+                + " for version " + logVersion);
+          }
+          fsDir.unprotectedSetQuota(FSImage.readString(in), 
+                                    readLongWritable(in), 
+                                    FSConstants.QUOTA_DONT_SET);
+          break;
+        }
+        case OP_CLEAR_NS_QUOTA: {
+          if (logVersion > -16) {
+            throw new IOException("Unexpected opcode " + opcode
+                + " for version " + logVersion);
+          }
+          fsDir.unprotectedSetQuota(FSImage.readString(in),
+                                    FSConstants.QUOTA_RESET,
+                                    FSConstants.QUOTA_DONT_SET);
+          break;
+        }
+
+        case OP_SET_QUOTA:
+          fsDir.unprotectedSetQuota(FSImage.readString(in),
+                                    readLongWritable(in),
+                                    readLongWritable(in));
+                                      
+          break;
+
+        case OP_TIMES: {
+          numOpTimes++;
+          int length = in.readInt();
+          if (length != 3) {
+            throw new IOException("Incorrect data format. " 
+                                  + "times operation.");
+          }
+          path = FSImage.readString(in);
+          mtime = readLong(in);
+          atime = readLong(in);
+          fsDir.unprotectedSetTimes(path, mtime, atime, true);
+          break;
+        }
+        default: {
+          throw new IOException("Never seen opcode " + opcode);
+        }
+        }
+      }
+    } finally {
+      if(closeOnExit)
+        in.close();
+    }
+    if (FSImage.LOG.isDebugEnabled()) {
+      FSImage.LOG.debug("numOpAdd = " + numOpAdd + " numOpClose = " + numOpClose 
+          + " numOpDelete = " + numOpDelete + " numOpRename = " + numOpRename 
+          + " numOpSetRepl = " + numOpSetRepl + " numOpMkDir = " + numOpMkDir
+          + " numOpSetPerm = " + numOpSetPerm 
+          + " numOpSetOwner = " + numOpSetOwner
+          + " numOpSetGenStamp = " + numOpSetGenStamp 
+          + " numOpTimes = " + numOpTimes
+          + " numOpOther = " + numOpOther);
+    }
+    return numEdits;
+  }
+
+  // a place holder for reading a long
+  private static final LongWritable longWritable = new LongWritable();
+
+  /** Read an integer from an input stream */
+  private static long readLongWritable(DataInputStream in) throws IOException {
+    synchronized (longWritable) {
+      longWritable.readFields(in);
+      return longWritable.get();
+    }
+  }
+  
+  short adjustReplication(short replication) {
+    FSNamesystem fsNamesys = fsimage.getFSNamesystem();
+    short minReplication = fsNamesys.getMinReplication();
+    if (replication<minReplication) {
+      replication = minReplication;
+    }
+    short maxReplication = fsNamesys.getMaxReplication();
+    if (replication>maxReplication) {
+      replication = maxReplication;
+    }
+    return replication;
+  }
+
+  /**
+   * Write an operation to the edit log. Do not sync to persistent
+   * store yet.
+   */
+  synchronized void logEdit(byte op, Writable ... writables) {
+    if(getNumEditStreams() == 0)
+      throw new java.lang.IllegalStateException(NO_JOURNAL_STREAMS_WARNING);
+    ArrayList<EditLogOutputStream> errorStreams = null;
+    long start = FSNamesystem.now();
+    for(EditLogOutputStream eStream : editStreams) {
+      Log.debug("loggin edits into " + eStream.getName()  + " stream");
+      if(!eStream.isOperationSupported(op))
+        continue;
+      try {
+        eStream.write(op, writables);
+      } catch (IOException ie) {
+        FSImage.LOG.warn("logEdit: removing "+ eStream.getName(), ie);
+        if(errorStreams == null)
+          errorStreams = new ArrayList<EditLogOutputStream>(1);
+        errorStreams.add(eStream);
+      }
+    }
+    processIOError(errorStreams, true);
+    recordTransaction(start);
+  }
+
+  private void recordTransaction(long start) {
+    // get a new transactionId
+    txid++;
+
+    //
+    // record the transactionId when new data was written to the edits log
+    //
+    TransactionId id = myTransactionId.get();
+    id.txid = txid;
+
+    // update statistics
+    long end = FSNamesystem.now();
+    numTransactions++;
+    totalTimeTransactions += (end-start);
+    if (metrics != null) // Metrics is non-null only when used inside name node
+      metrics.transactions.inc((end-start));
+  }
+
+  //
+  // Sync all modifications done by this thread.
+  //
+  public void logSync() throws IOException {
+    ArrayList<EditLogOutputStream> errorStreams = null;
+    long syncStart = 0;
+
+    // Fetch the transactionId of this thread. 
+    long mytxid = myTransactionId.get().txid;
+
+    synchronized (this) {
+      assert editStreams.size() > 0 : "no editlog streams";
+      printStatistics(false);
+
+      // if somebody is already syncing, then wait
+      while (mytxid > synctxid && isSyncRunning) {
+        try {
+          wait(1000);
+        } catch (InterruptedException ie) { 
+        }
+      }
+
+      //
+      // If this transaction was already flushed, then nothing to do
+      //
+      if (mytxid <= synctxid) {
+        numTransactionsBatchedInSync++;
+        if (metrics != null) // Metrics is non-null only when used inside name node
+          metrics.transactionsBatchedInSync.inc();
+        return;
+      }
+   
+      // now, this thread will do the sync
+      syncStart = txid;
+      isSyncRunning = true;   
+
+      // swap buffers
+      for(EditLogOutputStream eStream : editStreams) {
+        eStream.setReadyToFlush();
+      }
+    }
+
+    // do the sync
+    long start = FSNamesystem.now();
+    for (int idx = 0; idx < editStreams.size(); idx++) {
+      EditLogOutputStream eStream = editStreams.get(idx);
+      try {
+        eStream.flush();
+      } catch (IOException ie) {
+        //
+        // remember the streams that encountered an error.
+        //
+        if (errorStreams == null) {
+          errorStreams = new ArrayList<EditLogOutputStream>(1);
+        }
+        errorStreams.add(eStream);
+        FSNamesystem.LOG.error("Unable to sync edit log. " +
+                               "Fatal Error.");
+      }
+    }
+    long elapsed = FSNamesystem.now() - start;
+
+    synchronized (this) {
+       processIOError(errorStreams, true);
+       synctxid = syncStart;
+       isSyncRunning = false;
+       this.notifyAll();
+    }
+
+    if (metrics != null) // Metrics is non-null only when used inside name node
+      metrics.syncs.inc(elapsed);
+  }
+
+  //
+  // print statistics every 1 minute.
+  //
+  private void printStatistics(boolean force) {
+    long now = FSNamesystem.now();
+    if (lastPrintTime + 60000 > now && !force) {
+      return;
+    }
+    if (editStreams == null) {
+      return;
+    }
+    lastPrintTime = now;
+    StringBuilder buf = new StringBuilder();
+    buf.append("Number of transactions: ");
+    buf.append(numTransactions);
+    buf.append(" Total time for transactions(ms): ");
+    buf.append(totalTimeTransactions);
+    buf.append("Number of transactions batched in Syncs: ");
+    buf.append(numTransactionsBatchedInSync);
+    buf.append(" Number of syncs: ");
+    buf.append(editStreams.get(0).getNumSync());
+    buf.append(" SyncTimes(ms): ");
+
+    int numEditStreams = editStreams.size();
+    for (int idx = 0; idx < numEditStreams; idx++) {
+      EditLogOutputStream eStream = editStreams.get(idx);
+      buf.append(eStream.getTotalSyncTime());
+      buf.append(" ");
+    }
+    FSNamesystem.LOG.info(buf);
+  }
+
+  /** 
+   * Add open lease record to edit log. 
+   * Records the block locations of the last block.
+   */
+  public void logOpenFile(String path, INodeFileUnderConstruction newNode) {
+
+    DeprecatedUTF8 nameReplicationPair[] = new DeprecatedUTF8[] { 
+      new DeprecatedUTF8(path), 
+      FSEditLog.toLogReplication(newNode.getReplication()),
+      FSEditLog.toLogLong(newNode.getModificationTime()),
+      FSEditLog.toLogLong(newNode.getAccessTime()),
+      FSEditLog.toLogLong(newNode.getPreferredBlockSize())};
+    logEdit(OP_ADD,
+            new ArrayWritable(DeprecatedUTF8.class, nameReplicationPair), 
+            new ArrayWritable(Block.class, newNode.getBlocks()),
+            newNode.getPermissionStatus(),
+            new DeprecatedUTF8(newNode.getClientName()),
+            new DeprecatedUTF8(newNode.getClientMachine()));
+  }
+
+  /** 
+   * Add close lease record to edit log.
+   */
+  public void logCloseFile(String path, INodeFile newNode) {
+    DeprecatedUTF8 nameReplicationPair[] = new DeprecatedUTF8[] {
+      new DeprecatedUTF8(path),
+      FSEditLog.toLogReplication(newNode.getReplication()),
+      FSEditLog.toLogLong(newNode.getModificationTime()),
+      FSEditLog.toLogLong(newNode.getAccessTime()),
+      FSEditLog.toLogLong(newNode.getPreferredBlockSize())};
+    logEdit(OP_CLOSE,
+            new ArrayWritable(DeprecatedUTF8.class, nameReplicationPair),
+            new ArrayWritable(Block.class, newNode.getBlocks()),
+            newNode.getPermissionStatus());
+  }
+  
+  /** 
+   * Add create directory record to edit log
+   */
+  public void logMkDir(String path, INode newNode) {
+    DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
+      new DeprecatedUTF8(path),
+      FSEditLog.toLogLong(newNode.getModificationTime()),
+      FSEditLog.toLogLong(newNode.getAccessTime())
+    };
+    logEdit(OP_MKDIR, new ArrayWritable(DeprecatedUTF8.class, info),
+        newNode.getPermissionStatus());
+  }
+  
+  /** 
+   * Add rename record to edit log
+   * TODO: use String parameters until just before writing to disk
+   */
+  void logRename(String src, String dst, long timestamp) {
+    DeprecatedUTF8 info[] = new DeprecatedUTF8[] { 
+      new DeprecatedUTF8(src),
+      new DeprecatedUTF8(dst),
+      FSEditLog.toLogLong(timestamp)};
+    logEdit(OP_RENAME, new ArrayWritable(DeprecatedUTF8.class, info));
+  }
+  
+  /** 
+   * Add set replication record to edit log
+   */
+  void logSetReplication(String src, short replication) {
+    logEdit(OP_SET_REPLICATION, 
+            new DeprecatedUTF8(src), 
+            FSEditLog.toLogReplication(replication));
+  }
+  
+  /** Add set namespace quota record to edit log
+   * 
+   * @param src the string representation of the path to a directory
+   * @param quota the directory size limit
+   */
+  void logSetQuota(String src, long nsQuota, long dsQuota) {
+    logEdit(OP_SET_QUOTA, new DeprecatedUTF8(src), 
+            new LongWritable(nsQuota), new LongWritable(dsQuota));
+  }
+
+  /**  Add set permissions record to edit log */
+  void logSetPermissions(String src, FsPermission permissions) {
+    logEdit(OP_SET_PERMISSIONS, new DeprecatedUTF8(src), permissions);
+  }
+
+  /**  Add set owner record to edit log */
+  void logSetOwner(String src, String username, String groupname) {
+    DeprecatedUTF8 u = new DeprecatedUTF8(username == null? "": username);
+    DeprecatedUTF8 g = new DeprecatedUTF8(groupname == null? "": groupname);
+    logEdit(OP_SET_OWNER, new DeprecatedUTF8(src), u, g);
+  }
+
+  /** 
+   * Add delete file record to edit log
+   */
+  void logDelete(String src, long timestamp) {
+    DeprecatedUTF8 info[] = new DeprecatedUTF8[] { 
+      new DeprecatedUTF8(src),
+      FSEditLog.toLogLong(timestamp)};
+    logEdit(OP_DELETE, new ArrayWritable(DeprecatedUTF8.class, info));
+  }
+
+  /** 
+   * Add generation stamp record to edit log
+   */
+  void logGenerationStamp(long genstamp) {
+    logEdit(OP_SET_GENSTAMP, new LongWritable(genstamp));
+  }
+
+  /** 
+   * Add access time record to edit log
+   */
+  void logTimes(String src, long mtime, long atime) {
+    DeprecatedUTF8 info[] = new DeprecatedUTF8[] { 
+      new DeprecatedUTF8(src),
+      FSEditLog.toLogLong(mtime),
+      FSEditLog.toLogLong(atime)};
+    logEdit(OP_TIMES, new ArrayWritable(DeprecatedUTF8.class, info));
+  }
+  
+  static private DeprecatedUTF8 toLogReplication(short replication) {
+    return new DeprecatedUTF8(Short.toString(replication));
+  }
+  
+  static private DeprecatedUTF8 toLogLong(long timestamp) {
+    return new DeprecatedUTF8(Long.toString(timestamp));
+  }
+
+  /**
+   * Return the size of the current EditLog
+   */
+  synchronized long getEditLogSize() throws IOException {
+    assert getNumEditsDirs() <= getNumEditStreams() : 
+        "Number of edits directories should not exceed the number of streams.";
+    long size = 0;
+    ArrayList<EditLogOutputStream> al = null;
+    for (int idx = 0; idx < getNumEditStreams(); idx++) {
+      EditLogOutputStream es = editStreams.get(idx);
+      try {
+        long curSize = es.length();
+        assert (size == 0 || size == curSize || curSize ==0) :
+          "Wrong streams size";
+        size = Math.max(size, curSize);
+      } catch (IOException e) {
+        FSImage.LOG.warn("getEditLogSize: editstream.length failed. removing editlog (" +
+            idx + ") " + es.getName());
+        if(al==null) al = new ArrayList<EditLogOutputStream>(1);
+        al.add(es);
+      }
+    }
+    if(al!=null) processIOError(al, true);
+    return size;
+  }
+  
+  public String listEditsStreams() {
+    StringBuffer buf = new StringBuffer();
+    for (EditLogOutputStream os : editStreams) {
+      buf.append(os.getName()  + ";");
+    }
+    return buf.toString();
+  }
+
+  /**
+   * Closes the current edit log and opens edits.new. 
+   */
+  synchronized void rollEditLog() throws IOException {
+    Iterator<StorageDirectory> it = fsimage.dirIterator(NameNodeDirType.EDITS);
+    if(!it.hasNext()) 
+      return;
+    //
+    // If edits.new already exists in some directory, verify it
+    // exists in all directories.
+    //
+    boolean alreadyExists = existsNew(it.next());
+    while(it.hasNext()) {
+      StorageDirectory sd = it.next();
+      if(alreadyExists != existsNew(sd))
+        throw new IOException(getEditNewFile(sd) 
+              + "should " + (alreadyExists ? "" : "not ") + "exist.");
+    }
+    if(alreadyExists)
+      return; // nothing to do, edits.new exists!
+
+    // check if any of failed storage is now available and put it back
+    fsimage.attemptRestoreRemovedStorage();
+
+    divertFileStreams(
+        Storage.STORAGE_DIR_CURRENT + "/" + NameNodeFile.EDITS_NEW.getName());
+  }
+
+  /**
+   * Divert file streams from file edits to file edits.new.<p>
+   * Close file streams, which are currently writing into edits files.
+   * Create new streams based on file getRoot()/dest.
+   * @param dest new stream path relative to the storage directory root.
+   * @throws IOException
+   */
+  synchronized void divertFileStreams(String dest) throws IOException {
+    assert getNumEditStreams() >= getNumEditsDirs() :
+      "Inconsistent number of streams";
+    ArrayList<EditLogOutputStream> errorStreams = null;
+    EditStreamIterator itE = 
+      (EditStreamIterator)getOutputStreamIterator(JournalType.FILE);
+    Iterator<StorageDirectory> itD = 
+      fsimage.dirIterator(NameNodeDirType.EDITS);
+    while(itE.hasNext() && itD.hasNext()) {
+      EditLogOutputStream eStream = itE.next();
+      StorageDirectory sd = itD.next();
+      if(!eStream.getName().startsWith(sd.getRoot().getPath()))
+        throw new IOException("Inconsistent order of edit streams: " + eStream);
+      try {
+        // close old stream
+        closeStream(eStream);
+        // create new stream
+        eStream = new EditLogFileOutputStream(new File(sd.getRoot(), dest),
+            sizeOutputFlushBuffer);
+        eStream.create();
+        // replace by the new stream
+        itE.replace(eStream);
+      } catch (IOException e) {
+        if(errorStreams == null)
+          errorStreams = new ArrayList<EditLogOutputStream>(1);
+        errorStreams.add(eStream);
+      }
+    }
+    processIOError(errorStreams, true);
+  }
+
+  /**
+   * Removes the old edit log and renames edits.new to edits.
+   * Reopens the edits file.
+   */
+  synchronized void purgeEditLog() throws IOException {
+    revertFileStreams(
+        Storage.STORAGE_DIR_CURRENT + "/" + NameNodeFile.EDITS_NEW.getName());
+  }
+
+  /**
+   * Revert file streams from file edits.new back to file edits.<p>
+   * Close file streams, which are currently writing into getRoot()/source.
+   * Rename getRoot()/source to edits.
+   * Reopen streams so that they start writing into edits files.
+   * @param dest new stream path relative to the storage directory root.
+   * @throws IOException
+   */
+  synchronized void revertFileStreams(String source) throws IOException {
+    assert getNumEditStreams() >= getNumEditsDirs() :
+      "Inconsistent number of streams";
+    ArrayList<EditLogOutputStream> errorStreams = null;
+    EditStreamIterator itE = 
+      (EditStreamIterator)getOutputStreamIterator(JournalType.FILE);
+    Iterator<StorageDirectory> itD = 
+      fsimage.dirIterator(NameNodeDirType.EDITS);
+    while(itE.hasNext() && itD.hasNext()) {
+      EditLogOutputStream eStream = itE.next();
+      StorageDirectory sd = itD.next();
+      if(!eStream.getName().startsWith(sd.getRoot().getPath()))
+        throw new IOException("Inconsistent order of edit streams: " + eStream);
+      try {
+        // close old stream
+        closeStream(eStream);
+        // rename edits.new to edits
+        File editFile = getEditFile(sd);
+        File prevEditFile = new File(sd.getRoot(), source);
+        if(prevEditFile.exists()) {
+          if(!prevEditFile.renameTo(editFile)) {
+            //
+            // renameTo() fails on Windows if the destination
+            // file exists.
+            //
+            if(!editFile.delete() || !prevEditFile.renameTo(editFile)) {
+              throw new IOException("Rename failed for " + sd.getRoot());
+            }
+          }
+        }
+        // open new stream
+        eStream = new EditLogFileOutputStream(editFile, sizeOutputFlushBuffer);
+        // replace by the new stream
+        itE.replace(eStream);
+      } catch (IOException e) {
+        if(errorStreams == null)
+          errorStreams = new ArrayList<EditLogOutputStream>(1);
+        errorStreams.add(eStream);
+      }
+    }
+    processIOError(errorStreams, true);
+  }
+
+  /**
+   * Return the name of the edit file
+   */
+  synchronized File getFsEditName() {
+    StorageDirectory sd = null;   
+    for (Iterator<StorageDirectory> it = 
+      fsimage.dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
+      sd = it.next();   
+      if(sd.getRoot().canRead())
+        return getEditFile(sd);
+    }
+    return null;
+  }
+
+  /**
+   * Returns the timestamp of the edit log
+   */
+  synchronized long getFsEditTime() {
+    Iterator<StorageDirectory> it = fsimage.dirIterator(NameNodeDirType.EDITS);
+    if(it.hasNext())
+      return getEditFile(it.next()).lastModified();
+    return 0;
+  }
+
+  // sets the initial capacity of the flush buffer.
+  public void setBufferCapacity(int size) {
+    sizeOutputFlushBuffer = size;
+  }
+
+  /**
+   * A class to read in blocks stored in the old format. The only two
+   * fields in the block were blockid and length.
+   */
+  static class BlockTwo implements Writable {
+    long blkid;
+    long len;
+
+    static {                                      // register a ctor
+      WritableFactories.setFactory
+        (BlockTwo.class,
+         new WritableFactory() {
+           public Writable newInstance() { return new BlockTwo(); }
+         });
+    }
+
+
+    BlockTwo() {
+      blkid = 0;
+      len = 0;
+    }
+    /////////////////////////////////////
+    // Writable
+    /////////////////////////////////////
+    public void write(DataOutput out) throws IOException {
+      out.writeLong(blkid);
+      out.writeLong(len);
+    }
+
+    public void readFields(DataInput in) throws IOException {
+      this.blkid = in.readLong();
+      this.len = in.readLong();
+    }
+  }
+
+  /** This method is defined for compatibility reason. */
+  static private DatanodeDescriptor[] readDatanodeDescriptorArray(DataInput in
+      ) throws IOException {
+    DatanodeDescriptor[] locations = new DatanodeDescriptor[in.readInt()];
+    for (int i = 0; i < locations.length; i++) {
+      locations[i] = new DatanodeDescriptor();
+      locations[i].readFieldsFromFSEditLog(in);
+    }
+    return locations;
+  }
+
+  static private short readShort(DataInputStream in) throws IOException {
+    return Short.parseShort(FSImage.readString(in));
+  }
+
+  static private long readLong(DataInputStream in) throws IOException {
+    return Long.parseLong(FSImage.readString(in));
+  }
+
+  static private Block[] readBlocks(DataInputStream in) throws IOException {
+    int numBlocks = in.readInt();
+    Block[] blocks = new Block[numBlocks];
+    for (int i = 0; i < numBlocks; i++) {
+      blocks[i] = new Block();
+      blocks[i].readFields(in);
+    }
+    return blocks;
+  }
+
+  boolean isEmpty() throws IOException {
+    return getEditLogSize() <= 0;
+  }
+
+  /**
+   * Create (or find if already exists) an edit output stream, which
+   * streams journal records (edits) to the specified backup node.<br>
+   * Send a record, prescribing to start journal spool.<br>
+   * This should be sent via regular stream of journal records so that
+   * the backup node new exactly after which record it should start spooling.
+   * 
+   * @param bnReg the backup node registration information.
+   * @param nnReg this (active) name-node registration.
+   * @throws IOException
+   */
+  void logJSpoolStart(NamenodeRegistration bnReg, // backup node
+                      NamenodeRegistration nnReg) // active name-node
+  throws IOException {
+    if(bnReg.isRole(NamenodeRole.CHECKPOINT))
+      return; // checkpoint node does not stream edits
+    if(editStreams == null)
+      editStreams = new ArrayList<EditLogOutputStream>();
+    EditLogOutputStream boStream = null;
+    for(EditLogOutputStream eStream : editStreams) {
+      if(eStream.getName().equals(bnReg.getAddress())) {
+        boStream = eStream; // already there
+        break;
+      }
+    }
+    if(boStream == null) {
+      boStream = new EditLogBackupOutputStream(bnReg, nnReg);
+      editStreams.add(boStream);
+    }
+    logEdit(OP_JSPOOL_START, (Writable[])null);
+  }
+
+  /**
+   * Write an operation to the edit log. Do not sync to persistent
+   * store yet.
+   */
+  synchronized void logEdit(int length, byte[] data) {
+    if(getNumEditStreams() == 0)
+      throw new java.lang.IllegalStateException(NO_JOURNAL_STREAMS_WARNING);
+    ArrayList<EditLogOutputStream> errorStreams = null;
+    long start = FSNamesystem.now();
+    for(EditLogOutputStream eStream : editStreams) {
+      try {
+        eStream.write(data, 0, length);
+      } catch (IOException ie) {
+        if(errorStreams == null)
+          errorStreams = new ArrayList<EditLogOutputStream>(1);
+        errorStreams.add(eStream);
+      }
+    }
+    processIOError(errorStreams, true);
+    recordTransaction(start);
+  }
+
+  /**
+   * Iterates output streams based of the same type.
+   * Type null will iterate over all streams.
+   */
+  private class EditStreamIterator implements Iterator<EditLogOutputStream> {
+    JournalType type;
+    int prevIndex; // for remove()
+    int nextIndex; // for next()
+
+    EditStreamIterator(JournalType streamType) {
+      this.type = streamType;
+      this.nextIndex = 0;
+      this.prevIndex = 0;
+    }
+
+    public boolean hasNext() {
+      if(editStreams == null || 
+         editStreams.isEmpty() || nextIndex >= editStreams.size())
+        return false;
+      while(nextIndex < editStreams.size()
+            && !editStreams.get(nextIndex).getType().isOfType(type))
+        nextIndex++;
+      return nextIndex < editStreams.size();
+    }
+
+    public EditLogOutputStream next() {
+      EditLogOutputStream stream = editStreams.get(nextIndex);
+      prevIndex = nextIndex;
+      nextIndex++;
+      while(nextIndex < editStreams.size()
+          && !editStreams.get(nextIndex).getType().isOfType(type))
+      nextIndex++;
+      return stream;
+    }
+
+    public void remove() {
+      nextIndex = prevIndex; // restore previous state
+      removeStream(prevIndex); // remove last returned element
+      hasNext(); // reset nextIndex to correct place
+    }
+
+    void replace(EditLogOutputStream newStream) {
+      assert 0 <= prevIndex && prevIndex < editStreams.size() :
+                                                        "Index out of bound.";
+      editStreams.set(prevIndex, newStream);
+    }
+  }
+
+  /**
+   * Get stream iterator for the specified type.
+   */
+  public Iterator<EditLogOutputStream>
+  getOutputStreamIterator(JournalType streamType) {
+    return new EditStreamIterator(streamType);
+  }
+
+  private void closeStream(EditLogOutputStream eStream) throws IOException {
+    eStream.setReadyToFlush();
+    eStream.flush();
+    eStream.close();
+  }
+
+  void incrementCheckpointTime() {
+    fsimage.incrementCheckpointTime();
+    Writable[] args = {new LongWritable(fsimage.getCheckpointTime())};
+    logEdit(OP_CHECKPOINT_TIME, args);
+  }
+
+  synchronized void releaseBackupStream(NamenodeRegistration registration) {
+    Iterator<EditLogOutputStream> it =
+                                  getOutputStreamIterator(JournalType.BACKUP);
+    ArrayList<EditLogOutputStream> errorStreams = null;
+    NamenodeRegistration backupNode = null;
+    while(it.hasNext()) {
+      EditLogBackupOutputStream eStream = (EditLogBackupOutputStream)it.next();
+      backupNode = eStream.getRegistration();
+      if(backupNode.getAddress().equals(registration.getAddress()) &&
+            backupNode.isRole(registration.getRole())) {
+        errorStreams = new ArrayList<EditLogOutputStream>(1);
+        errorStreams.add(eStream);
+        break;
+      }
+    }
+    assert backupNode == null || backupNode.isRole(NamenodeRole.BACKUP) :
+      "Not a backup node corresponds to a backup stream";
+    processIOError(errorStreams, true);
+  }
+
+  synchronized boolean checkBackupRegistration(
+      NamenodeRegistration registration) {
+    Iterator<EditLogOutputStream> it =
+                                  getOutputStreamIterator(JournalType.BACKUP);
+    boolean regAllowed = !it.hasNext();
+    NamenodeRegistration backupNode = null;
+    ArrayList<EditLogOutputStream> errorStreams = null;
+    while(it.hasNext()) {
+      EditLogBackupOutputStream eStream = (EditLogBackupOutputStream)it.next();
+      backupNode = eStream.getRegistration();
+      if(backupNode.getAddress().equals(registration.getAddress()) &&
+          backupNode.isRole(registration.getRole())) {
+        regAllowed = true; // same node re-registers
+        break;
+      }
+      if(!eStream.isAlive()) {
+        if(errorStreams == null)
+          errorStreams = new ArrayList<EditLogOutputStream>(1);
+        errorStreams.add(eStream);
+        regAllowed = true; // previous backup node failed
+      }
+    }
+    assert backupNode == null || backupNode.isRole(NamenodeRole.BACKUP) :
+      "Not a backup node corresponds to a backup stream";
+    processIOError(errorStreams, true);
+    return regAllowed;
+  }
+}

+ 1880 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -0,0 +1,1880 @@
+/**
+ * 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 java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+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.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.common.UpgradeManager;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.io.DeprecatedUTF8;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * FSImage handles checkpointing and logging of the namespace edits.
+ * 
+ */
+public class FSImage extends Storage {
+
+  private static final SimpleDateFormat DATE_FORM =
+    new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+  //
+  // The filenames used for storing the images
+  //
+  enum NameNodeFile {
+    IMAGE     ("fsimage"),
+    TIME      ("fstime"),
+    EDITS     ("edits"),
+    IMAGE_NEW ("fsimage.ckpt"),
+    EDITS_NEW ("edits.new");
+    
+    private String fileName = null;
+    private NameNodeFile(String name) {this.fileName = name;}
+    String getName() {return fileName;}
+  }
+
+  // checkpoint states
+  enum CheckpointStates{START, ROLLED_EDITS, UPLOAD_START, UPLOAD_DONE; }
+  /**
+   * Implementation of StorageDirType specific to namenode storage
+   * A Storage directory could be of type IMAGE which stores only fsimage,
+   * or of type EDITS which stores edits or of type IMAGE_AND_EDITS which 
+   * stores both fsimage and edits.
+   */
+  static enum NameNodeDirType implements StorageDirType {
+    UNDEFINED,
+    IMAGE,
+    EDITS,
+    IMAGE_AND_EDITS;
+    
+    public StorageDirType getStorageDirType() {
+      return this;
+    }
+    
+    public boolean isOfType(StorageDirType type) {
+      if ((this == IMAGE_AND_EDITS) && (type == IMAGE || type == EDITS))
+        return true;
+      return this == type;
+    }
+  }
+
+  protected FSNamesystem namesystem = null;
+  protected long checkpointTime = -1L;  // The age of the image
+  protected FSEditLog editLog = null;
+  private boolean isUpgradeFinalized = false;
+
+  /**
+   * flag that controls if we try to restore failed storages
+   */
+  private boolean restoreFailedStorage = false;
+
+  /**
+   * list of failed (and thus removed) storages
+   */
+  protected List<StorageDirectory> removedStorageDirs = new ArrayList<StorageDirectory>();
+  
+  /**
+   * Directories for importing an image from a checkpoint.
+   */
+  private Collection<File> checkpointDirs;
+  private Collection<File> checkpointEditsDirs;
+
+  /**
+   * Can fs-image be rolled?
+   */
+  volatile protected CheckpointStates ckptState = FSImage.CheckpointStates.START; 
+
+  /**
+   * Used for saving the image to disk
+   */
+  static private final FsPermission FILE_PERM = new FsPermission((short)0);
+  static private final byte[] PATH_SEPARATOR = INode.string2Bytes(Path.SEPARATOR);
+
+  /**
+   */
+  FSImage() {
+    this((FSNamesystem)null);
+  }
+
+  FSImage(FSNamesystem ns) {
+    super(NodeType.NAME_NODE);
+    this.editLog = new FSEditLog(this);
+    setFSNamesystem(ns);
+  }
+
+  /**
+   */
+  FSImage(Collection<File> fsDirs, Collection<File> fsEditsDirs) {
+    this();
+    setStorageDirectories(fsDirs, fsEditsDirs);
+  }
+
+  public FSImage(StorageInfo storageInfo) {
+    super(NodeType.NAME_NODE, storageInfo);
+  }
+
+  /**
+   * Represents an Image (image and edit file).
+   */
+  FSImage(File imageDir) {
+    this();
+    ArrayList<File> dirs = new ArrayList<File>(1);
+    ArrayList<File> editsDirs = new ArrayList<File>(1);
+    dirs.add(imageDir);
+    editsDirs.add(imageDir);
+    setStorageDirectories(dirs, editsDirs);
+  }
+  
+  protected FSNamesystem getFSNamesystem() {
+    return namesystem;
+  }
+
+  void setFSNamesystem(FSNamesystem ns) {
+    namesystem = ns;
+  }
+
+  public void setRestoreFailedStorage(boolean val) {
+    LOG.info("set restore failed storage to " + val);
+    restoreFailedStorage=val;
+  }
+  
+  public boolean getRestoreFailedStorage() {
+    return restoreFailedStorage;
+  }
+  
+  void setStorageDirectories(Collection<File> fsNameDirs,
+                             Collection<File> fsEditsDirs) {
+    this.storageDirs = new ArrayList<StorageDirectory>();
+    this.removedStorageDirs = new ArrayList<StorageDirectory>();
+   // Add all name dirs with appropriate NameNodeDirType 
+    for (File dirName : fsNameDirs) {
+      boolean isAlsoEdits = false;
+      for (File editsDirName : fsEditsDirs) {
+        if (editsDirName.compareTo(dirName) == 0) {
+          isAlsoEdits = true;
+          fsEditsDirs.remove(editsDirName);
+          break;
+        }
+      }
+      NameNodeDirType dirType = (isAlsoEdits) ?
+                          NameNodeDirType.IMAGE_AND_EDITS :
+                          NameNodeDirType.IMAGE;
+      this.addStorageDir(new StorageDirectory(dirName, dirType));
+    }
+    
+    // Add edits dirs if they are different from name dirs
+    for (File dirName : fsEditsDirs) {
+      this.addStorageDir(new StorageDirectory(dirName, 
+                    NameNodeDirType.EDITS));
+    }
+  }
+
+  void setCheckpointDirectories(Collection<File> dirs,
+                                Collection<File> editsDirs) {
+    checkpointDirs = dirs;
+    checkpointEditsDirs = editsDirs;
+  }
+  
+  static File getImageFile(StorageDirectory sd, NameNodeFile type) {
+    return new File(sd.getCurrentDir(), type.getName());
+  }
+  
+  List<StorageDirectory> getRemovedStorageDirs() {
+	  return this.removedStorageDirs;
+  }
+  
+  File getEditFile(StorageDirectory sd) {
+    return getImageFile(sd, NameNodeFile.EDITS);
+  }
+  
+  File getEditNewFile(StorageDirectory sd) {
+    return getImageFile(sd, NameNodeFile.EDITS_NEW);
+  }
+
+  Collection<File> getFiles(NameNodeFile type, NameNodeDirType dirType) {
+    ArrayList<File> list = new ArrayList<File>();
+    Iterator<StorageDirectory> it = (dirType == null) ? dirIterator() :
+                                    dirIterator(dirType);
+    for ( ;it.hasNext(); ) {
+      list.add(getImageFile(it.next(), type));
+    }
+    return list;
+  }
+
+  Collection<File> getDirectories(NameNodeDirType dirType) {
+    ArrayList<File> list = new ArrayList<File>();
+    Iterator<StorageDirectory> it = (dirType == null) ? dirIterator() :
+                                    dirIterator(dirType);
+    for ( ;it.hasNext(); ) {
+      list.add(it.next().getRoot());
+    }
+    return list;
+  }
+
+  Collection<File> getImageDirectories() {
+    return getDirectories(NameNodeDirType.IMAGE);
+  }
+
+  Collection<File> getEditsDirectories() {
+    return getDirectories(NameNodeDirType.EDITS);
+  }
+
+  /**
+   * Return number of storage directories of the given type.
+   * @param dirType directory type
+   * @return number of storage directories of type dirType
+   */
+  int getNumStorageDirs(NameNodeDirType dirType) {
+    if(dirType == null)
+      return getNumStorageDirs();
+    Iterator<StorageDirectory> it = dirIterator(dirType);
+    int numDirs = 0;
+    for(; it.hasNext(); it.next())
+      numDirs++;
+    return numDirs;
+  }
+
+  /**
+   * Analyze storage directories.
+   * Recover from previous transitions if required. 
+   * Perform fs state transition if necessary depending on the namespace info.
+   * Read storage info. 
+   * 
+   * @param dataDirs
+   * @param startOpt startup option
+   * @throws IOException
+   * @return true if the image needs to be saved or false otherwise
+   */
+  boolean recoverTransitionRead(Collection<File> dataDirs,
+                             Collection<File> editsDirs,
+                                StartupOption startOpt
+                                ) throws IOException {
+    assert startOpt != StartupOption.FORMAT : 
+      "NameNode formatting should be performed before reading the image";
+    
+    // none of the data dirs exist
+    if((dataDirs.size() == 0 || editsDirs.size() == 0) 
+                             && startOpt != StartupOption.IMPORT)  
+      throw new IOException(
+        "All specified directories are not accessible or do not exist.");
+    
+    if(startOpt == StartupOption.IMPORT 
+        && (checkpointDirs == null || checkpointDirs.isEmpty()))
+      throw new IOException("Cannot import image from a checkpoint. "
+                          + "\"fs.checkpoint.dir\" is not set." );
+
+    if(startOpt == StartupOption.IMPORT 
+        && (checkpointEditsDirs == null || checkpointEditsDirs.isEmpty()))
+      throw new IOException("Cannot import image from a checkpoint. "
+                          + "\"fs.checkpoint.edits.dir\" is not set." );
+    
+    setStorageDirectories(dataDirs, editsDirs);
+    // 1. For each data directory calculate its state and 
+    // check whether all is consistent before transitioning.
+    Map<StorageDirectory, StorageState> dataDirStates = 
+             new HashMap<StorageDirectory, StorageState>();
+    boolean isFormatted = false;
+    for (Iterator<StorageDirectory> it = 
+                      dirIterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      StorageState curState;
+      try {
+        curState = sd.analyzeStorage(startOpt);
+        // sd is locked but not opened
+        switch(curState) {
+        case NON_EXISTENT:
+          // name-node fails if any of the configured storage dirs are missing
+          throw new InconsistentFSStateException(sd.getRoot(),
+                      "storage directory does not exist or is not accessible.");
+        case NOT_FORMATTED:
+          break;
+        case NORMAL:
+          break;
+        default:  // recovery is possible
+          sd.doRecover(curState);      
+        }
+        if (curState != StorageState.NOT_FORMATTED 
+            && startOpt != StartupOption.ROLLBACK) {
+          sd.read(); // read and verify consistency with other directories
+          isFormatted = true;
+        }
+        if (startOpt == StartupOption.IMPORT && isFormatted)
+          // import of a checkpoint is allowed only into empty image directories
+          throw new IOException("Cannot import image from a checkpoint. " 
+              + " NameNode already contains an image in " + sd.getRoot());
+      } catch (IOException ioe) {
+        sd.unlock();
+        throw ioe;
+      }
+      dataDirStates.put(sd,curState);
+    }
+    
+    if (!isFormatted && startOpt != StartupOption.ROLLBACK 
+                     && startOpt != StartupOption.IMPORT)
+      throw new IOException("NameNode is not formatted.");
+    if (layoutVersion < LAST_PRE_UPGRADE_LAYOUT_VERSION) {
+      checkVersionUpgradable(layoutVersion);
+    }
+    if (startOpt != StartupOption.UPGRADE
+          && layoutVersion < LAST_PRE_UPGRADE_LAYOUT_VERSION
+          && layoutVersion != FSConstants.LAYOUT_VERSION)
+        throw new IOException(
+           "\nFile system image contains an old layout version " + layoutVersion
+         + ".\nAn upgrade to version " + FSConstants.LAYOUT_VERSION
+         + " is required.\nPlease restart NameNode with -upgrade option.");
+    // check whether distributed upgrade is reguired and/or should be continued
+    verifyDistributedUpgradeProgress(startOpt);
+
+    // 2. Format unformatted dirs.
+    this.checkpointTime = 0L;
+    for (Iterator<StorageDirectory> it = 
+                     dirIterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      StorageState curState = dataDirStates.get(sd);
+      switch(curState) {
+      case NON_EXISTENT:
+        throw new IOException(StorageState.NON_EXISTENT + 
+                              " state cannot be here");
+      case NOT_FORMATTED:
+        LOG.info("Storage directory " + sd.getRoot() + " is not formatted.");
+        LOG.info("Formatting ...");
+        sd.clearDirectory(); // create empty currrent dir
+        break;
+      default:
+        break;
+      }
+    }
+
+    // 3. Do transitions
+    switch(startOpt) {
+    case UPGRADE:
+      doUpgrade();
+      return false; // upgrade saved image already
+    case IMPORT:
+      doImportCheckpoint();
+      return false; // import checkpoint saved image already
+    case ROLLBACK:
+      doRollback();
+      break;
+    case REGULAR:
+      // just load the image
+    }
+    return loadFSImage();
+  }
+
+  private void doUpgrade() throws IOException {
+    if(getDistributedUpgradeState()) {
+      // only distributed upgrade need to continue
+      // don't do version upgrade
+      this.loadFSImage();
+      initializeDistributedUpgrade();
+      return;
+    }
+    // Upgrade is allowed only if there are 
+    // no previous fs states in any of the directories
+    for (Iterator<StorageDirectory> it = 
+                           dirIterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      if (sd.getPreviousDir().exists())
+        throw new InconsistentFSStateException(sd.getRoot(),
+                                               "previous fs state should not exist during upgrade. "
+                                               + "Finalize or rollback first.");
+    }
+
+    // load the latest image
+    this.loadFSImage();
+
+    // Do upgrade for each directory
+    long oldCTime = this.getCTime();
+    this.cTime = FSNamesystem.now();  // generate new cTime for the state
+    int oldLV = this.getLayoutVersion();
+    this.layoutVersion = FSConstants.LAYOUT_VERSION;
+    this.checkpointTime = FSNamesystem.now();
+    for (Iterator<StorageDirectory> it = 
+                           dirIterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      LOG.info("Upgrading image directory " + sd.getRoot()
+               + ".\n   old LV = " + oldLV
+               + "; old CTime = " + oldCTime
+               + ".\n   new LV = " + this.getLayoutVersion()
+               + "; new CTime = " + this.getCTime());
+      File curDir = sd.getCurrentDir();
+      File prevDir = sd.getPreviousDir();
+      File tmpDir = sd.getPreviousTmp();
+      assert curDir.exists() : "Current directory must exist.";
+      assert !prevDir.exists() : "prvious directory must not exist.";
+      assert !tmpDir.exists() : "prvious.tmp directory must not exist.";
+      // rename current to tmp
+      rename(curDir, tmpDir);
+      // save new image
+      if (!curDir.mkdir())
+        throw new IOException("Cannot create directory " + curDir);
+      saveFSImage(getImageFile(sd, NameNodeFile.IMAGE));
+      editLog.createEditLogFile(getImageFile(sd, NameNodeFile.EDITS));
+      // write version and time files
+      sd.write();
+      // rename tmp to previous
+      rename(tmpDir, prevDir);
+      isUpgradeFinalized = false;
+      LOG.info("Upgrade of " + sd.getRoot() + " is complete.");
+    }
+    initializeDistributedUpgrade();
+    editLog.open();
+  }
+
+  private void doRollback() throws IOException {
+    // Rollback is allowed only if there is 
+    // a previous fs states in at least one of the storage directories.
+    // Directories that don't have previous state do not rollback
+    boolean canRollback = false;
+    FSImage prevState = new FSImage(getFSNamesystem());
+    prevState.layoutVersion = FSConstants.LAYOUT_VERSION;
+    for (Iterator<StorageDirectory> it = 
+                       dirIterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      File prevDir = sd.getPreviousDir();
+      if (!prevDir.exists()) {  // use current directory then
+        LOG.info("Storage directory " + sd.getRoot()
+                 + " does not contain previous fs state.");
+        sd.read(); // read and verify consistency with other directories
+        continue;
+      }
+      StorageDirectory sdPrev = prevState.new StorageDirectory(sd.getRoot());
+      sdPrev.read(sdPrev.getPreviousVersionFile());  // read and verify consistency of the prev dir
+      canRollback = true;
+    }
+    if (!canRollback)
+      throw new IOException("Cannot rollback. " 
+                            + "None of the storage directories contain previous fs state.");
+
+    // Now that we know all directories are going to be consistent
+    // Do rollback for each directory containing previous state
+    for (Iterator<StorageDirectory> it = 
+                          dirIterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      File prevDir = sd.getPreviousDir();
+      if (!prevDir.exists())
+        continue;
+
+      LOG.info("Rolling back storage directory " + sd.getRoot()
+               + ".\n   new LV = " + prevState.getLayoutVersion()
+               + "; new CTime = " + prevState.getCTime());
+      File tmpDir = sd.getRemovedTmp();
+      assert !tmpDir.exists() : "removed.tmp directory must not exist.";
+      // rename current to tmp
+      File curDir = sd.getCurrentDir();
+      assert curDir.exists() : "Current directory must exist.";
+      rename(curDir, tmpDir);
+      // rename previous to current
+      rename(prevDir, curDir);
+
+      // delete tmp dir
+      deleteDir(tmpDir);
+      LOG.info("Rollback of " + sd.getRoot()+ " is complete.");
+    }
+    isUpgradeFinalized = true;
+    // check whether name-node can start in regular mode
+    verifyDistributedUpgradeProgress(StartupOption.REGULAR);
+  }
+
+  private void doFinalize(StorageDirectory sd) throws IOException {
+    File prevDir = sd.getPreviousDir();
+    if (!prevDir.exists()) { // already discarded
+      LOG.info("Directory " + prevDir + " does not exist.");
+      LOG.info("Finalize upgrade for " + sd.getRoot()+ " is not required.");
+      return;
+    }
+    LOG.info("Finalizing upgrade for storage directory " 
+             + sd.getRoot() + "."
+             + (getLayoutVersion()==0 ? "" :
+                   "\n   cur LV = " + this.getLayoutVersion()
+                   + "; cur CTime = " + this.getCTime()));
+    assert sd.getCurrentDir().exists() : "Current directory must exist.";
+    final File tmpDir = sd.getFinalizedTmp();
+    // rename previous to tmp and remove
+    rename(prevDir, tmpDir);
+    deleteDir(tmpDir);
+    isUpgradeFinalized = true;
+    LOG.info("Finalize upgrade for " + sd.getRoot()+ " is complete.");
+  }
+
+  /**
+   * Load image from a checkpoint directory and save it into the current one.
+   * @throws IOException
+   */
+  void doImportCheckpoint() throws IOException {
+    FSNamesystem fsNamesys = getFSNamesystem();
+    FSImage ckptImage = new FSImage(fsNamesys);
+    // replace real image with the checkpoint image
+    FSImage realImage = fsNamesys.getFSImage();
+    assert realImage == this;
+    fsNamesys.dir.fsImage = ckptImage;
+    // load from the checkpoint dirs
+    try {
+      ckptImage.recoverTransitionRead(checkpointDirs, checkpointEditsDirs,
+                                              StartupOption.REGULAR);
+    } finally {
+      ckptImage.close();
+    }
+    // return back the real image
+    realImage.setStorageInfo(ckptImage);
+    checkpointTime = ckptImage.checkpointTime;
+    fsNamesys.dir.fsImage = realImage;
+    // and save it but keep the same checkpointTime
+    saveFSImage(false);
+  }
+
+  void finalizeUpgrade() throws IOException {
+    for (Iterator<StorageDirectory> it = 
+                          dirIterator(); it.hasNext();) {
+      doFinalize(it.next());
+    }
+  }
+
+  boolean isUpgradeFinalized() {
+    return isUpgradeFinalized;
+  }
+
+  protected void getFields(Properties props, 
+                           StorageDirectory sd 
+                           ) throws IOException {
+    super.getFields(props, sd);
+    if (layoutVersion == 0)
+      throw new IOException("NameNode directory " 
+                            + sd.getRoot() + " is not formatted.");
+    String sDUS, sDUV;
+    sDUS = props.getProperty("distributedUpgradeState"); 
+    sDUV = props.getProperty("distributedUpgradeVersion");
+    setDistributedUpgradeState(
+        sDUS == null? false : Boolean.parseBoolean(sDUS),
+        sDUV == null? getLayoutVersion() : Integer.parseInt(sDUV));
+    this.checkpointTime = readCheckpointTime(sd);
+  }
+
+  /**
+   * Determine the checkpoint time of the specified StorageDirectory
+   * 
+   * @param sd StorageDirectory to check
+   * @return If file exists and can be read, last checkpoint time. If not, 0L.
+   * @throws IOException On errors processing file pointed to by sd
+   */
+  long readCheckpointTime(StorageDirectory sd) throws IOException {
+    File timeFile = getImageFile(sd, NameNodeFile.TIME);
+    long timeStamp = 0L;
+    if (timeFile.exists() && timeFile.canRead()) {
+      DataInputStream in = new DataInputStream(new FileInputStream(timeFile));
+      try {
+        timeStamp = in.readLong();
+      } finally {
+        in.close();
+      }
+    }
+    return timeStamp;
+  }
+
+  /**
+   * Write last checkpoint time and version file into the storage directory.
+   * 
+   * The version file should always be written last.
+   * Missing or corrupted version file indicates that 
+   * the checkpoint is not valid.
+   * 
+   * @param sd storage directory
+   * @throws IOException
+   */
+  protected void setFields(Properties props, 
+                           StorageDirectory sd 
+                           ) throws IOException {
+    super.setFields(props, sd);
+    boolean uState = getDistributedUpgradeState();
+    int uVersion = getDistributedUpgradeVersion();
+    if(uState && uVersion != getLayoutVersion()) {
+      props.setProperty("distributedUpgradeState", Boolean.toString(uState));
+      props.setProperty("distributedUpgradeVersion", Integer.toString(uVersion)); 
+    }
+    writeCheckpointTime(sd);
+  }
+
+  /**
+   * Write last checkpoint time into a separate file.
+   * 
+   * @param sd
+   * @throws IOException
+   */
+  void writeCheckpointTime(StorageDirectory sd) throws IOException {
+    if (checkpointTime < 0L)
+      return; // do not write negative time
+    File timeFile = getImageFile(sd, NameNodeFile.TIME);
+    if (timeFile.exists() && ! timeFile.delete()) {
+        LOG.error("Cannot delete chekpoint time file: "
+                  + timeFile.getCanonicalPath());
+    }
+    DataOutputStream out = new DataOutputStream(
+                                                new FileOutputStream(timeFile));
+    try {
+      out.writeLong(checkpointTime);
+    } finally {
+      out.close();
+    }
+  }
+
+  /**
+   * Record new checkpoint time in order to
+   * distinguish healthy directories from the removed ones.
+   * If there is an error writing new checkpoint time, the corresponding
+   * storage directory is removed from the list.
+   */
+  void incrementCheckpointTime() {
+    setCheckpointTime(checkpointTime + 1);
+  }
+
+  /**
+   * The age of the namespace state.<p>
+   * Reflects the latest time the image was saved.
+   * Modified with every save or a checkpoint.
+   * Persisted in VERSION file.
+   */
+  long getCheckpointTime() {
+    return checkpointTime;
+  }
+
+  void setCheckpointTime(long newCpT) {
+    checkpointTime = newCpT;
+    // Write new checkpoint time in all storage directories
+    for(Iterator<StorageDirectory> it =
+                          dirIterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      try {
+        writeCheckpointTime(sd);
+      } catch(IOException e) {
+        // Close any edits stream associated with this dir and remove directory
+        LOG.warn("incrementCheckpointTime failed on " + sd.getRoot().getPath() + ";type="+sd.getStorageDirType());
+      }
+    }
+  }
+
+  /**
+   * @param sds - array of SDs to process
+   * @param propagate - flag, if set - then call corresponding EditLog stream's 
+   * processIOError function.
+   */
+  void processIOError(ArrayList<StorageDirectory> sds, boolean propagate) {
+    ArrayList<EditLogOutputStream> al = null;
+    for(StorageDirectory sd:sds) {
+      // if has a stream assosiated with it - remove it too..
+      if (propagate && sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
+        EditLogOutputStream eStream = editLog.getEditsStream(sd);
+        if(al == null) al = new ArrayList<EditLogOutputStream>(1);
+        al.add(eStream);
+      }
+      
+      for (Iterator<StorageDirectory> it = dirIterator(); it.hasNext();) {
+        StorageDirectory sd1 = it.next();
+        if (sd.equals(sd1)) {
+          //add storage to the removed list
+          LOG.warn("FSImage:processIOError: removing storage: "
+              + sd.getRoot().getPath());
+          try {
+            sd1.unlock(); //unlock before removing (in case it will be restored)
+          } catch (Exception e) {
+            // nothing
+          }
+          removedStorageDirs.add(sd1);
+          it.remove();
+          break;
+        }
+      }
+    }
+    // if there are some edit log streams to remove		
+    if(propagate && al != null) 
+      editLog.processIOError(al, false);
+    
+    //if called from edits log, the it will call increment from there
+    if(propagate) incrementCheckpointTime(); 
+  }
+
+  public FSEditLog getEditLog() {
+    return editLog;
+  }
+
+  public boolean isConversionNeeded(StorageDirectory sd) throws IOException {
+    File oldImageDir = new File(sd.getRoot(), "image");
+    if (!oldImageDir.exists()) {
+      if(sd.getVersionFile().exists())
+        throw new InconsistentFSStateException(sd.getRoot(),
+            oldImageDir + " does not exist.");
+      return false;
+    }
+    // check the layout version inside the image file
+    File oldF = new File(oldImageDir, "fsimage");
+    RandomAccessFile oldFile = new RandomAccessFile(oldF, "rws");
+    try {
+      oldFile.seek(0);
+      int odlVersion = oldFile.readInt();
+      if (odlVersion < LAST_PRE_UPGRADE_LAYOUT_VERSION)
+        return false;
+    } finally {
+      oldFile.close();
+    }
+    return true;
+  }
+  
+  //
+  // Atomic move sequence, to recover from interrupted checkpoint
+  //
+  boolean recoverInterruptedCheckpoint(StorageDirectory nameSD,
+                                       StorageDirectory editsSD) 
+                                       throws IOException {
+    boolean needToSave = false;
+    File curFile = getImageFile(nameSD, NameNodeFile.IMAGE);
+    File ckptFile = getImageFile(nameSD, NameNodeFile.IMAGE_NEW);
+
+    //
+    // If we were in the midst of a checkpoint
+    //
+    if (ckptFile.exists()) {
+      needToSave = true;
+      if (getImageFile(editsSD, NameNodeFile.EDITS_NEW).exists()) {
+        //
+        // checkpointing migth have uploaded a new
+        // merged image, but we discard it here because we are
+        // not sure whether the entire merged image was uploaded
+        // before the namenode crashed.
+        //
+        if (!ckptFile.delete()) {
+          throw new IOException("Unable to delete " + ckptFile);
+        }
+      } else {
+        //
+        // checkpointing was in progress when the namenode
+        // shutdown. The fsimage.ckpt was created and the edits.new
+        // file was moved to edits. We complete that checkpoint by
+        // moving fsimage.new to fsimage. There is no need to 
+        // update the fstime file here. renameTo fails on Windows
+        // if the destination file already exists.
+        //
+        if (!ckptFile.renameTo(curFile)) {
+          if (!curFile.delete())
+            LOG.warn("Unable to delete dir " + curFile + " before rename");
+          if (!ckptFile.renameTo(curFile)) {
+            throw new IOException("Unable to rename " + ckptFile +
+                                  " to " + curFile);
+          }
+        }
+      }
+    }
+    return needToSave;
+  }
+
+  /**
+   * Choose latest image from one of the directories,
+   * load it and merge with the edits from that directory.
+   * 
+   * @return whether the image should be saved
+   * @throws IOException
+   */
+  boolean loadFSImage() throws IOException {
+    long latestNameCheckpointTime = Long.MIN_VALUE;
+    long latestEditsCheckpointTime = Long.MIN_VALUE;
+    boolean needToSave = false;
+    isUpgradeFinalized = true;
+    
+    StorageDirectory latestNameSD = null;
+    StorageDirectory latestEditsSD = null;
+    
+    Collection<String> imageDirs = new ArrayList<String>();
+    Collection<String> editsDirs = new ArrayList<String>();
+    
+    // Set to determine if all of storageDirectories share the same checkpoint
+    Set<Long> checkpointTimes = new HashSet<Long>();
+
+    // Process each of the storage directories to find the pair of
+    // newest image file and edit file
+    for (Iterator<StorageDirectory> it = dirIterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+
+      // Was the file just formatted?
+      if (!sd.getVersionFile().exists()) {
+        needToSave |= true;
+        continue;
+      }
+      
+      boolean imageExists = false;
+      boolean editsExists = false;
+      
+      // Determine if sd is image, edits or both
+      if (sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE)) {
+        imageExists = getImageFile(sd, NameNodeFile.IMAGE).exists();
+        imageDirs.add(sd.getRoot().getCanonicalPath());
+      }
+      
+      if (sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
+        editsExists = getImageFile(sd, NameNodeFile.EDITS).exists();
+        editsDirs.add(sd.getRoot().getCanonicalPath());
+      }
+      
+      checkpointTime = readCheckpointTime(sd);
+
+      checkpointTimes.add(checkpointTime);
+      
+      if (sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE) && 
+         (latestNameCheckpointTime < checkpointTime) && imageExists) {
+        latestNameCheckpointTime = checkpointTime;
+        latestNameSD = sd;
+      }
+      
+      if (sd.getStorageDirType().isOfType(NameNodeDirType.EDITS) && 
+           (latestEditsCheckpointTime < checkpointTime) && editsExists) {
+        latestEditsCheckpointTime = checkpointTime;
+        latestEditsSD = sd;
+      }
+      
+      // check that we have a valid, non-default checkpointTime
+      if (checkpointTime <= 0L)
+        needToSave |= true;
+      
+      // set finalized flag
+      isUpgradeFinalized = isUpgradeFinalized && !sd.getPreviousDir().exists();
+    }
+
+    // We should have at least one image and one edits dirs
+    if (latestNameSD == null)
+      throw new IOException("Image file is not found in " + imageDirs);
+    if (latestEditsSD == null)
+      throw new IOException("Edits file is not found in " + editsDirs);
+
+    // Make sure we are loading image and edits from same checkpoint
+    if (latestNameCheckpointTime != latestEditsCheckpointTime)
+      throw new IOException("Inconsistent storage detected, " +
+                            "name and edits storage do not match");
+    
+    // If there was more than one checkpointTime recorded we should save
+    needToSave |= checkpointTimes.size() != 1;
+    
+    // Recover from previous interrupted checkpoint, if any
+    needToSave |= recoverInterruptedCheckpoint(latestNameSD, latestEditsSD);
+
+    long startTime = FSNamesystem.now();
+    long imageSize = getImageFile(latestNameSD, NameNodeFile.IMAGE).length();
+
+    //
+    // Load in bits
+    //
+    latestNameSD.read();
+    needToSave |= loadFSImage(getImageFile(latestNameSD, NameNodeFile.IMAGE));
+    LOG.info("Image file of size " + imageSize + " loaded in " 
+        + (FSNamesystem.now() - startTime)/1000 + " seconds.");
+    
+    // Load latest edits
+    needToSave |= (loadFSEdits(latestEditsSD) > 0);
+
+    assert editLog != null : "editLog must be initialized";
+    if(!editLog.isOpen())
+      editLog.open();
+    
+    return needToSave;
+  }
+
+  /**
+   * Load in the filesystem image from file. It's a big list of
+   * filenames and blocks.  Return whether we should
+   * "re-save" and consolidate the edit-logs
+   */
+  boolean loadFSImage(File curFile) throws IOException {
+    assert this.getLayoutVersion() < 0 : "Negative layout version is expected.";
+    assert curFile != null : "curFile is null";
+
+    FSNamesystem fsNamesys = getFSNamesystem();
+    FSDirectory fsDir = fsNamesys.dir;
+
+    //
+    // Load in bits
+    //
+    boolean needToSave = true;
+    DataInputStream in = new DataInputStream(new BufferedInputStream(
+                              new FileInputStream(curFile)));
+    try {
+      /*
+       * Note: Remove any checks for version earlier than 
+       * Storage.LAST_UPGRADABLE_LAYOUT_VERSION since we should never get 
+       * to here with older images.
+       */
+      
+      /*
+       * TODO we need to change format of the image file
+       * it should not contain version and namespace fields
+       */
+      // read image version: first appeared in version -1
+      int imgVersion = in.readInt();
+      // read namespaceID: first appeared in version -2
+      this.namespaceID = in.readInt();
+
+      // read number of files
+      long numFiles;
+      if (imgVersion <= -16) {
+        numFiles = in.readLong();
+      } else {
+        numFiles = in.readInt();
+      }
+
+      this.layoutVersion = imgVersion;
+      // read in the last generation stamp.
+      if (imgVersion <= -12) {
+        long genstamp = in.readLong();
+        fsNamesys.setGenerationStamp(genstamp); 
+      }
+
+      needToSave = (imgVersion != FSConstants.LAYOUT_VERSION);
+
+      // read file info
+      short replication = fsNamesys.getDefaultReplication();
+
+      LOG.info("Number of files = " + numFiles);
+
+      String path;
+      String parentPath = "";
+      INodeDirectory parentINode = fsDir.rootDir;
+      for (long i = 0; i < numFiles; i++) {
+        long modificationTime = 0;
+        long atime = 0;
+        long blockSize = 0;
+        path = readString(in);
+        replication = in.readShort();
+        replication = editLog.adjustReplication(replication);
+        modificationTime = in.readLong();
+        if (imgVersion <= -17) {
+          atime = in.readLong();
+        }
+        if (imgVersion <= -8) {
+          blockSize = in.readLong();
+        }
+        int numBlocks = in.readInt();
+        Block blocks[] = null;
+
+        // for older versions, a blocklist of size 0
+        // indicates a directory.
+        if ((-9 <= imgVersion && numBlocks > 0) ||
+            (imgVersion < -9 && numBlocks >= 0)) {
+          blocks = new Block[numBlocks];
+          for (int j = 0; j < numBlocks; j++) {
+            blocks[j] = new Block();
+            if (-14 < imgVersion) {
+              blocks[j].set(in.readLong(), in.readLong(), 
+                            Block.GRANDFATHER_GENERATION_STAMP);
+            } else {
+              blocks[j].readFields(in);
+            }
+          }
+        }
+        // Older versions of HDFS does not store the block size in inode.
+        // If the file has more than one block, use the size of the 
+        // first block as the blocksize. Otherwise use the default block size.
+        //
+        if (-8 <= imgVersion && blockSize == 0) {
+          if (numBlocks > 1) {
+            blockSize = blocks[0].getNumBytes();
+          } else {
+            long first = ((numBlocks == 1) ? blocks[0].getNumBytes(): 0);
+            blockSize = Math.max(fsNamesys.getDefaultBlockSize(), first);
+          }
+        }
+        
+        // get quota only when the node is a directory
+        long nsQuota = -1L;
+        if (imgVersion <= -16 && blocks == null) {
+          nsQuota = in.readLong();
+        }
+        long dsQuota = -1L;
+        if (imgVersion <= -18 && blocks == null) {
+          dsQuota = in.readLong();
+        }
+        
+        PermissionStatus permissions = fsNamesys.getUpgradePermission();
+        if (imgVersion <= -11) {
+          permissions = PermissionStatus.read(in);
+        }
+        if (path.length() == 0) { // it is the root
+          // update the root's attributes
+          if (nsQuota != -1 || dsQuota != -1) {
+            fsDir.rootDir.setQuota(nsQuota, dsQuota);
+          }
+          fsDir.rootDir.setModificationTime(modificationTime);
+          fsDir.rootDir.setPermissionStatus(permissions);
+          continue;
+        }
+        // check if the new inode belongs to the same parent
+        if(!isParent(path, parentPath)) {
+          parentINode = null;
+          parentPath = getParent(path);
+        }
+        // add new inode
+        parentINode = fsDir.addToParent(path, parentINode, permissions,
+                                        blocks, replication, modificationTime, 
+                                        atime, nsQuota, dsQuota, blockSize);
+      }
+      
+      // load datanode info
+      this.loadDatanodes(imgVersion, in);
+
+      // load Files Under Construction
+      this.loadFilesUnderConstruction(imgVersion, in, fsNamesys);
+      
+    } finally {
+      in.close();
+    }
+    
+    return needToSave;
+  }
+
+  /**
+   * Return string representing the parent of the given path.
+   */
+  String getParent(String path) {
+    return path.substring(0, path.lastIndexOf(Path.SEPARATOR));
+  }
+
+  private boolean isParent(String path, String parent) {
+    return parent != null && path != null
+          && path.indexOf(parent) == 0
+          && path.lastIndexOf(Path.SEPARATOR) == parent.length();
+  }
+
+  /**
+   * Load and merge edits from two edits files
+   * 
+   * @param sd storage directory
+   * @return number of edits loaded
+   * @throws IOException
+   */
+  int loadFSEdits(StorageDirectory sd) throws IOException {
+    int numEdits = 0;
+    EditLogFileInputStream edits = 
+      new EditLogFileInputStream(getImageFile(sd, NameNodeFile.EDITS));
+    
+    numEdits = editLog.loadFSEdits(edits);
+    edits.close();
+    File editsNew = getImageFile(sd, NameNodeFile.EDITS_NEW);
+    
+    if (editsNew.exists() && editsNew.length() > 0) {
+      edits = new EditLogFileInputStream(editsNew);
+      numEdits += editLog.loadFSEdits(edits);
+      edits.close();
+    }
+    
+    // update the counts.
+    getFSNamesystem().dir.updateCountForINodeWithQuota();    
+    
+    return numEdits;
+  }
+
+  /**
+   * Save the contents of the FS image to the file.
+   */
+  void saveFSImage(File newFile) throws IOException {
+    FSNamesystem fsNamesys = getFSNamesystem();
+    FSDirectory fsDir = fsNamesys.dir;
+    long startTime = FSNamesystem.now();
+    //
+    // Write out data
+    //
+    DataOutputStream out = new DataOutputStream(
+                                                new BufferedOutputStream(
+                                                                         new FileOutputStream(newFile)));
+    try {
+      out.writeInt(FSConstants.LAYOUT_VERSION);
+      out.writeInt(namespaceID);
+      out.writeLong(fsDir.rootDir.numItemsInTree());
+      out.writeLong(fsNamesys.getGenerationStamp());
+      byte[] byteStore = new byte[4*FSConstants.MAX_PATH_LENGTH];
+      ByteBuffer strbuf = ByteBuffer.wrap(byteStore);
+      // save the root
+      saveINode2Image(strbuf, fsDir.rootDir, out);
+      // save the rest of the nodes
+      saveImage(strbuf, 0, fsDir.rootDir, out);
+      fsNamesys.saveFilesUnderConstruction(out);
+      strbuf = null;
+    } finally {
+      out.close();
+    }
+
+    LOG.info("Image file of size " + newFile.length() + " saved in " 
+        + (FSNamesystem.now() - startTime)/1000 + " seconds.");
+  }
+
+  /**
+   * Save the contents of the FS image
+   * and create empty edits.
+   */
+  public void saveFSImage() throws IOException {
+    saveFSImage(true);
+  }
+
+  public void saveFSImage(boolean renewCheckpointTime) throws IOException {
+    assert editLog != null : "editLog must be initialized";
+    if(!editLog.isOpen())
+      editLog.open();
+
+    editLog.createNewIfMissing();
+    for (Iterator<StorageDirectory> it = 
+                           dirIterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      NameNodeDirType dirType = (NameNodeDirType)sd.getStorageDirType();
+      if (dirType.isOfType(NameNodeDirType.IMAGE))
+        saveFSImage(getImageFile(sd, NameNodeFile.IMAGE_NEW));
+      if (dirType.isOfType(NameNodeDirType.EDITS)) {    
+        editLog.createEditLogFile(getImageFile(sd, NameNodeFile.EDITS));
+        File editsNew = getImageFile(sd, NameNodeFile.EDITS_NEW);
+        if (editsNew.exists()) 
+          editLog.createEditLogFile(editsNew);
+      }
+    }
+    ckptState = CheckpointStates.UPLOAD_DONE;
+    rollFSImage(renewCheckpointTime);
+  }
+
+  /**
+   * Generate new namespaceID.
+   * 
+   * namespaceID is a persistent attribute of the namespace.
+   * It is generated when the namenode is formatted and remains the same
+   * during the life cycle of the namenode.
+   * When a datanodes register they receive it as the registrationID,
+   * which is checked every time the datanode is communicating with the 
+   * namenode. Datanodes that do not 'know' the namespaceID are rejected.
+   * 
+   * @return new namespaceID
+   */
+  private int newNamespaceID() {
+    Random r = new Random();
+    r.setSeed(FSNamesystem.now());
+    int newID = 0;
+    while(newID == 0)
+      newID = r.nextInt(0x7FFFFFFF);  // use 31 bits only
+    return newID;
+  }
+
+  /** Create new dfs name directory.  Caution: this destroys all files
+   * in this filesystem. */
+  void format(StorageDirectory sd) throws IOException {
+    sd.clearDirectory(); // create currrent dir
+    sd.lock();
+    try {
+      NameNodeDirType dirType = (NameNodeDirType)sd.getStorageDirType();
+      if (dirType.isOfType(NameNodeDirType.IMAGE))
+        saveFSImage(getImageFile(sd, NameNodeFile.IMAGE));
+      if (dirType.isOfType(NameNodeDirType.EDITS))
+        editLog.createEditLogFile(getImageFile(sd, NameNodeFile.EDITS));
+      sd.write();
+    } finally {
+      sd.unlock();
+    }
+    LOG.info("Storage directory " + sd.getRoot()
+             + " has been successfully formatted.");
+  }
+
+  public void format() throws IOException {
+    this.layoutVersion = FSConstants.LAYOUT_VERSION;
+    this.namespaceID = newNamespaceID();
+    this.cTime = 0L;
+    this.checkpointTime = FSNamesystem.now();
+    for (Iterator<StorageDirectory> it = 
+                           dirIterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      format(sd);
+    }
+  }
+
+  /*
+   * Save one inode's attributes to the image.
+   */
+  private static void saveINode2Image(ByteBuffer name,
+                                      INode node,
+                                      DataOutputStream out) throws IOException {
+    int nameLen = name.position();
+    out.writeShort(nameLen);
+    out.write(name.array(), name.arrayOffset(), nameLen);
+    if (!node.isDirectory()) {  // write file inode
+      INodeFile fileINode = (INodeFile)node;
+      out.writeShort(fileINode.getReplication());
+      out.writeLong(fileINode.getModificationTime());
+      out.writeLong(fileINode.getAccessTime());
+      out.writeLong(fileINode.getPreferredBlockSize());
+      Block[] blocks = fileINode.getBlocks();
+      out.writeInt(blocks.length);
+      for (Block blk : blocks)
+        blk.write(out);
+      FILE_PERM.fromShort(fileINode.getFsPermissionShort());
+      PermissionStatus.write(out, fileINode.getUserName(),
+                             fileINode.getGroupName(),
+                             FILE_PERM);
+    } else {   // write directory inode
+      out.writeShort(0);  // replication
+      out.writeLong(node.getModificationTime());
+      out.writeLong(0);   // access time
+      out.writeLong(0);   // preferred block size
+      out.writeInt(-1);    // # of blocks
+      out.writeLong(node.getNsQuota());
+      out.writeLong(node.getDsQuota());
+      FILE_PERM.fromShort(node.getFsPermissionShort());
+      PermissionStatus.write(out, node.getUserName(),
+                             node.getGroupName(),
+                             FILE_PERM);
+    }
+  }
+  /**
+   * Save file tree image starting from the given root.
+   * This is a recursive procedure, which first saves all children of
+   * a current directory and then moves inside the sub-directories.
+   */
+  private static void saveImage(ByteBuffer parentPrefix,
+                                int prefixLength,
+                                INodeDirectory current,
+                                DataOutputStream out) throws IOException {
+    int newPrefixLength = prefixLength;
+    if (current.getChildrenRaw() == null)
+      return;
+    for(INode child : current.getChildren()) {
+      // print all children first
+      parentPrefix.position(prefixLength);
+      parentPrefix.put(PATH_SEPARATOR).put(child.getLocalNameBytes());
+      saveINode2Image(parentPrefix, child, out);
+    }
+    for(INode child : current.getChildren()) {
+      if(!child.isDirectory())
+        continue;
+      parentPrefix.position(prefixLength);
+      parentPrefix.put(PATH_SEPARATOR).put(child.getLocalNameBytes());
+      newPrefixLength = parentPrefix.position();
+      saveImage(parentPrefix, newPrefixLength, (INodeDirectory)child, out);
+    }
+    parentPrefix.position(prefixLength);
+  }
+
+  void loadDatanodes(int version, DataInputStream in) throws IOException {
+    if (version > -3) // pre datanode image version
+      return;
+    if (version <= -12) {
+      return; // new versions do not store the datanodes any more.
+    }
+    int size = in.readInt();
+    for(int i = 0; i < size; i++) {
+      DatanodeImage nodeImage = new DatanodeImage();
+      nodeImage.readFields(in);
+      // We don't need to add these descriptors any more.
+    }
+  }
+
+  private void loadFilesUnderConstruction(int version, DataInputStream in, 
+                                  FSNamesystem fs) throws IOException {
+
+    FSDirectory fsDir = fs.dir;
+    if (version > -13) // pre lease image version
+      return;
+    int size = in.readInt();
+
+    LOG.info("Number of files under construction = " + size);
+
+    for (int i = 0; i < size; i++) {
+      INodeFileUnderConstruction cons = readINodeUnderConstruction(in);
+
+      // verify that file exists in namespace
+      String path = cons.getLocalName();
+      INode old = fsDir.getFileINode(path);
+      if (old == null) {
+        throw new IOException("Found lease for non-existent file " + path);
+      }
+      if (old.isDirectory()) {
+        throw new IOException("Found lease for directory " + path);
+      }
+      INodeFile oldnode = (INodeFile) old;
+      fsDir.replaceNode(path, oldnode, cons);
+      fs.leaseManager.addLease(cons.clientName, path); 
+    }
+  }
+
+  // Helper function that reads in an INodeUnderConstruction
+  // from the input stream
+  //
+  static INodeFileUnderConstruction readINodeUnderConstruction(
+                            DataInputStream in) throws IOException {
+    byte[] name = readBytes(in);
+    short blockReplication = in.readShort();
+    long modificationTime = in.readLong();
+    long preferredBlockSize = in.readLong();
+    int numBlocks = in.readInt();
+    BlockInfo[] blocks = new BlockInfo[numBlocks];
+    Block blk = new Block();
+    for (int i = 0; i < numBlocks; i++) {
+      blk.readFields(in);
+      blocks[i] = new BlockInfo(blk, blockReplication);
+    }
+    PermissionStatus perm = PermissionStatus.read(in);
+    String clientName = readString(in);
+    String clientMachine = readString(in);
+
+    // These locations are not used at all
+    int numLocs = in.readInt();
+    DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocs];
+    for (int i = 0; i < numLocs; i++) {
+      locations[i] = new DatanodeDescriptor();
+      locations[i].readFields(in);
+    }
+
+    return new INodeFileUnderConstruction(name, 
+                                          blockReplication, 
+                                          modificationTime,
+                                          preferredBlockSize,
+                                          blocks,
+                                          perm,
+                                          clientName,
+                                          clientMachine,
+                                          null);
+  }
+
+  // Helper function that writes an INodeUnderConstruction
+  // into the input stream
+  //
+  static void writeINodeUnderConstruction(DataOutputStream out,
+                                           INodeFileUnderConstruction cons,
+                                           String path) 
+                                           throws IOException {
+    writeString(path, out);
+    out.writeShort(cons.getReplication());
+    out.writeLong(cons.getModificationTime());
+    out.writeLong(cons.getPreferredBlockSize());
+    int nrBlocks = cons.getBlocks().length;
+    out.writeInt(nrBlocks);
+    for (int i = 0; i < nrBlocks; i++) {
+      cons.getBlocks()[i].write(out);
+    }
+    cons.getPermissionStatus().write(out);
+    writeString(cons.getClientName(), out);
+    writeString(cons.getClientMachine(), out);
+
+    out.writeInt(0); //  do not store locations of last block
+  }
+
+  /**
+   * Moves fsimage.ckpt to fsImage and edits.new to edits
+   * Reopens the new edits file.
+   */
+  void rollFSImage() throws IOException {
+    rollFSImage(true);
+  }
+
+  void rollFSImage(boolean renewCheckpointTime) throws IOException {
+    if (ckptState != CheckpointStates.UPLOAD_DONE
+      && !(ckptState == CheckpointStates.ROLLED_EDITS
+      && getNumStorageDirs(NameNodeDirType.IMAGE) == 0)) {
+      throw new IOException("Cannot roll fsImage before rolling edits log.");
+    }
+
+    for (Iterator<StorageDirectory> it = 
+                       dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      File ckpt = getImageFile(sd, NameNodeFile.IMAGE_NEW);
+      if (!ckpt.exists()) {
+        throw new IOException("Checkpoint file " + ckpt +
+                              " does not exist");
+      }
+    }
+    editLog.purgeEditLog(); // renamed edits.new to edits
+    LOG.debug("rollFSImage after purgeEditLog: storageList=" + listStorageDirectories());
+    //
+    // Renames new image
+    //
+    renameCheckpoint();
+    resetVersion(renewCheckpointTime);
+  }
+
+  /**
+   * Renames new image
+   */
+  void renameCheckpoint() {
+    ArrayList<StorageDirectory> al = null;
+    for (Iterator<StorageDirectory> it = 
+      dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      File ckpt = getImageFile(sd, NameNodeFile.IMAGE_NEW);
+      File curFile = getImageFile(sd, NameNodeFile.IMAGE);
+      // renameTo fails on Windows if the destination file 
+      // already exists.
+      LOG.debug("renaming  " + ckpt.getAbsolutePath() + " to "  + curFile.getAbsolutePath());
+      if (!ckpt.renameTo(curFile)) {
+        if (!curFile.delete() || !ckpt.renameTo(curFile)) {
+          LOG.warn("renaming  " + ckpt.getAbsolutePath() + " to "  + 
+              curFile.getAbsolutePath() + " FAILED");
+
+          if(al == null) al = new ArrayList<StorageDirectory> (1);
+          al.add(sd);
+        }
+      }
+    }
+    if(al != null) processIOError(al, true);
+  }
+
+  /**
+   * Updates version and fstime files in all directories (fsimage and edits).
+   */
+  void resetVersion(boolean renewCheckpointTime) throws IOException {
+    this.layoutVersion = FSConstants.LAYOUT_VERSION;
+    if(renewCheckpointTime)
+      this.checkpointTime = FSNamesystem.now();
+    
+    ArrayList<StorageDirectory> al = null;
+    for (Iterator<StorageDirectory> it = 
+                           dirIterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      // delete old edits if sd is the image only the directory
+      if (!sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
+        File editsFile = getImageFile(sd, NameNodeFile.EDITS);
+        if(editsFile.exists() && !editsFile.delete())
+          throw new IOException("Cannot delete edits file " 
+                                + editsFile.getCanonicalPath());
+      }
+      // delete old fsimage if sd is the edits only the directory
+      if (!sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE)) {
+        File imageFile = getImageFile(sd, NameNodeFile.IMAGE);
+        if(imageFile.exists() && !imageFile.delete())
+          throw new IOException("Cannot delete image file " 
+                                + imageFile.getCanonicalPath());
+      }
+      try {
+        sd.write();
+      } catch (IOException e) {
+        LOG.error("Cannot write file " + sd.getRoot(), e);
+        
+        if(al == null) al = new ArrayList<StorageDirectory> (1);
+        al.add(sd);       
+      }
+    }
+    if(al != null) processIOError(al, true);
+    ckptState = FSImage.CheckpointStates.START;
+  }
+
+  CheckpointSignature rollEditLog() throws IOException {
+    getEditLog().rollEditLog();
+    ckptState = CheckpointStates.ROLLED_EDITS;
+    // If checkpoint fails this should be the most recent image, therefore
+    incrementCheckpointTime();
+    return new CheckpointSignature(this);
+  }
+
+  /**
+   * This is called just before a new checkpoint is uploaded to the
+   * namenode.
+   */
+  void validateCheckpointUpload(CheckpointSignature sig) throws IOException {
+    if (ckptState != CheckpointStates.ROLLED_EDITS) {
+      throw new IOException("Namenode is not expecting an new image " +
+                             ckptState);
+    } 
+    // verify token
+    long modtime = getEditLog().getFsEditTime();
+    if (sig.editsTime != modtime) {
+      throw new IOException("Namenode has an edit log with timestamp of " +
+                            DATE_FORM.format(new Date(modtime)) +
+                            " but new checkpoint was created using editlog " +
+                            " with timestamp " + 
+                            DATE_FORM.format(new Date(sig.editsTime)) + 
+                            ". Checkpoint Aborted.");
+    }
+    sig.validateStorageInfo(this);
+    ckptState = FSImage.CheckpointStates.UPLOAD_START;
+  }
+
+  /**
+   * Start checkpoint.
+   * <p>
+   * If backup storage contains image that is newer than or incompatible with 
+   * what the active name-node has, then the backup node should shutdown.<br>
+   * If the backup image is older than the active one then it should 
+   * be discarded and downloaded from the active node.<br>
+   * If the images are the same then the backup image will be used as current.
+   * 
+   * @param bnReg the backup node registration.
+   * @param nnReg this (active) name-node registration.
+   * @return {@link NamenodeCommand} if backup node should shutdown or
+   * {@link CheckpointCommand} prescribing what backup node should 
+   *         do with its image.
+   * @throws IOException
+   */
+  NamenodeCommand startCheckpoint(NamenodeRegistration bnReg, // backup node
+                                  NamenodeRegistration nnReg) // active name-node
+  throws IOException {
+    String msg = null;
+    // Verify that checkpoint is allowed
+    if(bnReg.getNamespaceID() != this.getNamespaceID())
+      msg = "Name node " + bnReg.getAddress()
+            + " has incompatible namespace id: " + bnReg.getNamespaceID()
+            + " expected: " + getNamespaceID();
+    else if(bnReg.isRole(NamenodeRole.ACTIVE))
+      msg = "Name node " + bnReg.getAddress()
+            + " role " + bnReg.getRole() + ": checkpoint is not allowed.";
+    else if(bnReg.getLayoutVersion() < this.getLayoutVersion()
+        || (bnReg.getLayoutVersion() == this.getLayoutVersion()
+            && bnReg.getCTime() > this.getCTime())
+        || (bnReg.getLayoutVersion() == this.getLayoutVersion()
+            && bnReg.getCTime() == this.getCTime()
+            && bnReg.getCheckpointTime() > this.checkpointTime))
+      // remote node has newer image age
+      msg = "Name node " + bnReg.getAddress()
+            + " has newer image layout version: LV = " +bnReg.getLayoutVersion()
+            + " cTime = " + bnReg.getCTime()
+            + " checkpointTime = " + bnReg.getCheckpointTime()
+            + ". Current version: LV = " + getLayoutVersion()
+            + " cTime = " + getCTime()
+            + " checkpointTime = " + checkpointTime;
+    if(msg != null) {
+      LOG.error(msg);
+      return new NamenodeCommand(NamenodeProtocol.ACT_SHUTDOWN);
+    }
+    boolean isImgObsolete = true;
+    if(bnReg.getLayoutVersion() == this.getLayoutVersion()
+        && bnReg.getCTime() == this.getCTime()
+        && bnReg.getCheckpointTime() == this.checkpointTime)
+      isImgObsolete = false;
+    boolean needToReturnImg = true;
+    if(getNumStorageDirs(NameNodeDirType.IMAGE) == 0)
+      // do not return image if there are no image directories
+      needToReturnImg = false;
+    CheckpointSignature sig = rollEditLog();
+    getEditLog().logJSpoolStart(bnReg, nnReg);
+    return new CheckpointCommand(sig, isImgObsolete, needToReturnImg);
+  }
+
+  /**
+   * End checkpoint.
+   * <p>
+   * Rename uploaded checkpoint to the new image;
+   * purge old edits file;
+   * rename edits.new to edits;
+   * redirect edit log streams to the new edits;
+   * update checkpoint time if the remote node is a checkpoint only node.
+   * 
+   * @param sig
+   * @param remoteNNRole
+   * @throws IOException
+   */
+  void endCheckpoint(CheckpointSignature sig, 
+                     NamenodeRole remoteNNRole) throws IOException {
+    sig.validateStorageInfo(this);
+    // Renew checkpoint time for the active if the other is a checkpoint-node.
+    // The checkpoint-node should have older image for the next checkpoint 
+    // to take effect.
+    // The backup-node always has up-to-date image and will have the same
+    // checkpoint time as the active node.
+    boolean renewCheckpointTime = remoteNNRole.equals(NamenodeRole.CHECKPOINT);
+    rollFSImage(renewCheckpointTime);
+  }
+
+  CheckpointStates getCheckpointState() {
+    return ckptState;
+  }
+
+  void setCheckpointState(CheckpointStates cs) {
+    ckptState = cs;
+  }
+
+  /**
+   * This is called when a checkpoint upload finishes successfully.
+   */
+  synchronized void checkpointUploadDone() {
+    ckptState = CheckpointStates.UPLOAD_DONE;
+  }
+
+  void close() throws IOException {
+    getEditLog().close();
+    unlockAll();
+  }
+
+  /**
+   * Return the name of the image file.
+   */
+  File getFsImageName() {
+    StorageDirectory sd = null;
+    for (Iterator<StorageDirectory> it = 
+      dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
+      sd = it.next();
+      if(sd.getRoot().canRead())
+        return getImageFile(sd, NameNodeFile.IMAGE); 
+    }
+    return null;
+  }
+
+  /**
+   * See if any of removed storages iw "writable" again, and can be returned 
+   * into service
+   */
+  synchronized void attemptRestoreRemovedStorage() {   
+    // if directory is "alive" - copy the images there...
+    if(!restoreFailedStorage || removedStorageDirs.size() == 0) 
+      return; //nothing to restore
+    
+    LOG.info("FSImage.attemptRestoreRemovedStorage: check removed(failed) " +
+    		"storarge. removedStorages size = " + removedStorageDirs.size());
+    for(Iterator<StorageDirectory> it = this.removedStorageDirs.iterator(); it.hasNext();) {
+      StorageDirectory sd = it.next();
+      File root = sd.getRoot();
+      LOG.info("currently disabled dir " + root.getAbsolutePath() + 
+          "; type="+sd.getStorageDirType() + ";canwrite="+root.canWrite());
+      try {
+        
+        if(root.exists() && root.canWrite()) { 
+          format(sd);
+          LOG.info("restoring dir " + sd.getRoot().getAbsolutePath());
+          if(sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
+            File eFile = getEditFile(sd);
+            editLog.addNewEditLogStream(eFile);
+          }
+          this.addStorageDir(sd); // restore
+          it.remove();
+        }
+      } catch(IOException e) {
+        LOG.warn("failed to restore " + sd.getRoot().getAbsolutePath(), e);
+      }
+    }    
+  }
+  
+  public File getFsEditName() throws IOException {
+    return getEditLog().getFsEditName();
+  }
+
+  File getFsTimeName() {
+    StorageDirectory sd = null;
+    // NameNodeFile.TIME shoul be same on all directories
+    for (Iterator<StorageDirectory> it = 
+             dirIterator(); it.hasNext();)
+      sd = it.next();
+    return getImageFile(sd, NameNodeFile.TIME);
+  }
+
+  /**
+   * Return the name of the image file that is uploaded by periodic
+   * checkpointing.
+   */
+  File[] getFsImageNameCheckpoint() {
+    ArrayList<File> list = new ArrayList<File>();
+    for (Iterator<StorageDirectory> it = 
+                 dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
+      list.add(getImageFile(it.next(), NameNodeFile.IMAGE_NEW));
+    }
+    return list.toArray(new File[list.size()]);
+  }
+
+  /**
+   * DatanodeImage is used to store persistent information
+   * about datanodes into the fsImage.
+   */
+  static class DatanodeImage implements Writable {
+    DatanodeDescriptor node = new DatanodeDescriptor();
+
+    /////////////////////////////////////////////////
+    // Writable
+    /////////////////////////////////////////////////
+    /**
+     * Public method that serializes the information about a
+     * Datanode to be stored in the fsImage.
+     */
+    public void write(DataOutput out) throws IOException {
+      new DatanodeID(node).write(out);
+      out.writeLong(node.getCapacity());
+      out.writeLong(node.getRemaining());
+      out.writeLong(node.getLastUpdate());
+      out.writeInt(node.getXceiverCount());
+    }
+
+    /**
+     * Public method that reads a serialized Datanode
+     * from the fsImage.
+     */
+    public void readFields(DataInput in) throws IOException {
+      DatanodeID id = new DatanodeID();
+      id.readFields(in);
+      long capacity = in.readLong();
+      long remaining = in.readLong();
+      long lastUpdate = in.readLong();
+      int xceiverCount = in.readInt();
+
+      // update the DatanodeDescriptor with the data we read in
+      node.updateRegInfo(id);
+      node.setStorageID(id.getStorageID());
+      node.setCapacity(capacity);
+      node.setRemaining(remaining);
+      node.setLastUpdate(lastUpdate);
+      node.setXceiverCount(xceiverCount);
+    }
+  }
+
+  protected void corruptPreUpgradeStorage(File rootDir) throws IOException {
+    File oldImageDir = new File(rootDir, "image");
+    if (!oldImageDir.exists())
+      if (!oldImageDir.mkdir())
+        throw new IOException("Cannot create directory " + oldImageDir);
+    File oldImage = new File(oldImageDir, "fsimage");
+    if (!oldImage.exists())
+      // recreate old image file to let pre-upgrade versions fail
+      if (!oldImage.createNewFile())
+        throw new IOException("Cannot create file " + oldImage);
+    RandomAccessFile oldFile = new RandomAccessFile(oldImage, "rws");
+    // write new version into old image file
+    try {
+      writeCorruptedData(oldFile);
+    } finally {
+      oldFile.close();
+    }
+  }
+
+  private boolean getDistributedUpgradeState() {
+    FSNamesystem ns = getFSNamesystem();
+    return ns == null ? false : ns.getDistributedUpgradeState();
+  }
+
+  private int getDistributedUpgradeVersion() {
+    FSNamesystem ns = getFSNamesystem();
+    return ns == null ? 0 : ns.getDistributedUpgradeVersion();
+  }
+
+  private void setDistributedUpgradeState(boolean uState, int uVersion) {
+    getFSNamesystem().upgradeManager.setUpgradeState(uState, uVersion);
+  }
+
+  private void verifyDistributedUpgradeProgress(StartupOption startOpt
+                                                ) throws IOException {
+    if(startOpt == StartupOption.ROLLBACK || startOpt == StartupOption.IMPORT)
+      return;
+    UpgradeManager um = getFSNamesystem().upgradeManager;
+    assert um != null : "FSNameSystem.upgradeManager is null.";
+    if(startOpt != StartupOption.UPGRADE) {
+      if(um.getUpgradeState())
+        throw new IOException(
+                    "\n   Previous distributed upgrade was not completed. "
+                  + "\n   Please restart NameNode with -upgrade option.");
+      if(um.getDistributedUpgrades() != null)
+        throw new IOException("\n   Distributed upgrade for NameNode version " 
+          + um.getUpgradeVersion() + " to current LV " + FSConstants.LAYOUT_VERSION
+          + " is required.\n   Please restart NameNode with -upgrade option.");
+    }
+  }
+
+  private void initializeDistributedUpgrade() throws IOException {
+    UpgradeManagerNamenode um = getFSNamesystem().upgradeManager;
+    if(! um.initializeUpgrade())
+      return;
+    // write new upgrade state into disk
+    writeAll();
+    NameNode.LOG.info("\n   Distributed upgrade for NameNode version " 
+        + um.getUpgradeVersion() + " to current LV " 
+        + FSConstants.LAYOUT_VERSION + " is initialized.");
+  }
+
+  static Collection<File> getCheckpointDirs(Configuration conf,
+                                            String defaultName) {
+    Collection<String> dirNames = conf.getStringCollection("fs.checkpoint.dir");
+    if (dirNames.size() == 0 && defaultName != null) {
+      dirNames.add(defaultName);
+    }
+    Collection<File> dirs = new ArrayList<File>(dirNames.size());
+    for(String name : dirNames) {
+      dirs.add(new File(name));
+    }
+    return dirs;
+  }
+  
+  static Collection<File> getCheckpointEditsDirs(Configuration conf,
+                                                 String defaultName) {
+    Collection<String> dirNames = 
+                conf.getStringCollection("fs.checkpoint.edits.dir");
+ if (dirNames.size() == 0 && defaultName != null) {
+   dirNames.add(defaultName);
+ }
+ Collection<File> dirs = new ArrayList<File>(dirNames.size());
+ for(String name : dirNames) {
+   dirs.add(new File(name));
+ }
+ return dirs;    
+  }
+
+  static private final DeprecatedUTF8 U_STR = new DeprecatedUTF8();
+  // This should be reverted to package private once the ImageLoader
+  // code is moved into this package. This method should not be called
+  // by other code.
+  public static String readString(DataInputStream in) throws IOException {
+    U_STR.readFields(in);
+    return U_STR.toString();
+  }
+
+  static String readString_EmptyAsNull(DataInputStream in) throws IOException {
+    final String s = readString(in);
+    return s.isEmpty()? null: s;
+  }
+
+  // Same comments apply for this method as for readString()
+  public static byte[] readBytes(DataInputStream in) throws IOException {
+    U_STR.readFields(in);
+    int len = U_STR.getLength();
+    byte[] bytes = new byte[len];
+    System.arraycopy(U_STR.getBytes(), 0, bytes, 0, len);
+    return bytes;
+  }
+
+  static void writeString(String str, DataOutputStream out) throws IOException {
+    U_STR.set(str);
+    U_STR.write(out);
+  }
+}

+ 3726 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -0,0 +1,3726 @@
+/**
+ * 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 org.apache.commons.logging.*;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+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.namenode.BlocksMap.BlockInfo;
+import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
+import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMetrics;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.AccessTokenHandler;
+import org.apache.hadoop.security.ExportedAccessKeys;
+import org.apache.hadoop.security.PermissionChecker;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.*;
+import org.apache.hadoop.metrics.util.MBeanUtil;
+import org.apache.hadoop.net.CachedDNSToSwitchMapping;
+import org.apache.hadoop.net.DNSToSwitchMapping;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.net.NodeBase;
+import org.apache.hadoop.net.ScriptBasedMapping;
+import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
+import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.io.IOUtils;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.DataOutputStream;
+import java.net.InetAddress;
+import java.util.*;
+import java.util.Map.Entry;
+
+import javax.management.NotCompliantMBeanException;
+import javax.management.ObjectName;
+import javax.management.StandardMBean;
+import javax.security.auth.login.LoginException;
+
+/***************************************************
+ * FSNamesystem does the actual bookkeeping work for the
+ * DataNode.
+ *
+ * It tracks several important tables.
+ *
+ * 1)  valid fsname --> blocklist  (kept on disk, logged)
+ * 2)  Set of all valid blocks (inverted #1)
+ * 3)  block --> machinelist (kept in memory, rebuilt dynamically from reports)
+ * 4)  machine --> blocklist (inverted #2)
+ * 5)  LRU cache of updated-heartbeat machines
+ ***************************************************/
+public class FSNamesystem implements FSConstants, FSNamesystemMBean {
+  public static final Log LOG = LogFactory.getLog(FSNamesystem.class);
+  public static final String AUDIT_FORMAT =
+    "ugi=%s\t" +  // ugi
+    "ip=%s\t" +   // remote IP
+    "cmd=%s\t" +  // command
+    "src=%s\t" +  // src path
+    "dst=%s\t" +  // dst path (optional)
+    "perm=%s";    // permissions (optional)
+
+  private static final ThreadLocal<Formatter> auditFormatter =
+    new ThreadLocal<Formatter>() {
+      protected Formatter initialValue() {
+        return new Formatter(new StringBuilder(AUDIT_FORMAT.length() * 4));
+      }
+  };
+
+  private static final void logAuditEvent(UserGroupInformation ugi,
+      InetAddress addr, String cmd, String src, String dst,
+      FileStatus stat) {
+    final Formatter fmt = auditFormatter.get();
+    ((StringBuilder)fmt.out()).setLength(0);
+    auditLog.info(fmt.format(AUDIT_FORMAT, ugi, addr, cmd, src, dst,
+                  (stat == null)
+                    ? null
+                    : stat.getOwner() + ':' + stat.getGroup() + ':' +
+                      stat.getPermission()
+          ).toString());
+
+  }
+
+  public static final Log auditLog = LogFactory.getLog(
+      FSNamesystem.class.getName() + ".audit");
+
+  private boolean isPermissionEnabled;
+  private UserGroupInformation fsOwner;
+  private String supergroup;
+  private PermissionStatus defaultPermission;
+  // FSNamesystemMetrics counter variables
+  private FSNamesystemMetrics myFSMetrics;
+  private long capacityTotal = 0L, capacityUsed = 0L, capacityRemaining = 0L;
+  private int totalLoad = 0;
+  boolean isAccessTokenEnabled;
+  AccessTokenHandler accessTokenHandler;
+  private long accessKeyUpdateInterval;
+  private long accessTokenLifetime;
+
+  //
+  // Stores the correct file name hierarchy
+  //
+  public FSDirectory dir;
+
+  BlockManager blockManager;
+    
+  /**
+   * Stores the datanode -> block map.  
+   * <p>
+   * Done by storing a set of {@link DatanodeDescriptor} objects, sorted by 
+   * storage id. In order to keep the storage map consistent it tracks 
+   * all storages ever registered with the namenode.
+   * A descriptor corresponding to a specific storage id can be
+   * <ul> 
+   * <li>added to the map if it is a new storage id;</li>
+   * <li>updated with a new datanode started as a replacement for the old one 
+   * with the same storage id; and </li>
+   * <li>removed if and only if an existing datanode is restarted to serve a
+   * different storage id.</li>
+   * </ul> <br>
+   * The list of the {@link DatanodeDescriptor}s in the map is checkpointed
+   * in the namespace image file. Only the {@link DatanodeInfo} part is 
+   * persistent, the list of blocks is restored from the datanode block
+   * reports. 
+   * <p>
+   * Mapping: StorageID -> DatanodeDescriptor
+   */
+  NavigableMap<String, DatanodeDescriptor> datanodeMap = 
+    new TreeMap<String, DatanodeDescriptor>();
+
+  Random r = new Random();
+
+  /**
+   * 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,
+   * and removes them from the list.
+   */
+  ArrayList<DatanodeDescriptor> heartbeats = new ArrayList<DatanodeDescriptor>();
+
+  public LeaseManager leaseManager = new LeaseManager(this); 
+
+  //
+  // Threaded object that checks to see if we have been
+  // getting heartbeats from all clients. 
+  //
+  Daemon hbthread = null;   // HeartbeatMonitor thread
+  public Daemon lmthread = null;   // LeaseMonitor thread
+  Daemon smmthread = null;  // SafeModeMonitor thread
+  public Daemon replthread = null;  // Replication thread
+  
+  private volatile boolean fsRunning = true;
+  long systemStart = 0;
+
+  // heartbeatRecheckInterval is how often namenode checks for expired datanodes
+  private long heartbeatRecheckInterval;
+  // heartbeatExpireInterval is how long namenode waits for datanode to report
+  // heartbeat
+  private long heartbeatExpireInterval;
+  //replicationRecheckInterval is how often namenode checks for new replication work
+  private long replicationRecheckInterval;
+  // default block size of a file
+  private long defaultBlockSize = 0;
+  // allow appending to hdfs files
+  private boolean supportAppends = true;
+
+  private volatile SafeModeInfo safeMode;  // safe mode information
+  private Host2NodesMap host2DataNodeMap = new Host2NodesMap();
+    
+  // datanode networktoplogy
+  NetworkTopology clusterMap = new NetworkTopology();
+  private DNSToSwitchMapping dnsToSwitchMapping;
+
+  private HostsFileReader hostsReader; 
+  private Daemon dnthread = null;
+
+  private long maxFsObjects = 0;          // maximum number of fs objects
+
+  /**
+   * The global generation stamp for this file system. 
+   */
+  private final GenerationStamp generationStamp = new GenerationStamp();
+
+  // Ask Datanode only up to this many blocks to delete.
+  int blockInvalidateLimit = FSConstants.BLOCK_INVALIDATE_CHUNK;
+
+  // precision of access times.
+  private long accessTimePrecision = 0;
+
+  /**
+   * FSNamesystem constructor.
+   */
+  FSNamesystem(Configuration conf) throws IOException {
+    try {
+      initialize(conf, null);
+    } catch(IOException e) {
+      LOG.error(getClass().getSimpleName() + " initialization failed.", e);
+      close();
+      throw e;
+    }
+  }
+
+  /**
+   * Initialize FSNamesystem.
+   */
+  private void initialize(Configuration conf, FSImage fsImage) throws IOException {
+    this.systemStart = now();
+    this.blockManager = new BlockManager(this, conf);
+    setConfigurationParameters(conf);
+    this.registerMBean(conf); // register the MBean for the FSNamesystemStutus
+    if(fsImage == null) {
+      this.dir = new FSDirectory(this, conf);
+      StartupOption startOpt = NameNode.getStartupOption(conf);
+      this.dir.loadFSImage(getNamespaceDirs(conf),
+                           getNamespaceEditsDirs(conf), startOpt);
+      long timeTakenToLoadFSImage = now() - systemStart;
+      LOG.info("Finished loading FSImage in " + timeTakenToLoadFSImage + " msecs");
+      NameNode.getNameNodeMetrics().fsImageLoadTime.set(
+                                (int) timeTakenToLoadFSImage);
+    } else {
+      this.dir = new FSDirectory(fsImage, this, conf);
+    }
+    this.safeMode = new SafeModeInfo(conf);
+    this.hostsReader = new HostsFileReader(conf.get("dfs.hosts",""),
+                        conf.get("dfs.hosts.exclude",""));
+    if (isAccessTokenEnabled) {
+      accessTokenHandler = new AccessTokenHandler(true,
+          accessKeyUpdateInterval, accessTokenLifetime);
+    }
+  }
+
+  /**
+   * Activate FSNamesystem daemons.
+   */
+  void activate(Configuration conf) throws IOException {
+    setBlockTotal();
+    blockManager.activate();
+    this.hbthread = new Daemon(new HeartbeatMonitor());
+    this.lmthread = new Daemon(leaseManager.new Monitor());
+    this.replthread = new Daemon(new ReplicationMonitor());
+    hbthread.start();
+    lmthread.start();
+    replthread.start();
+
+    this.dnthread = new Daemon(new DecommissionManager(this).new Monitor(
+        conf.getInt("dfs.namenode.decommission.interval", 30),
+        conf.getInt("dfs.namenode.decommission.nodes.per.interval", 5)));
+    dnthread.start();
+
+    this.dnsToSwitchMapping = ReflectionUtils.newInstance(
+        conf.getClass("topology.node.switch.mapping.impl", ScriptBasedMapping.class,
+            DNSToSwitchMapping.class), conf);
+    
+    /* If the dns to swith mapping supports cache, resolve network 
+     * locations of those hosts in the include list, 
+     * and store the mapping in the cache; so future calls to resolve
+     * will be fast.
+     */
+    if (dnsToSwitchMapping instanceof CachedDNSToSwitchMapping) {
+      dnsToSwitchMapping.resolve(new ArrayList<String>(hostsReader.getHosts()));
+    }
+  }
+
+  public static Collection<File> getNamespaceDirs(Configuration conf) {
+    return getStorageDirs(conf, "dfs.name.dir");
+  }
+
+  public static Collection<File> getStorageDirs(Configuration conf,
+                                                String propertyName) {
+    Collection<String> dirNames = conf.getStringCollection(propertyName);
+    StartupOption startOpt = NameNode.getStartupOption(conf);
+    if(startOpt == StartupOption.IMPORT) {
+      // In case of IMPORT this will get rid of default directories 
+      // but will retain directories specified in hdfs-site.xml
+      // When importing image from a checkpoint, the name-node can
+      // start with empty set of storage directories.
+      Configuration cE = new Configuration(false);
+      cE.addResource("core-default.xml");
+      cE.addResource("core-site.xml");
+      cE.addResource("hdfs-default.xml");
+      Collection<String> dirNames2 = cE.getStringCollection(propertyName);
+      dirNames.removeAll(dirNames2);
+      if(dirNames.isEmpty())
+        LOG.warn("!!! WARNING !!!" +
+          "\n\tThe NameNode currently runs without persistent storage." +
+          "\n\tAny changes to the file system meta-data may be lost." +
+          "\n\tRecommended actions:" +
+          "\n\t\t- shutdown and restart NameNode with configured \"" 
+          + propertyName + "\" in hdfs-site.xml;" +
+          "\n\t\t- use Backup Node as a persistent and up-to-date storage " +
+          "of the file system meta-data.");
+    } else if (dirNames.isEmpty())
+      dirNames.add("/tmp/hadoop/dfs/name");
+    Collection<File> dirs = new ArrayList<File>(dirNames.size());
+    for(String name : dirNames) {
+      dirs.add(new File(name));
+    }
+    return dirs;
+  }
+
+  public static Collection<File> getNamespaceEditsDirs(Configuration conf) {
+    return getStorageDirs(conf, "dfs.name.edits.dir");
+  }
+
+  /**
+   * dirs is a list of directories where the filesystem directory state 
+   * is stored
+   */
+  FSNamesystem(FSImage fsImage, Configuration conf) throws IOException {
+    this.blockManager = new BlockManager(this, conf);
+    setConfigurationParameters(conf);
+    this.dir = new FSDirectory(fsImage, this, conf);
+  }
+
+  /**
+   * Create FSNamesystem for {@link BackupNode}.
+   * Should do everything that would be done for the NameNode,
+   * except for loading the image.
+   * 
+   * @param bnImage {@link BackupStorage}
+   * @param conf configuration
+   * @throws IOException
+   */
+  FSNamesystem(Configuration conf, BackupStorage bnImage) throws IOException {
+    try {
+      initialize(conf, bnImage);
+    } catch(IOException e) {
+      LOG.error(getClass().getSimpleName() + " initialization failed.", e);
+      close();
+      throw e;
+    }
+  }
+
+  /**
+   * Initializes some of the members from configuration
+   */
+  private void setConfigurationParameters(Configuration conf) 
+                                          throws IOException {
+    try {
+      fsOwner = UnixUserGroupInformation.login(conf);
+    } catch (LoginException e) {
+      throw new IOException(StringUtils.stringifyException(e));
+    }
+    LOG.info("fsOwner=" + fsOwner);
+
+    this.supergroup = conf.get("dfs.permissions.supergroup", "supergroup");
+    this.isPermissionEnabled = conf.getBoolean("dfs.permissions", true);
+    LOG.info("supergroup=" + supergroup);
+    LOG.info("isPermissionEnabled=" + isPermissionEnabled);
+    short filePermission = (short)conf.getInt("dfs.upgrade.permission", 00777);
+    this.defaultPermission = PermissionStatus.createImmutable(
+        fsOwner.getUserName(), supergroup, new FsPermission(filePermission));
+
+    long heartbeatInterval = conf.getLong("dfs.heartbeat.interval", 3) * 1000;
+    this.heartbeatRecheckInterval = conf.getInt(
+        "heartbeat.recheck.interval", 5 * 60 * 1000); // 5 minutes
+    this.heartbeatExpireInterval = 2 * heartbeatRecheckInterval +
+      10 * heartbeatInterval;
+    this.replicationRecheckInterval = 
+      conf.getInt("dfs.replication.interval", 3) * 1000L;
+    this.defaultBlockSize = conf.getLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    this.maxFsObjects = conf.getLong("dfs.max.objects", 0);
+    this.blockInvalidateLimit = Math.max(this.blockInvalidateLimit, 
+                                         20*(int)(heartbeatInterval/1000));
+    this.accessTimePrecision = conf.getLong("dfs.access.time.precision", 0);
+    this.supportAppends = conf.getBoolean("dfs.support.append", false);
+    this.isAccessTokenEnabled = conf.getBoolean(
+        AccessTokenHandler.STRING_ENABLE_ACCESS_TOKEN, false);
+    if (isAccessTokenEnabled) {
+      this.accessKeyUpdateInterval = conf.getLong(
+          AccessTokenHandler.STRING_ACCESS_KEY_UPDATE_INTERVAL, 600) * 60 * 1000L; // 10 hrs
+      this.accessTokenLifetime = conf.getLong(
+          AccessTokenHandler.STRING_ACCESS_TOKEN_LIFETIME, 600) * 60 * 1000L; // 10 hrs
+    }
+    LOG.info("isAccessTokenEnabled=" + isAccessTokenEnabled
+        + " accessKeyUpdateInterval=" + accessKeyUpdateInterval / (60 * 1000)
+        + " min(s), accessTokenLifetime=" + accessTokenLifetime / (60 * 1000)
+        + " min(s)");
+  }
+
+  /**
+   * Return the default path permission when upgrading from releases with no
+   * permissions (<=0.15) to releases with permissions (>=0.16)
+   */
+  protected PermissionStatus getUpgradePermission() {
+    return defaultPermission;
+  }
+  
+  NamespaceInfo getNamespaceInfo() {
+    return new NamespaceInfo(dir.fsImage.getNamespaceID(),
+                             dir.fsImage.getCTime(),
+                             getDistributedUpgradeVersion());
+  }
+
+  /**
+   * Close down this file system manager.
+   * Causes heartbeat and lease daemons to stop; waits briefly for
+   * them to finish, but a short timeout returns control back to caller.
+   */
+  public void close() {
+    fsRunning = false;
+    try {
+      if (blockManager != null) blockManager.close();
+      if (hbthread != null) hbthread.interrupt();
+      if (replthread != null) replthread.interrupt();
+      if (dnthread != null) dnthread.interrupt();
+      if (smmthread != null) smmthread.interrupt();
+    } catch (Exception e) {
+      LOG.warn("Exception shutting down FSNamesystem", e);
+    } finally {
+      // using finally to ensure we also wait for lease daemon
+      try {
+        if (lmthread != null) {
+          lmthread.interrupt();
+          lmthread.join(3000);
+        }
+        dir.close();
+      } catch (InterruptedException ie) {
+      } catch (IOException ie) {
+        LOG.error("Error closing FSDirectory", ie);
+        IOUtils.cleanup(LOG, dir);
+      }
+    }
+  }
+
+  /** Is this name system running? */
+  boolean isRunning() {
+    return fsRunning;
+  }
+
+  /**
+   * Dump all metadata into specified file
+   */
+  synchronized void metaSave(String filename) throws IOException {
+    checkSuperuserPrivilege();
+    File file = new File(System.getProperty("hadoop.log.dir"), 
+                         filename);
+    PrintWriter out = new PrintWriter(new BufferedWriter(
+                                                         new FileWriter(file, true)));
+
+    blockManager.metaSave(out);
+
+    //
+    // Dump all datanodes
+    //
+    datanodeDump(out);
+
+    out.flush();
+    out.close();
+  }
+
+  long getDefaultBlockSize() {
+    return defaultBlockSize;
+  }
+
+  long getAccessTimePrecision() {
+    return accessTimePrecision;
+  }
+
+  private boolean isAccessTimeSupported() {
+    return accessTimePrecision > 0;
+  }
+
+  /////////////////////////////////////////////////////////
+  //
+  // These methods are called by secondary namenodes
+  //
+  /////////////////////////////////////////////////////////
+  /**
+   * return a list of blocks & their locations on <code>datanode</code> whose
+   * total size is <code>size</code>
+   * 
+   * @param datanode on which blocks are located
+   * @param size total size of blocks
+   */
+  synchronized BlocksWithLocations getBlocks(DatanodeID datanode, long size)
+      throws IOException {
+    checkSuperuserPrivilege();
+
+    DatanodeDescriptor node = getDatanode(datanode);
+    if (node == null) {
+      NameNode.stateChangeLog.warn("BLOCK* NameSystem.getBlocks: "
+          + "Asking for blocks from an unrecorded node " + datanode.getName());
+      throw new IllegalArgumentException(
+          "Unexpected exception.  Got getBlocks message for datanode " +
+          datanode.getName() + ", but there is no info for it");
+    }
+
+    int numBlocks = node.numBlocks();
+    if(numBlocks == 0) {
+      return new BlocksWithLocations(new BlockWithLocations[0]);
+    }
+    Iterator<Block> iter = node.getBlockIterator();
+    int startBlock = r.nextInt(numBlocks); // starting from a random block
+    // skip blocks
+    for(int i=0; i<startBlock; i++) {
+      iter.next();
+    }
+    List<BlockWithLocations> results = new ArrayList<BlockWithLocations>();
+    long totalSize = 0;
+    while(totalSize<size && iter.hasNext()) {
+      totalSize += addBlock(iter.next(), results);
+    }
+    if(totalSize<size) {
+      iter = node.getBlockIterator(); // start from the beginning
+      for(int i=0; i<startBlock&&totalSize<size; i++) {
+        totalSize += addBlock(iter.next(), results);
+      }
+    }
+
+    return new BlocksWithLocations(
+        results.toArray(new BlockWithLocations[results.size()]));
+  }
+
+  /**
+   * Get access keys
+   * 
+   * @return current access keys
+   */
+  ExportedAccessKeys getAccessKeys() {
+    return isAccessTokenEnabled ? accessTokenHandler.exportKeys()
+        : ExportedAccessKeys.DUMMY_KEYS;
+  }
+
+  /**
+   * Get all valid locations of the block & add the block to results
+   * return the length of the added block; 0 if the block is not added
+   */
+  private long addBlock(Block block, List<BlockWithLocations> results) {
+    ArrayList<String> machineSet = blockManager.addBlock(block);
+    if(machineSet.size() == 0) {
+      return 0;
+    } else {
+      results.add(new BlockWithLocations(block, 
+          machineSet.toArray(new String[machineSet.size()])));
+      return block.getNumBytes();
+    }
+  }
+
+  /////////////////////////////////////////////////////////
+  //
+  // These methods are called by HadoopFS clients
+  //
+  /////////////////////////////////////////////////////////
+  /**
+   * Set permissions for an existing file.
+   * @throws IOException
+   */
+  public synchronized void setPermission(String src, FsPermission permission
+      ) throws IOException {
+    checkOwner(src);
+    dir.setPermission(src, permission);
+    getEditLog().logSync();
+    if (auditLog.isInfoEnabled()) {
+      final FileStatus stat = dir.getFileInfo(src);
+      logAuditEvent(UserGroupInformation.getCurrentUGI(),
+                    Server.getRemoteIp(),
+                    "setPermission", src, null, stat);
+    }
+  }
+
+  /**
+   * Set owner for an existing file.
+   * @throws IOException
+   */
+  public synchronized void setOwner(String src, String username, String group
+      ) throws IOException {
+    FSPermissionChecker pc = checkOwner(src);
+    if (!pc.isSuper) {
+      if (username != null && !pc.user.equals(username)) {
+        throw new AccessControlException("Non-super user cannot change owner.");
+      }
+      if (group != null && !pc.containsGroup(group)) {
+        throw new AccessControlException("User does not belong to " + group
+            + " .");
+      }
+    }
+    dir.setOwner(src, username, group);
+    getEditLog().logSync();
+    if (auditLog.isInfoEnabled()) {
+      final FileStatus stat = dir.getFileInfo(src);
+      logAuditEvent(UserGroupInformation.getCurrentUGI(),
+                    Server.getRemoteIp(),
+                    "setOwner", src, null, stat);
+    }
+  }
+
+  /**
+   * Get block locations within the specified range.
+   * 
+   * @see #getBlockLocations(String, long, long)
+   */
+  LocatedBlocks getBlockLocations(String clientMachine, String src,
+      long offset, long length) throws IOException {
+    if (isPermissionEnabled) {
+      checkPathAccess(src, FsAction.READ);
+    }
+
+    LocatedBlocks blocks = getBlockLocations(src, offset, length, true);
+    if (blocks != null) {
+      //sort the blocks
+      DatanodeDescriptor client = host2DataNodeMap.getDatanodeByHost(
+          clientMachine);
+      for (LocatedBlock b : blocks.getLocatedBlocks()) {
+        clusterMap.pseudoSortByDistance(client, b.getLocations());
+      }
+    }
+    return blocks;
+  }
+
+  /**
+   * Get block locations within the specified range.
+   * @see ClientProtocol#getBlockLocations(String, long, long)
+   */
+  public LocatedBlocks getBlockLocations(String src, long offset, long length
+      ) throws IOException {
+    return getBlockLocations(src, offset, length, false);
+  }
+
+  /**
+   * Get block locations within the specified range.
+   * @see ClientProtocol#getBlockLocations(String, long, long)
+   * @throws FileNotFoundException
+   */
+  public LocatedBlocks getBlockLocations(String src, long offset, long length,
+      boolean doAccessTime) throws IOException {
+    if (offset < 0) {
+      throw new IOException("Negative offset is not supported. File: " + src );
+    }
+    if (length < 0) {
+      throw new IOException("Negative length is not supported. File: " + src );
+    }
+    INodeFile inode = dir.getFileINode(src);
+    if (inode == null)
+      throw new FileNotFoundException();
+    final LocatedBlocks ret = getBlockLocationsInternal(src, inode,
+        offset, length, Integer.MAX_VALUE, doAccessTime);  
+    if (auditLog.isInfoEnabled()) {
+      logAuditEvent(UserGroupInformation.getCurrentUGI(),
+                    Server.getRemoteIp(),
+                    "open", src, null, null);
+    }
+    return ret;
+  }
+
+  private synchronized LocatedBlocks getBlockLocationsInternal(String src,
+                                                       INodeFile inode,
+                                                       long offset, 
+                                                       long length,
+                                                       int nrBlocksToReturn,
+                                                       boolean doAccessTime
+                                                       ) throws IOException {
+    if(inode == null) {
+      return null;
+    }
+    if (doAccessTime && isAccessTimeSupported()) {
+      dir.setTimes(src, inode, -1, now(), false);
+    }
+    Block[] blocks = inode.getBlocks();
+    if (blocks == null) {
+      return null;
+    }
+    if (blocks.length == 0) {
+      return inode.createLocatedBlocks(new ArrayList<LocatedBlock>(blocks.length));
+    }
+    
+    List<LocatedBlock> results = blockManager.getBlockLocations(blocks,
+        offset, length, nrBlocksToReturn);
+    return inode.createLocatedBlocks(results);
+  }
+
+  /**
+   * stores the modification and access time for this inode. 
+   * The access time is precise upto an hour. The transaction, if needed, is
+   * written to the edits log but is not flushed.
+   */
+  public synchronized void setTimes(String src, long mtime, long atime) throws IOException {
+    if (!isAccessTimeSupported() && atime != -1) {
+      throw new IOException("Access time for hdfs is not configured. " +
+                            " Please set dfs.support.accessTime configuration parameter.");
+    }
+    //
+    // The caller needs to have write access to set access & modification times.
+    if (isPermissionEnabled) {
+      checkPathAccess(src, FsAction.WRITE);
+    }
+    INodeFile inode = dir.getFileINode(src);
+    if (inode != null) {
+      dir.setTimes(src, inode, mtime, atime, true);
+      if (auditLog.isInfoEnabled()) {
+        final FileStatus stat = dir.getFileInfo(src);
+        logAuditEvent(UserGroupInformation.getCurrentUGI(),
+                      Server.getRemoteIp(),
+                      "setTimes", src, null, stat);
+      }
+    } else {
+      throw new FileNotFoundException("File " + src + " does not exist.");
+    }
+  }
+
+  /**
+   * Set replication for an existing file.
+   * 
+   * The NameNode sets new replication and schedules either replication of 
+   * under-replicated data blocks or removal of the eccessive block copies 
+   * if the blocks are over-replicated.
+   * 
+   * @see ClientProtocol#setReplication(String, short)
+   * @param src file name
+   * @param replication new replication
+   * @return true if successful; 
+   *         false if file does not exist or is a directory
+   */
+  public boolean setReplication(String src, short replication) 
+                                throws IOException {
+    boolean status = setReplicationInternal(src, replication);
+    getEditLog().logSync();
+    if (status && auditLog.isInfoEnabled()) {
+      logAuditEvent(UserGroupInformation.getCurrentUGI(),
+                    Server.getRemoteIp(),
+                    "setReplication", src, null, null);
+    }
+    return status;
+  }
+
+  private synchronized boolean setReplicationInternal(String src, 
+                                             short replication
+                                             ) throws IOException {
+    if (isInSafeMode())
+      throw new SafeModeException("Cannot set replication for " + src, safeMode);
+    blockManager.verifyReplication(src, replication, null);
+    if (isPermissionEnabled) {
+      checkPathAccess(src, FsAction.WRITE);
+    }
+
+    int[] oldReplication = new int[1];
+    Block[] fileBlocks;
+    fileBlocks = dir.setReplication(src, replication, oldReplication);
+    if (fileBlocks == null)  // file not found or is a directory
+      return false;
+    int oldRepl = oldReplication[0];
+    if (oldRepl == replication) // the same replication
+      return true;
+
+    // update needReplication priority queues
+    for(int idx = 0; idx < fileBlocks.length; idx++)
+      blockManager.updateNeededReplications(fileBlocks[idx], 0, replication-oldRepl);
+      
+    if (oldRepl > replication) {  
+      // old replication > the new one; need to remove copies
+      LOG.info("Reducing replication for file " + src 
+               + ". New replication is " + replication);
+      for(int idx = 0; idx < fileBlocks.length; idx++)
+        blockManager.processOverReplicatedBlock(fileBlocks[idx], replication, null, null);
+    } else { // replication factor is increased
+      LOG.info("Increasing replication for file " + src 
+          + ". New replication is " + replication);
+    }
+    return true;
+  }
+    
+  long getPreferredBlockSize(String filename) throws IOException {
+    if (isPermissionEnabled) {
+      checkTraverse(filename);
+    }
+    return dir.getPreferredBlockSize(filename);
+  }
+
+  /**
+   * Create a new file entry in the namespace.
+   * 
+   * @see ClientProtocol#create(String, FsPermission, String, boolean, short, long)
+   * 
+   * @throws IOException if file name is invalid
+   *         {@link FSDirectory#isValidToCreate(String)}.
+   */
+  void startFile(String src, PermissionStatus permissions,
+                 String holder, String clientMachine,
+                 boolean overwrite, short replication, long blockSize
+                ) throws IOException {
+    startFileInternal(src, permissions, holder, clientMachine, overwrite, false,
+                      replication, blockSize);
+    getEditLog().logSync();
+    if (auditLog.isInfoEnabled()) {
+      final FileStatus stat = dir.getFileInfo(src);
+      logAuditEvent(UserGroupInformation.getCurrentUGI(),
+                    Server.getRemoteIp(),
+                    "create", src, null, stat);
+    }
+  }
+
+  private synchronized void startFileInternal(String src,
+                                              PermissionStatus permissions,
+                                              String holder, 
+                                              String clientMachine, 
+                                              boolean overwrite,
+                                              boolean append,
+                                              short replication,
+                                              long blockSize
+                                              ) throws IOException {
+    if (NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: src=" + src
+          + ", holder=" + holder
+          + ", clientMachine=" + clientMachine
+          + ", replication=" + replication
+          + ", overwrite=" + overwrite
+          + ", append=" + append);
+    }
+
+    if (isInSafeMode())
+      throw new SafeModeException("Cannot create file" + src, safeMode);
+    if (!DFSUtil.isValidName(src)) {
+      throw new IOException("Invalid file name: " + src);
+    }
+
+    // Verify that the destination does not exist as a directory already.
+    boolean pathExists = dir.exists(src);
+    if (pathExists && dir.isDir(src)) {
+      throw new IOException("Cannot create file "+ src + "; already exists as a directory.");
+    }
+
+    if (isPermissionEnabled) {
+      if (append || (overwrite && pathExists)) {
+        checkPathAccess(src, FsAction.WRITE);
+      }
+      else {
+        checkAncestorAccess(src, FsAction.WRITE);
+      }
+    }
+
+    try {
+      INode myFile = dir.getFileINode(src);
+      if (myFile != null && myFile.isUnderConstruction()) {
+        INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction) myFile;
+        //
+        // If the file is under construction , then it must be in our
+        // leases. Find the appropriate lease record.
+        //
+        Lease lease = leaseManager.getLease(holder);
+        //
+        // We found the lease for this file. And surprisingly the original
+        // holder is trying to recreate this file. This should never occur.
+        //
+        if (lease != null) {
+          throw new AlreadyBeingCreatedException(
+                                                 "failed to create file " + src + " for " + holder +
+                                                 " on client " + clientMachine + 
+                                                 " because current leaseholder is trying to recreate file.");
+        }
+        //
+        // Find the original holder.
+        //
+        lease = leaseManager.getLease(pendingFile.clientName);
+        if (lease == null) {
+          throw new AlreadyBeingCreatedException(
+                                                 "failed to create file " + src + " for " + holder +
+                                                 " on client " + clientMachine + 
+                                                 " because pendingCreates is non-null but no leases found.");
+        }
+        //
+        // If the original holder has not renewed in the last SOFTLIMIT 
+        // period, then start lease recovery.
+        //
+        if (lease.expiredSoftLimit()) {
+          LOG.info("startFile: recover lease " + lease + ", src=" + src);
+          internalReleaseLease(lease, src);
+        }
+        throw new AlreadyBeingCreatedException("failed to create file " + src + " for " + holder +
+                                               " on client " + clientMachine + 
+                                               ", because this file is already being created by " +
+                                               pendingFile.getClientName() + 
+                                               " on " + pendingFile.getClientMachine());
+      }
+
+      try {
+        blockManager.verifyReplication(src, replication, clientMachine);
+      } catch(IOException e) {
+        throw new IOException("failed to create "+e.getMessage());
+      }
+      if (append) {
+        if (myFile == null) {
+          throw new FileNotFoundException("failed to append to non-existent file "
+              + src + " on client " + clientMachine);
+        } else if (myFile.isDirectory()) {
+          throw new IOException("failed to append to directory " + src 
+                                +" on client " + clientMachine);
+        }
+      } else if (!dir.isValidToCreate(src)) {
+        if (overwrite) {
+          delete(src, true);
+        } else {
+          throw new IOException("failed to create file " + src 
+                                +" on client " + clientMachine
+                                +" either because the filename is invalid or the file exists");
+        }
+      }
+
+      DatanodeDescriptor clientNode = 
+        host2DataNodeMap.getDatanodeByHost(clientMachine);
+
+      if (append) {
+        //
+        // Replace current node with a INodeUnderConstruction.
+        // Recreate in-memory lease record.
+        //
+        INodeFile node = (INodeFile) myFile;
+        INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
+                                        node.getLocalNameBytes(),
+                                        node.getReplication(),
+                                        node.getModificationTime(),
+                                        node.getPreferredBlockSize(),
+                                        node.getBlocks(),
+                                        node.getPermissionStatus(),
+                                        holder,
+                                        clientMachine,
+                                        clientNode);
+        dir.replaceNode(src, node, cons);
+        leaseManager.addLease(cons.clientName, src);
+
+      } else {
+       // Now we can add the name to the filesystem. This file has no
+       // blocks associated with it.
+       //
+       checkFsObjectLimit();
+
+        // increment global generation stamp
+        long genstamp = nextGenerationStamp();
+        INodeFileUnderConstruction newNode = dir.addFile(src, permissions,
+            replication, blockSize, holder, clientMachine, clientNode, genstamp);
+        if (newNode == null) {
+          throw new IOException("DIR* NameSystem.startFile: " +
+                                "Unable to add file to namespace.");
+        }
+        leaseManager.addLease(newNode.clientName, src);
+        if (NameNode.stateChangeLog.isDebugEnabled()) {
+          NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: "
+                                     +"add "+src+" to namespace for "+holder);
+        }
+      }
+    } catch (IOException ie) {
+      NameNode.stateChangeLog.warn("DIR* NameSystem.startFile: "
+                                   +ie.getMessage());
+      throw ie;
+    }
+  }
+
+  /**
+   * Append to an existing file in the namespace.
+   */
+  LocatedBlock appendFile(String src, String holder, String clientMachine
+      ) throws IOException {
+    if (supportAppends == false) {
+      throw new IOException("Append to hdfs not supported." +
+                            " Please refer to dfs.support.append configuration parameter.");
+    }
+    startFileInternal(src, null, holder, clientMachine, false, true, 
+                      (short)blockManager.maxReplication, (long)0);
+    getEditLog().logSync();
+
+    //
+    // Create a LocatedBlock object for the last block of the file
+    // to be returned to the client. Return null if the file does not
+    // have a partial block at the end.
+    //
+    LocatedBlock lb = null;
+    synchronized (this) {
+      INodeFileUnderConstruction file = (INodeFileUnderConstruction)dir.getFileINode(src);
+
+      Block[] blocks = file.getBlocks();
+      if (blocks != null && blocks.length > 0) {
+        Block last = blocks[blocks.length-1];
+        BlockInfo storedBlock = blockManager.getStoredBlock(last);
+        if (file.getPreferredBlockSize() > storedBlock.getNumBytes()) {
+          long fileLength = file.computeContentSummary().getLength();
+          DatanodeDescriptor[] targets = blockManager.getNodes(last);
+          // remove the replica locations of this block from the node
+          for (int i = 0; i < targets.length; i++) {
+            targets[i].removeBlock(storedBlock);
+          }
+          // set the locations of the last block in the lease record
+          file.setLastBlock(storedBlock, targets);
+
+          lb = new LocatedBlock(last, targets, 
+                                fileLength-storedBlock.getNumBytes());
+          if (isAccessTokenEnabled) {
+            lb.setAccessToken(accessTokenHandler.generateToken(lb.getBlock()
+                .getBlockId(), EnumSet.of(AccessTokenHandler.AccessMode.WRITE)));
+          }
+
+          // Remove block from replication queue.
+          blockManager.updateNeededReplications(last, 0, 0);
+
+          // remove this block from the list of pending blocks to be deleted. 
+          // This reduces the possibility of triggering HADOOP-1349.
+          //
+          for (DatanodeDescriptor dd : targets) {
+            String datanodeId = dd.getStorageID();
+            blockManager.removeFromInvalidates(datanodeId, last);
+          }
+        }
+      }
+    }
+    if (lb != null) {
+      if (NameNode.stateChangeLog.isDebugEnabled()) {
+        NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: file "
+            +src+" for "+holder+" at "+clientMachine
+            +" block " + lb.getBlock()
+            +" block size " + lb.getBlock().getNumBytes());
+      }
+    }
+
+    if (auditLog.isInfoEnabled()) {
+      logAuditEvent(UserGroupInformation.getCurrentUGI(),
+                    Server.getRemoteIp(),
+                    "append", src, null, null);
+    }
+    return lb;
+  }
+
+  /**
+   * The client would like to obtain an additional block for the indicated
+   * filename (which is being written-to).  Return an array that consists
+   * of the block, plus a set of machines.  The first on this list should
+   * be where the client writes data.  Subsequent items in the list must
+   * be provided in the connection to the first datanode.
+   *
+   * Make sure the previous blocks have been reported by datanodes and
+   * are replicated.  Will return an empty 2-elt array if we want the
+   * client to "try again later".
+   */
+  public LocatedBlock getAdditionalBlock(String src, 
+                                         String clientName
+                                         ) throws IOException {
+    long fileLength, blockSize;
+    int replication;
+    DatanodeDescriptor clientNode = null;
+    Block newBlock = null;
+
+    NameNode.stateChangeLog.debug("BLOCK* NameSystem.getAdditionalBlock: file "
+                                  +src+" for "+clientName);
+
+    synchronized (this) {
+      if (isInSafeMode()) {
+        throw new SafeModeException("Cannot add block to " + src, safeMode);
+      }
+
+      // have we exceeded the configured limit of fs objects.
+      checkFsObjectLimit();
+
+      INodeFileUnderConstruction pendingFile  = checkLease(src, clientName);
+
+      //
+      // If we fail this, bad things happen!
+      //
+      if (!checkFileProgress(pendingFile, false)) {
+        throw new NotReplicatedYetException("Not replicated yet:" + src);
+      }
+      fileLength = pendingFile.computeContentSummary().getLength();
+      blockSize = pendingFile.getPreferredBlockSize();
+      clientNode = pendingFile.getClientNode();
+      replication = (int)pendingFile.getReplication();
+    }
+
+    // choose targets for the new block to be allocated.
+    DatanodeDescriptor targets[] = blockManager.replicator.chooseTarget(
+        replication, clientNode, null, blockSize);
+    if (targets.length < blockManager.minReplication) {
+      throw new IOException("File " + src + " could only be replicated to " +
+                            targets.length + " nodes, instead of " +
+                            blockManager.minReplication);
+    }
+
+    // Allocate a new block and record it in the INode. 
+    synchronized (this) {
+      INode[] pathINodes = dir.getExistingPathINodes(src);
+      int inodesLen = pathINodes.length;
+      checkLease(src, clientName, pathINodes[inodesLen-1]);
+      INodeFileUnderConstruction pendingFile  = (INodeFileUnderConstruction) 
+                                                pathINodes[inodesLen - 1];
+                                                           
+      if (!checkFileProgress(pendingFile, false)) {
+        throw new NotReplicatedYetException("Not replicated yet:" + src);
+      }
+
+      // allocate new block record block locations in INode.
+      newBlock = allocateBlock(src, pathINodes);
+      pendingFile.setTargets(targets);
+      
+      for (DatanodeDescriptor dn : targets) {
+        dn.incBlocksScheduled();
+      }      
+    }
+        
+    // Create next block
+    LocatedBlock b = new LocatedBlock(newBlock, targets, fileLength);
+    if (isAccessTokenEnabled) {
+      b.setAccessToken(accessTokenHandler.generateToken(b.getBlock()
+          .getBlockId(), EnumSet.of(AccessTokenHandler.AccessMode.WRITE)));
+    }
+    return b;
+  }
+
+  /**
+   * The client would like to let go of the given block
+   */
+  public synchronized boolean abandonBlock(Block b, String src, String holder
+      ) throws IOException {
+    //
+    // Remove the block from the pending creates list
+    //
+    NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: "
+                                  +b+"of file "+src);
+    INodeFileUnderConstruction file = checkLease(src, holder);
+    dir.removeBlock(src, file, b);
+    NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: "
+                                    + b
+                                    + " is removed from pendingCreates");
+    return true;
+  }
+  
+  // make sure that we still have the lease on this file.
+  private INodeFileUnderConstruction checkLease(String src, String holder) 
+                                                      throws IOException {
+    INodeFile file = dir.getFileINode(src);
+    checkLease(src, holder, file);
+    return (INodeFileUnderConstruction)file;
+  }
+
+  private void checkLease(String src, String holder, INode file) 
+                                                     throws IOException {
+
+    if (file == null || file.isDirectory()) {
+      Lease lease = leaseManager.getLease(holder);
+      throw new LeaseExpiredException("No lease on " + src +
+                                      " File does not exist. " +
+                                      (lease != null ? lease.toString() :
+                                       "Holder " + holder + 
+                                       " does not have any open files."));
+    }
+    if (!file.isUnderConstruction()) {
+      Lease lease = leaseManager.getLease(holder);
+      throw new LeaseExpiredException("No lease on " + src + 
+                                      " File is not open for writing. " +
+                                      (lease != null ? lease.toString() :
+                                       "Holder " + holder + 
+                                       " does not have any open files."));
+    }
+    INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction)file;
+    if (holder != null && !pendingFile.getClientName().equals(holder)) {
+      throw new LeaseExpiredException("Lease mismatch on " + src + " owned by "
+          + pendingFile.getClientName() + " but is accessed by " + holder);
+    }
+  }
+
+  /**
+   * The FSNamesystem will already know the blocks that make up the file.
+   * Before we return, we make sure that all the file's blocks have 
+   * been reported by datanodes and are replicated correctly.
+   */
+  
+  enum CompleteFileStatus {
+    OPERATION_FAILED,
+    STILL_WAITING,
+    COMPLETE_SUCCESS
+  }
+  
+  public CompleteFileStatus completeFile(String src, String holder) throws IOException {
+    CompleteFileStatus status = completeFileInternal(src, holder);
+    getEditLog().logSync();
+    return status;
+  }
+
+
+  private synchronized CompleteFileStatus completeFileInternal(String src, 
+                                                String holder) throws IOException {
+    NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " + src + " for " + holder);
+    if (isInSafeMode())
+      throw new SafeModeException("Cannot complete file " + src, safeMode);
+    INode iFile = dir.getFileINode(src);
+    INodeFileUnderConstruction pendingFile = null;
+    Block[] fileBlocks = null;
+
+    if (iFile != null && iFile.isUnderConstruction()) {
+      pendingFile = (INodeFileUnderConstruction) iFile;
+      fileBlocks =  dir.getFileBlocks(src);
+    }
+    if (fileBlocks == null ) {    
+      NameNode.stateChangeLog.warn("DIR* NameSystem.completeFile: "
+                                   + "failed to complete " + src
+                                   + " because dir.getFileBlocks() is null " + 
+                                   " and pendingFile is " + 
+                                   ((pendingFile == null) ? "null" : 
+                                     ("from " + pendingFile.getClientMachine()))
+                                  );                      
+      return CompleteFileStatus.OPERATION_FAILED;
+    } else if (!checkFileProgress(pendingFile, true)) {
+      return CompleteFileStatus.STILL_WAITING;
+    }
+
+    finalizeINodeFileUnderConstruction(src, pendingFile);
+
+    NameNode.stateChangeLog.info("DIR* NameSystem.completeFile: file " + src
+                                  + " is closed by " + holder);
+    return CompleteFileStatus.COMPLETE_SUCCESS;
+  }
+
+  /** 
+   * Check all blocks of a file. If any blocks are lower than their intended
+   * replication factor, then insert them into neededReplication
+   */
+  private void checkReplicationFactor(INodeFile file) {
+    int numExpectedReplicas = file.getReplication();
+    Block[] pendingBlocks = file.getBlocks();
+    int nrBlocks = pendingBlocks.length;
+    for (int i = 0; i < nrBlocks; i++) {
+      blockManager.checkReplication(pendingBlocks[i], numExpectedReplicas);
+    }
+  }
+
+  static Random randBlockId = new Random();
+    
+  /**
+   * Allocate a block at the given pending filename
+   * 
+   * @param src path to the file
+   * @param inodes INode representing each of the components of src. 
+   *        <code>inodes[inodes.length-1]</code> is the INode for the file.
+   */
+  private Block allocateBlock(String src, INode[] inodes) throws IOException {
+    Block b = new Block(FSNamesystem.randBlockId.nextLong(), 0, 0); 
+    while(isValidBlock(b)) {
+      b.setBlockId(FSNamesystem.randBlockId.nextLong());
+    }
+    b.setGenerationStamp(getGenerationStamp());
+    b = dir.addBlock(src, inodes, b);
+    NameNode.stateChangeLog.info("BLOCK* NameSystem.allocateBlock: "
+                                 +src+ ". "+b);
+    return b;
+  }
+
+  /**
+   * Check that the indicated file's blocks are present and
+   * replicated.  If not, return false. If checkall is true, then check
+   * all blocks, otherwise check only penultimate block.
+   */
+  synchronized boolean checkFileProgress(INodeFile v, boolean checkall) {
+    if (checkall) {
+      //
+      // check all blocks of the file.
+      //
+      for (Block block: v.getBlocks()) {
+        if (!blockManager.checkMinReplication(block)) {
+          return false;
+        }
+      }
+    } else {
+      //
+      // check the penultimate block of this file
+      //
+      Block b = v.getPenultimateBlock();
+      if (b != null && !blockManager.checkMinReplication(b)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+
+  /**
+   * Mark the block belonging to datanode as corrupt
+   * @param blk Block to be marked as corrupt
+   * @param dn Datanode which holds the corrupt replica
+   */
+  public synchronized void markBlockAsCorrupt(Block blk, DatanodeInfo dn)
+    throws IOException {
+    blockManager.markBlockAsCorrupt(blk, dn);
+  }
+
+
+  ////////////////////////////////////////////////////////////////
+  // Here's how to handle block-copy failure during client write:
+  // -- As usual, the client's write should result in a streaming
+  // backup write to a k-machine sequence.
+  // -- If one of the backup machines fails, no worries.  Fail silently.
+  // -- Before client is allowed to close and finalize file, make sure
+  // that the blocks are backed up.  Namenode may have to issue specific backup
+  // commands to make up for earlier datanode failures.  Once all copies
+  // are made, edit namespace and return to client.
+  ////////////////////////////////////////////////////////////////
+
+  /** Change the indicated filename. */
+  public boolean renameTo(String src, String dst) throws IOException {
+    boolean status = renameToInternal(src, dst);
+    getEditLog().logSync();
+    if (status && auditLog.isInfoEnabled()) {
+      final FileStatus stat = dir.getFileInfo(dst);
+      logAuditEvent(UserGroupInformation.getCurrentUGI(),
+                    Server.getRemoteIp(),
+                    "rename", src, dst, stat);
+    }
+    return status;
+  }
+
+  private synchronized boolean renameToInternal(String src, String dst
+      ) throws IOException {
+    NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: " + src + " to " + dst);
+    if (isInSafeMode())
+      throw new SafeModeException("Cannot rename " + src, safeMode);
+    if (!DFSUtil.isValidName(dst)) {
+      throw new IOException("Invalid name: " + dst);
+    }
+
+    if (isPermissionEnabled) {
+      //We should not be doing this.  This is move() not renameTo().
+      //but for now,
+      String actualdst = dir.isDir(dst)?
+          dst + Path.SEPARATOR + new Path(src).getName(): dst;
+      checkParentAccess(src, FsAction.WRITE);
+      checkAncestorAccess(actualdst, FsAction.WRITE);
+    }
+
+    FileStatus dinfo = dir.getFileInfo(dst);
+    if (dir.renameTo(src, dst)) {
+      changeLease(src, dst, dinfo);     // update lease with new filename
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Remove the indicated filename from namespace. If the filename 
+   * is a directory (non empty) and recursive is set to false then throw exception.
+   */
+    public boolean delete(String src, boolean recursive) throws IOException {
+      if ((!recursive) && (!dir.isDirEmpty(src))) {
+        throw new IOException(src + " is non empty");
+      }
+      boolean status = deleteInternal(src, true);
+      getEditLog().logSync();
+      if (status && auditLog.isInfoEnabled()) {
+        logAuditEvent(UserGroupInformation.getCurrentUGI(),
+                      Server.getRemoteIp(),
+                      "delete", src, null, null);
+      }
+      return status;
+    }
+    
+  /**
+   * Remove the indicated filename from the namespace.  This may
+   * invalidate some blocks that make up the file.
+   */
+  synchronized boolean deleteInternal(String src, 
+      boolean enforcePermission) throws IOException {
+    if (NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("DIR* NameSystem.delete: " + src);
+    }
+    if (isInSafeMode())
+      throw new SafeModeException("Cannot delete " + src, safeMode);
+    if (enforcePermission && isPermissionEnabled) {
+      checkPermission(src, false, null, FsAction.WRITE, null, FsAction.ALL);
+    }
+
+    return dir.delete(src) != null;
+  }
+
+  void removePathAndBlocks(String src, List<Block> blocks) {
+    leaseManager.removeLeaseWithPrefixPath(src);
+    for(Block b : blocks) {
+      blockManager.removeBlock(b);
+    }
+  }
+
+  /** Get the file info for a specific file.
+   * @param src The string representation of the path to the file
+   * @throws IOException if permission to access file is denied by the system 
+   * @return object containing information regarding the file
+   *         or null if file not found
+   */
+  FileStatus getFileInfo(String src) throws IOException {
+    if (isPermissionEnabled) {
+      checkTraverse(src);
+    }
+    return dir.getFileInfo(src);
+  }
+
+  /**
+   * Create all the necessary directories
+   */
+  public boolean mkdirs(String src, PermissionStatus permissions
+      ) throws IOException {
+    boolean status = mkdirsInternal(src, permissions);
+    getEditLog().logSync();
+    if (status && auditLog.isInfoEnabled()) {
+      final FileStatus stat = dir.getFileInfo(src);
+      logAuditEvent(UserGroupInformation.getCurrentUGI(),
+                    Server.getRemoteIp(),
+                    "mkdirs", src, null, stat);
+    }
+    return status;
+  }
+    
+  /**
+   * Create all the necessary directories
+   */
+  private synchronized boolean mkdirsInternal(String src,
+      PermissionStatus permissions) throws IOException {
+    NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
+    if (isPermissionEnabled) {
+      checkTraverse(src);
+    }
+    if (dir.isDir(src)) {
+      // all the users of mkdirs() are used to expect 'true' even if
+      // a new directory is not created.
+      return true;
+    }
+    if (isInSafeMode())
+      throw new SafeModeException("Cannot create directory " + src, safeMode);
+    if (!DFSUtil.isValidName(src)) {
+      throw new IOException("Invalid directory name: " + src);
+    }
+    if (isPermissionEnabled) {
+      checkAncestorAccess(src, FsAction.WRITE);
+    }
+
+    // validate that we have enough inodes. This is, at best, a 
+    // heuristic because the mkdirs() operation migth need to 
+    // create multiple inodes.
+    checkFsObjectLimit();
+
+    if (!dir.mkdirs(src, permissions, false, now())) {
+      throw new IOException("Invalid directory name: " + src);
+    }
+    return true;
+  }
+
+  ContentSummary getContentSummary(String src) throws IOException {
+    if (isPermissionEnabled) {
+      checkPermission(src, false, null, null, null, FsAction.READ_EXECUTE);
+    }
+    return dir.getContentSummary(src);
+  }
+
+  /**
+   * Set the namespace quota and diskspace quota for a directory.
+   * See {@link ClientProtocol#setQuota(String, long, long)} for the 
+   * contract.
+   */
+  void setQuota(String path, long nsQuota, long dsQuota) throws IOException {
+    if (isPermissionEnabled) {
+      checkSuperuserPrivilege();
+    }
+    
+    dir.setQuota(path, nsQuota, dsQuota);
+    getEditLog().logSync();
+  }
+  
+  /** Persist all metadata about this file.
+   * @param src The string representation of the path
+   * @param clientName The string representation of the client
+   * @throws IOException if path does not exist
+   */
+  void fsync(String src, String clientName) throws IOException {
+
+    NameNode.stateChangeLog.info("BLOCK* NameSystem.fsync: file "
+                                  + src + " for " + clientName);
+    synchronized (this) {
+      if (isInSafeMode()) {
+        throw new SafeModeException("Cannot fsync file " + src, safeMode);
+      }
+      INodeFileUnderConstruction pendingFile  = checkLease(src, clientName);
+      dir.persistBlocks(src, pendingFile);
+    }
+  }
+
+  /**
+   * Move a file that is being written to be immutable.
+   * @param src The filename
+   * @param lease The lease for the client creating the file
+   */
+  void internalReleaseLease(Lease lease, String src) throws IOException {
+    LOG.info("Recovering lease=" + lease + ", src=" + src);
+
+    INodeFile iFile = dir.getFileINode(src);
+    if (iFile == null) {
+      final String message = "DIR* NameSystem.internalReleaseCreate: "
+        + "attempt to release a create lock on "
+        + src + " file does not exist.";
+      NameNode.stateChangeLog.warn(message);
+      throw new IOException(message);
+    }
+    if (!iFile.isUnderConstruction()) {
+      final String message = "DIR* NameSystem.internalReleaseCreate: "
+        + "attempt to release a create lock on "
+        + src + " but file is already closed.";
+      NameNode.stateChangeLog.warn(message);
+      throw new IOException(message);
+    }
+
+    INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction) iFile;
+
+    // Initialize lease recovery for pendingFile. If there are no blocks 
+    // associated with this file, then reap lease immediately. Otherwise 
+    // renew the lease and trigger lease recovery.
+    if (pendingFile.getTargets() == null ||
+        pendingFile.getTargets().length == 0) {
+      if (pendingFile.getBlocks().length == 0) {
+        finalizeINodeFileUnderConstruction(src, pendingFile);
+        NameNode.stateChangeLog.warn("BLOCK*"
+          + " internalReleaseLease: No blocks found, lease removed.");
+        return;
+      }
+      // setup the Inode.targets for the last block from the blockManager
+      //
+      Block[] blocks = pendingFile.getBlocks();
+      Block last = blocks[blocks.length-1];
+      DatanodeDescriptor[] targets = blockManager.getNodes(last);
+      pendingFile.setTargets(targets);
+    }
+    // start lease recovery of the last block for this file.
+    pendingFile.assignPrimaryDatanode();
+    leaseManager.renewLease(lease);
+  }
+
+  private void finalizeINodeFileUnderConstruction(String src,
+      INodeFileUnderConstruction pendingFile) throws IOException {
+    leaseManager.removeLease(pendingFile.clientName, src);
+
+    // The file is no longer pending.
+    // Create permanent INode, update blockmap
+    INodeFile newFile = pendingFile.convertToInodeFile();
+    dir.replaceNode(src, pendingFile, newFile);
+
+    // close file and persist block allocations for this file
+    dir.closeFile(src, newFile);
+
+    checkReplicationFactor(newFile);
+  }
+
+  synchronized void commitBlockSynchronization(Block lastblock,
+      long newgenerationstamp, long newlength,
+      boolean closeFile, boolean deleteblock, DatanodeID[] newtargets
+      ) throws IOException {
+    LOG.info("commitBlockSynchronization(lastblock=" + lastblock
+          + ", newgenerationstamp=" + newgenerationstamp
+          + ", newlength=" + newlength
+          + ", newtargets=" + Arrays.asList(newtargets)
+          + ", closeFile=" + closeFile
+          + ", deleteBlock=" + deleteblock
+          + ")");
+    final BlockInfo oldblockinfo = blockManager.getStoredBlock(lastblock);
+    if (oldblockinfo == null) {
+      throw new IOException("Block (=" + lastblock + ") not found");
+    }
+    INodeFile iFile = oldblockinfo.getINode();
+    if (!iFile.isUnderConstruction()) {
+      throw new IOException("Unexpected block (=" + lastblock
+          + ") since the file (=" + iFile.getLocalName()
+          + ") is not under construction");
+    }
+    INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction)iFile;
+
+
+    // Remove old block from blocks map. This always have to be done
+    // because the generation stamp of this block is changing.
+    blockManager.removeBlockFromMap(oldblockinfo);
+
+    if (deleteblock) {
+      pendingFile.removeBlock(lastblock);
+    }
+    else {
+      // update last block, construct newblockinfo and add it to the blocks map
+      lastblock.set(lastblock.getBlockId(), newlength, newgenerationstamp);
+      final BlockInfo newblockinfo = blockManager.addINode(lastblock, pendingFile);
+
+      // find the DatanodeDescriptor objects
+      // There should be no locations in the blockManager till now because the
+      // file is underConstruction
+      DatanodeDescriptor[] descriptors = null;
+      if (newtargets.length > 0) {
+        descriptors = new DatanodeDescriptor[newtargets.length];
+        for(int i = 0; i < newtargets.length; i++) {
+          descriptors[i] = getDatanode(newtargets[i]);
+        }
+      }
+      if (closeFile) {
+        // the file is getting closed. Insert block locations into blockManager.
+        // Otherwise fsck will report these blocks as MISSING, especially if the
+        // blocksReceived from Datanodes take a long time to arrive.
+        for (int i = 0; i < descriptors.length; i++) {
+          descriptors[i].addBlock(newblockinfo);
+        }
+        pendingFile.setLastBlock(newblockinfo, null);
+      } else {
+        // add locations into the INodeUnderConstruction
+        pendingFile.setLastBlock(newblockinfo, descriptors);
+      }
+    }
+
+    // If this commit does not want to close the file, persist
+    // blocks only if append is supported and return
+    String src = leaseManager.findPath(pendingFile);
+    if (!closeFile) {
+      if (supportAppends) {
+        dir.persistBlocks(src, pendingFile);
+        getEditLog().logSync();
+      }
+      LOG.info("commitBlockSynchronization(" + lastblock + ") successful");
+      return;
+    }
+    
+    //remove lease, close file
+    finalizeINodeFileUnderConstruction(src, pendingFile);
+    getEditLog().logSync();
+    LOG.info("commitBlockSynchronization(newblock=" + lastblock
+          + ", file=" + src
+          + ", newgenerationstamp=" + newgenerationstamp
+          + ", newlength=" + newlength
+          + ", newtargets=" + Arrays.asList(newtargets) + ") successful");
+  }
+
+
+  /**
+   * Renew the lease(s) held by the given client
+   */
+  void renewLease(String holder) throws IOException {
+    if (isInSafeMode())
+      throw new SafeModeException("Cannot renew lease for " + holder, safeMode);
+    leaseManager.renewLease(holder);
+  }
+
+  /**
+   * Get a listing of all files at 'src'.  The Object[] array
+   * exists so we can return file attributes (soon to be implemented)
+   */
+  public FileStatus[] getListing(String src) throws IOException {
+    if (isPermissionEnabled) {
+      if (dir.isDir(src)) {
+        checkPathAccess(src, FsAction.READ_EXECUTE);
+      }
+      else {
+        checkTraverse(src);
+      }
+    }
+    if (auditLog.isInfoEnabled()) {
+      logAuditEvent(UserGroupInformation.getCurrentUGI(),
+                    Server.getRemoteIp(),
+                    "listStatus", src, null, null);
+    }
+    return dir.getListing(src);
+  }
+
+  /////////////////////////////////////////////////////////
+  //
+  // These methods are called by datanodes
+  //
+  /////////////////////////////////////////////////////////
+  /**
+   * Register Datanode.
+   * <p>
+   * The purpose of registration is to identify whether the new datanode
+   * serves a new data storage, and will report new data block copies,
+   * which the namenode was not aware of; or the datanode is a replacement
+   * node for the data storage that was previously served by a different
+   * or the same (in terms of host:port) datanode.
+   * The data storages are distinguished by their storageIDs. When a new
+   * data storage is reported the namenode issues a new unique storageID.
+   * <p>
+   * Finally, the namenode returns its namespaceID as the registrationID
+   * for the datanodes. 
+   * namespaceID is a persistent attribute of the name space.
+   * The registrationID is checked every time the datanode is communicating
+   * with the namenode. 
+   * Datanodes with inappropriate registrationID are rejected.
+   * If the namenode stops, and then restarts it can restore its 
+   * namespaceID and will continue serving the datanodes that has previously
+   * registered with the namenode without restarting the whole cluster.
+   * 
+   * @see org.apache.hadoop.hdfs.server.datanode.DataNode#register()
+   */
+  public synchronized void registerDatanode(DatanodeRegistration nodeReg
+                                            ) throws IOException {
+    String dnAddress = Server.getRemoteAddress();
+    if (dnAddress == null) {
+      // Mostly called inside an RPC.
+      // But if not, use address passed by the data-node.
+      dnAddress = nodeReg.getHost();
+    }      
+
+    // check if the datanode is allowed to be connect to the namenode
+    if (!verifyNodeRegistration(nodeReg, dnAddress)) {
+      throw new DisallowedDatanodeException(nodeReg);
+    }
+
+    String hostName = nodeReg.getHost();
+      
+    // update the datanode's name with ip:port
+    DatanodeID dnReg = new DatanodeID(dnAddress + ":" + nodeReg.getPort(),
+                                      nodeReg.getStorageID(),
+                                      nodeReg.getInfoPort(),
+                                      nodeReg.getIpcPort());
+    nodeReg.updateRegInfo(dnReg);
+    nodeReg.exportedKeys = getAccessKeys();
+      
+    NameNode.stateChangeLog.info(
+                                 "BLOCK* NameSystem.registerDatanode: "
+                                 + "node registration from " + nodeReg.getName()
+                                 + " storage " + nodeReg.getStorageID());
+
+    DatanodeDescriptor nodeS = datanodeMap.get(nodeReg.getStorageID());
+    DatanodeDescriptor nodeN = host2DataNodeMap.getDatanodeByName(nodeReg.getName());
+      
+    if (nodeN != null && nodeN != nodeS) {
+      NameNode.LOG.info("BLOCK* NameSystem.registerDatanode: "
+                        + "node from name: " + nodeN.getName());
+      // nodeN previously served a different data storage, 
+      // which is not served by anybody anymore.
+      removeDatanode(nodeN);
+      // physically remove node from datanodeMap
+      wipeDatanode(nodeN);
+      nodeN = null;
+    }
+
+    if (nodeS != null) {
+      if (nodeN == nodeS) {
+        // The same datanode has been just restarted to serve the same data 
+        // storage. We do not need to remove old data blocks, the delta will
+        // be calculated on the next block report from the datanode
+        NameNode.stateChangeLog.debug("BLOCK* NameSystem.registerDatanode: "
+                                      + "node restarted.");
+      } else {
+        // nodeS is found
+        /* The registering datanode is a replacement node for the existing 
+          data storage, which from now on will be served by a new node.
+          If this message repeats, both nodes might have same storageID 
+          by (insanely rare) random chance. User needs to restart one of the
+          nodes with its data cleared (or user can just remove the StorageID
+          value in "VERSION" file under the data directory of the datanode,
+          but this is might not work if VERSION file format has changed 
+       */        
+        NameNode.stateChangeLog.info( "BLOCK* NameSystem.registerDatanode: "
+                                      + "node " + nodeS.getName()
+                                      + " is replaced by " + nodeReg.getName() + 
+                                      " with the same storageID " +
+                                      nodeReg.getStorageID());
+      }
+      // update cluster map
+      clusterMap.remove(nodeS);
+      nodeS.updateRegInfo(nodeReg);
+      nodeS.setHostName(hostName);
+      
+      // resolve network location
+      resolveNetworkLocation(nodeS);
+      clusterMap.add(nodeS);
+        
+      // also treat the registration message as a heartbeat
+      synchronized(heartbeats) {
+        if( !heartbeats.contains(nodeS)) {
+          heartbeats.add(nodeS);
+          //update its timestamp
+          nodeS.updateHeartbeat(0L, 0L, 0L, 0);
+          nodeS.isAlive = true;
+        }
+      }
+      return;
+    } 
+
+    // this is a new datanode serving a new data storage
+    if (nodeReg.getStorageID().equals("")) {
+      // this data storage has never been registered
+      // it is either empty or was created by pre-storageID version of DFS
+      nodeReg.storageID = newStorageID();
+      NameNode.stateChangeLog.debug(
+                                    "BLOCK* NameSystem.registerDatanode: "
+                                    + "new storageID " + nodeReg.getStorageID() + " assigned.");
+    }
+    // register new datanode
+    DatanodeDescriptor nodeDescr 
+      = new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK, hostName);
+    resolveNetworkLocation(nodeDescr);
+    unprotectedAddDatanode(nodeDescr);
+    clusterMap.add(nodeDescr);
+      
+    // also treat the registration message as a heartbeat
+    synchronized(heartbeats) {
+      heartbeats.add(nodeDescr);
+      nodeDescr.isAlive = true;
+      // no need to update its timestamp
+      // because its is done when the descriptor is created
+    }
+    return;
+  }
+    
+  /* Resolve a node's network location */
+  private void resolveNetworkLocation (DatanodeDescriptor node) {
+    List<String> names = new ArrayList<String>(1);
+    if (dnsToSwitchMapping instanceof CachedDNSToSwitchMapping) {
+      // get the node's IP address
+      names.add(node.getHost());
+    } else {
+      // get the node's host name
+      String hostName = node.getHostName();
+      int colon = hostName.indexOf(":");
+      hostName = (colon==-1)?hostName:hostName.substring(0,colon);
+      names.add(hostName);
+    }
+    
+    // resolve its network location
+    List<String> rName = dnsToSwitchMapping.resolve(names);
+    String networkLocation;
+    if (rName == null) {
+      LOG.error("The resolve call returned null! Using " + 
+          NetworkTopology.DEFAULT_RACK + " for host " + names);
+      networkLocation = NetworkTopology.DEFAULT_RACK;
+    } else {
+      networkLocation = rName.get(0);
+    }
+    node.setNetworkLocation(networkLocation);
+  }
+  
+  /**
+   * Get registrationID for datanodes based on the namespaceID.
+   * 
+   * @see #registerDatanode(DatanodeRegistration)
+   * @see FSImage#newNamespaceID()
+   * @return registration ID
+   */
+  public String getRegistrationID() {
+    return Storage.getRegistrationID(dir.fsImage);
+  }
+    
+  /**
+   * Generate new storage ID.
+   * 
+   * @return unique storage ID
+   * 
+   * Note: that collisions are still possible if somebody will try 
+   * to bring in a data storage from a different cluster.
+   */
+  private String newStorageID() {
+    String newID = null;
+    while(newID == null) {
+      newID = "DS" + Integer.toString(r.nextInt());
+      if (datanodeMap.get(newID) != null)
+        newID = null;
+    }
+    return newID;
+  }
+    
+  private boolean isDatanodeDead(DatanodeDescriptor node) {
+    return (node.getLastUpdate() <
+            (now() - heartbeatExpireInterval));
+  }
+    
+  private void setDatanodeDead(DatanodeDescriptor node) throws IOException {
+    node.setLastUpdate(0);
+  }
+
+  /**
+   * The given node has reported in.  This method should:
+   * 1) Record the heartbeat, so the datanode isn't timed out
+   * 2) Adjust usage stats for future block allocation
+   * 
+   * If a substantial amount of time passed since the last datanode 
+   * heartbeat then request an immediate block report.  
+   * 
+   * @return an array of datanode commands 
+   * @throws IOException
+   */
+  DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
+      long capacity, long dfsUsed, long remaining,
+      int xceiverCount, int xmitsInProgress) throws IOException {
+    DatanodeCommand cmd = null;
+    synchronized (heartbeats) {
+      synchronized (datanodeMap) {
+        DatanodeDescriptor nodeinfo = null;
+        try {
+          nodeinfo = getDatanode(nodeReg);
+        } catch(UnregisteredNodeException e) {
+          return new DatanodeCommand[]{DatanodeCommand.REGISTER};
+        }
+          
+        // Check if this datanode should actually be shutdown instead. 
+        if (nodeinfo != null && shouldNodeShutdown(nodeinfo)) {
+          setDatanodeDead(nodeinfo);
+          throw new DisallowedDatanodeException(nodeinfo);
+        }
+
+        if (nodeinfo == null || !nodeinfo.isAlive) {
+          return new DatanodeCommand[]{DatanodeCommand.REGISTER};
+        }
+
+        updateStats(nodeinfo, false);
+        nodeinfo.updateHeartbeat(capacity, dfsUsed, remaining, xceiverCount);
+        updateStats(nodeinfo, true);
+        
+        //check lease recovery
+        cmd = nodeinfo.getLeaseRecoveryCommand(Integer.MAX_VALUE);
+        if (cmd != null) {
+          return new DatanodeCommand[] {cmd};
+        }
+      
+        ArrayList<DatanodeCommand> cmds = new ArrayList<DatanodeCommand>(3);
+        //check pending replication
+        cmd = nodeinfo.getReplicationCommand(
+              blockManager.maxReplicationStreams - xmitsInProgress);
+        if (cmd != null) {
+          cmds.add(cmd);
+        }
+        //check block invalidation
+        cmd = nodeinfo.getInvalidateBlocks(blockInvalidateLimit);
+        if (cmd != null) {
+          cmds.add(cmd);
+        }
+        // check access key update
+        if (isAccessTokenEnabled && nodeinfo.needKeyUpdate) {
+          cmds.add(new KeyUpdateCommand(accessTokenHandler.exportKeys()));
+          nodeinfo.needKeyUpdate = false;
+        }
+        if (!cmds.isEmpty()) {
+          return cmds.toArray(new DatanodeCommand[cmds.size()]);
+        }
+      }
+    }
+
+    //check distributed upgrade
+    cmd = getDistributedUpgradeCommand();
+    if (cmd != null) {
+      return new DatanodeCommand[] {cmd};
+    }
+    return null;
+  }
+
+  private void updateStats(DatanodeDescriptor node, boolean isAdded) {
+    //
+    // The statistics are protected by the heartbeat lock
+    //
+    assert(Thread.holdsLock(heartbeats));
+    if (isAdded) {
+      capacityTotal += node.getCapacity();
+      capacityUsed += node.getDfsUsed();
+      capacityRemaining += node.getRemaining();
+      totalLoad += node.getXceiverCount();
+    } else {
+      capacityTotal -= node.getCapacity();
+      capacityUsed -= node.getDfsUsed();
+      capacityRemaining -= node.getRemaining();
+      totalLoad -= node.getXceiverCount();
+    }
+  }
+
+  /**
+   * Update access keys.
+   */
+  void updateAccessKey() throws IOException {
+    this.accessTokenHandler.updateKeys();
+    synchronized (heartbeats) {
+      for (DatanodeDescriptor nodeInfo : heartbeats) {
+        nodeInfo.needKeyUpdate = true;
+      }
+    }
+  }
+
+  /**
+   * Periodically calls heartbeatCheck() and updateAccessKey()
+   */
+  class HeartbeatMonitor implements Runnable {
+    private long lastHeartbeatCheck;
+    private long lastAccessKeyUpdate;
+    /**
+     */
+    public void run() {
+      while (fsRunning) {
+        try {
+          long now = now();
+          if (lastHeartbeatCheck + heartbeatRecheckInterval < now) {
+            heartbeatCheck();
+            lastHeartbeatCheck = now;
+          }
+          if (isAccessTokenEnabled && (lastAccessKeyUpdate + accessKeyUpdateInterval < now)) {
+            updateAccessKey();
+            lastAccessKeyUpdate = now;
+          }
+        } catch (Exception e) {
+          FSNamesystem.LOG.error(StringUtils.stringifyException(e));
+        }
+        try {
+          Thread.sleep(5000);  // 5 seconds
+        } catch (InterruptedException ie) {
+        }
+      }
+    }
+  }
+
+  /**
+   * Periodically calls computeReplicationWork().
+   */
+  class ReplicationMonitor implements Runnable {
+    static final int INVALIDATE_WORK_PCT_PER_ITERATION = 32;
+    static final float REPLICATION_WORK_MULTIPLIER_PER_ITERATION = 2;
+    public void run() {
+      while (fsRunning) {
+        try {
+          computeDatanodeWork();
+          blockManager.processPendingReplications();
+          Thread.sleep(replicationRecheckInterval);
+        } catch (InterruptedException ie) {
+          LOG.warn("ReplicationMonitor thread received InterruptedException." + ie);
+          break;
+        } catch (IOException ie) {
+          LOG.warn("ReplicationMonitor thread received exception. " + ie);
+        } catch (Throwable t) {
+          LOG.warn("ReplicationMonitor thread received Runtime exception. " + t);
+          Runtime.getRuntime().exit(-1);
+        }
+      }
+    }
+  }
+
+  /////////////////////////////////////////////////////////
+  //
+  // These methods are called by the Namenode system, to see
+  // if there is any work for registered datanodes.
+  //
+  /////////////////////////////////////////////////////////
+  /**
+   * Compute block replication and block invalidation work 
+   * that can be scheduled on data-nodes.
+   * The datanode will be informed of this work at the next heartbeat.
+   * 
+   * @return number of blocks scheduled for replication or removal.
+   */
+  public int computeDatanodeWork() throws IOException {
+    int workFound = 0;
+    int blocksToProcess = 0;
+    int nodesToProcess = 0;
+    // blocks should not be replicated or removed if safe mode is on
+    if (isInSafeMode())
+      return workFound;
+    synchronized(heartbeats) {
+      blocksToProcess = (int)(heartbeats.size() 
+          * ReplicationMonitor.REPLICATION_WORK_MULTIPLIER_PER_ITERATION);
+      nodesToProcess = (int)Math.ceil((double)heartbeats.size() 
+          * ReplicationMonitor.INVALIDATE_WORK_PCT_PER_ITERATION / 100);
+    }
+
+    workFound = blockManager.computeReplicationWork(blocksToProcess);
+    
+    // Update FSNamesystemMetrics counters
+    synchronized (this) {
+      blockManager.updateState();
+      blockManager.scheduledReplicationBlocksCount = workFound;
+    }
+    
+    workFound += blockManager.computeInvalidateWork(nodesToProcess);
+    return workFound;
+  }
+
+  public void setNodeReplicationLimit(int limit) {
+    blockManager.maxReplicationStreams = limit;
+  }
+
+  /**
+   * remove a datanode descriptor
+   * @param nodeID datanode ID
+   */
+  synchronized public void removeDatanode(DatanodeID nodeID) 
+    throws IOException {
+    DatanodeDescriptor nodeInfo = getDatanode(nodeID);
+    if (nodeInfo != null) {
+      removeDatanode(nodeInfo);
+    } else {
+      NameNode.stateChangeLog.warn("BLOCK* NameSystem.removeDatanode: "
+                                   + nodeID.getName() + " does not exist");
+    }
+  }
+  
+  /**
+   * remove a datanode descriptor
+   * @param nodeInfo datanode descriptor
+   */
+  private void removeDatanode(DatanodeDescriptor nodeInfo) {
+    synchronized (heartbeats) {
+      if (nodeInfo.isAlive) {
+        updateStats(nodeInfo, false);
+        heartbeats.remove(nodeInfo);
+        nodeInfo.isAlive = false;
+      }
+    }
+
+    for (Iterator<Block> it = nodeInfo.getBlockIterator(); it.hasNext();) {
+      blockManager.removeStoredBlock(it.next(), nodeInfo);
+    }
+    unprotectedRemoveDatanode(nodeInfo);
+    clusterMap.remove(nodeInfo);
+  }
+
+  void unprotectedRemoveDatanode(DatanodeDescriptor nodeDescr) {
+    nodeDescr.resetBlocks();
+    blockManager.removeFromInvalidates(nodeDescr);
+    NameNode.stateChangeLog.debug(
+                                  "BLOCK* NameSystem.unprotectedRemoveDatanode: "
+                                  + nodeDescr.getName() + " is out of service now.");
+  }
+    
+  void unprotectedAddDatanode(DatanodeDescriptor nodeDescr) {
+    /* To keep host2DataNodeMap consistent with datanodeMap,
+       remove  from host2DataNodeMap the datanodeDescriptor removed
+       from datanodeMap before adding nodeDescr to host2DataNodeMap.
+    */
+    host2DataNodeMap.remove(
+                            datanodeMap.put(nodeDescr.getStorageID(), nodeDescr));
+    host2DataNodeMap.add(nodeDescr);
+      
+    NameNode.stateChangeLog.debug(
+                                  "BLOCK* NameSystem.unprotectedAddDatanode: "
+                                  + "node " + nodeDescr.getName() + " is added to datanodeMap.");
+  }
+
+  /**
+   * Physically remove node from datanodeMap.
+   * 
+   * @param nodeID node
+   */
+  void wipeDatanode(DatanodeID nodeID) throws IOException {
+    String key = nodeID.getStorageID();
+    host2DataNodeMap.remove(datanodeMap.remove(key));
+    NameNode.stateChangeLog.debug(
+                                  "BLOCK* NameSystem.wipeDatanode: "
+                                  + nodeID.getName() + " storage " + key 
+                                  + " is removed from datanodeMap.");
+  }
+
+  FSImage getFSImage() {
+    return dir.fsImage;
+  }
+
+  FSEditLog getEditLog() {
+    return getFSImage().getEditLog();
+  }
+
+  /**
+   * Check if there are any expired heartbeats, and if so,
+   * whether any blocks have to be re-replicated.
+   * While removing dead datanodes, make sure that only one datanode is marked
+   * dead at a time within the synchronized section. Otherwise, a cascading
+   * effect causes more datanodes to be declared dead.
+   */
+  void heartbeatCheck() {
+    boolean allAlive = false;
+    while (!allAlive) {
+      boolean foundDead = false;
+      DatanodeID nodeID = null;
+
+      // locate the first dead node.
+      synchronized(heartbeats) {
+        for (Iterator<DatanodeDescriptor> it = heartbeats.iterator();
+             it.hasNext();) {
+          DatanodeDescriptor nodeInfo = it.next();
+          if (isDatanodeDead(nodeInfo)) {
+            foundDead = true;
+            nodeID = nodeInfo;
+            break;
+          }
+        }
+      }
+
+      // acquire the fsnamesystem lock, and then remove the dead node.
+      if (foundDead) {
+        synchronized (this) {
+          synchronized(heartbeats) {
+            synchronized (datanodeMap) {
+              DatanodeDescriptor nodeInfo = null;
+              try {
+                nodeInfo = getDatanode(nodeID);
+              } catch (IOException e) {
+                nodeInfo = null;
+              }
+              if (nodeInfo != null && isDatanodeDead(nodeInfo)) {
+                NameNode.stateChangeLog.info("BLOCK* NameSystem.heartbeatCheck: "
+                                             + "lost heartbeat from " + nodeInfo.getName());
+                removeDatanode(nodeInfo);
+              }
+            }
+          }
+        }
+      }
+      allAlive = !foundDead;
+    }
+  }
+    
+  /**
+   * The given node is reporting all its blocks.  Use this info to 
+   * update the (machine-->blocklist) and (block-->machinelist) tables.
+   */
+  public synchronized void processReport(DatanodeID nodeID, 
+                                         BlockListAsLongs newReport
+                                        ) throws IOException {
+    long startTime = now();
+    if (NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("BLOCK* NameSystem.processReport: "
+                             + "from " + nodeID.getName()+" " + 
+                             newReport.getNumberOfBlocks()+" blocks");
+    }
+    DatanodeDescriptor node = getDatanode(nodeID);
+    if (node == null) {
+      throw new IOException("ProcessReport from unregisterted node: "
+                            + nodeID.getName());
+    }
+
+    // Check if this datanode should actually be shutdown instead.
+    if (shouldNodeShutdown(node)) {
+      setDatanodeDead(node);
+      throw new DisallowedDatanodeException(node);
+    }
+    
+    blockManager.processReport(node, newReport);
+    NameNode.getNameNodeMetrics().blockReport.inc((int) (now() - startTime));
+  }
+
+  /**
+   * We want "replication" replicates for the block, but we now have too many.  
+   * In this method, copy enough nodes from 'srcNodes' into 'dstNodes' such that:
+   *
+   * srcNodes.size() - dstNodes.size() == replication
+   *
+   * We pick node that make sure that replicas are spread across racks and
+   * also try hard to pick one with least free space.
+   * The algorithm is first to pick a node with least free space from nodes
+   * that are on a rack holding more than one replicas of the block.
+   * So removing such a replica won't remove a rack. 
+   * If no such a node is available,
+   * then pick a node with least free space
+   */
+  void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess, 
+                              Block b, short replication,
+                              DatanodeDescriptor addedNode,
+                              DatanodeDescriptor delNodeHint) {
+    // first form a rack to datanodes map and
+    HashMap<String, ArrayList<DatanodeDescriptor>> rackMap =
+      new HashMap<String, ArrayList<DatanodeDescriptor>>();
+    for (Iterator<DatanodeDescriptor> iter = nonExcess.iterator();
+         iter.hasNext();) {
+      DatanodeDescriptor node = iter.next();
+      String rackName = node.getNetworkLocation();
+      ArrayList<DatanodeDescriptor> datanodeList = rackMap.get(rackName);
+      if(datanodeList==null) {
+        datanodeList = new ArrayList<DatanodeDescriptor>();
+      }
+      datanodeList.add(node);
+      rackMap.put(rackName, datanodeList);
+    }
+    
+    // split nodes into two sets
+    // priSet contains nodes on rack with more than one replica
+    // remains contains the remaining nodes
+    ArrayList<DatanodeDescriptor> priSet = new ArrayList<DatanodeDescriptor>();
+    ArrayList<DatanodeDescriptor> remains = new ArrayList<DatanodeDescriptor>();
+    for( Iterator<Entry<String, ArrayList<DatanodeDescriptor>>> iter = 
+      rackMap.entrySet().iterator(); iter.hasNext(); ) {
+      Entry<String, ArrayList<DatanodeDescriptor>> rackEntry = iter.next();
+      ArrayList<DatanodeDescriptor> datanodeList = rackEntry.getValue(); 
+      if( datanodeList.size() == 1 ) {
+        remains.add(datanodeList.get(0));
+      } else {
+        priSet.addAll(datanodeList);
+      }
+    }
+    
+    // pick one node to delete that favors the delete hint
+    // otherwise pick one with least space from priSet if it is not empty
+    // otherwise one node with least space from remains
+    boolean firstOne = true;
+    while (nonExcess.size() - replication > 0) {
+      DatanodeInfo cur = null;
+      long minSpace = Long.MAX_VALUE;
+
+      // check if we can del delNodeHint
+      if (firstOne && delNodeHint !=null && nonExcess.contains(delNodeHint) &&
+            (priSet.contains(delNodeHint) || (addedNode != null && !priSet.contains(addedNode))) ) {
+          cur = delNodeHint;
+      } else { // regular excessive replica removal
+        Iterator<DatanodeDescriptor> iter = 
+          priSet.isEmpty() ? remains.iterator() : priSet.iterator();
+          while( iter.hasNext() ) {
+            DatanodeDescriptor node = iter.next();
+            long free = node.getRemaining();
+
+            if (minSpace > free) {
+              minSpace = free;
+              cur = node;
+            }
+          }
+      }
+
+      firstOne = false;
+      // adjust rackmap, priSet, and remains
+      String rack = cur.getNetworkLocation();
+      ArrayList<DatanodeDescriptor> datanodes = rackMap.get(rack);
+      datanodes.remove(cur);
+      if(datanodes.isEmpty()) {
+        rackMap.remove(rack);
+      }
+      if( priSet.remove(cur) ) {
+        if (datanodes.size() == 1) {
+          priSet.remove(datanodes.get(0));
+          remains.add(datanodes.get(0));
+        }
+      } else {
+        remains.remove(cur);
+      }
+
+      nonExcess.remove(cur);
+      blockManager.addToExcessReplicate(cur, b);
+
+      //
+      // The 'excessblocks' tracks blocks until we get confirmation
+      // that the datanode has deleted them; the only way we remove them
+      // is when we get a "removeBlock" message.  
+      //
+      // The 'invalidate' list is used to inform the datanode the block 
+      // should be deleted.  Items are removed from the invalidate list
+      // upon giving instructions to the namenode.
+      //
+      blockManager.addToInvalidates(b, cur);
+      NameNode.stateChangeLog.info("BLOCK* NameSystem.chooseExcessReplicates: "
+                +"("+cur.getName()+", "+b+") is added to recentInvalidateSets");
+    }
+  }
+
+
+  /**
+   * The given node is reporting that it received a certain block.
+   */
+  public synchronized void blockReceived(DatanodeID nodeID,  
+                                         Block block,
+                                         String delHint
+                                         ) throws IOException {
+    DatanodeDescriptor node = getDatanode(nodeID);
+    if (node == null) {
+      NameNode.stateChangeLog.warn("BLOCK* NameSystem.blockReceived: "
+                                   + block + " is received from an unrecorded node " 
+                                   + nodeID.getName());
+      throw new IllegalArgumentException(
+                                         "Unexpected exception.  Got blockReceived message from node " 
+                                         + block + ", but there is no info for it");
+    }
+        
+    if (NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("BLOCK* NameSystem.blockReceived: "
+                                    +block+" is received from " + nodeID.getName());
+    }
+
+    // Check if this datanode should actually be shutdown instead.
+    if (shouldNodeShutdown(node)) {
+      setDatanodeDead(node);
+      throw new DisallowedDatanodeException(node);
+    }
+
+    blockManager.addBlock(node, block, delHint);
+  }
+
+  public long getMissingBlocksCount() {
+    // not locking
+    return blockManager.getMissingBlocksCount();
+  }
+  
+  long[] getStats() {
+    synchronized(heartbeats) {
+      return new long[] {this.capacityTotal, this.capacityUsed, 
+                         this.capacityRemaining,
+                         getUnderReplicatedBlocks(),
+                         getCorruptReplicaBlocksCount(),
+                         getMissingBlocksCount()};
+    }
+  }
+
+  /**
+   * Total raw bytes including non-dfs used space.
+   */
+  public long getCapacityTotal() {
+    return getStats()[0];
+  }
+
+  /**
+   * Total used space by data nodes
+   */
+  public long getCapacityUsed() {
+    return getStats()[1];
+  }
+  /**
+   * Total used space by data nodes as percentage of total capacity
+   */
+  public float getCapacityUsedPercent() {
+    synchronized(heartbeats){
+      if (capacityTotal <= 0) {
+        return 100;
+      }
+
+      return ((float)capacityUsed * 100.0f)/(float)capacityTotal;
+    }
+  }
+  /**
+   * Total used space by data nodes for non DFS purposes such
+   * as storing temporary files on the local file system
+   */
+  public long getCapacityUsedNonDFS() {
+    long nonDFSUsed = 0;
+    synchronized(heartbeats){
+      nonDFSUsed = capacityTotal - capacityRemaining - capacityUsed;
+    }
+    return nonDFSUsed < 0 ? 0 : nonDFSUsed;
+  }
+  /**
+   * Total non-used raw bytes.
+   */
+  public long getCapacityRemaining() {
+    return getStats()[2];
+  }
+
+  /**
+   * Total remaining space by data nodes as percentage of total capacity
+   */
+  public float getCapacityRemainingPercent() {
+    synchronized(heartbeats){
+      if (capacityTotal <= 0) {
+        return 0;
+      }
+
+      return ((float)capacityRemaining * 100.0f)/(float)capacityTotal;
+    }
+  }
+  /**
+   * Total number of connections.
+   */
+  public int getTotalLoad() {
+    synchronized (heartbeats) {
+      return this.totalLoad;
+    }
+  }
+
+  int getNumberOfDatanodes(DatanodeReportType type) {
+    return getDatanodeListForReport(type).size(); 
+  }
+
+  private synchronized ArrayList<DatanodeDescriptor> getDatanodeListForReport(
+                                                      DatanodeReportType type) {                  
+    
+    boolean listLiveNodes = type == DatanodeReportType.ALL ||
+                            type == DatanodeReportType.LIVE;
+    boolean listDeadNodes = type == DatanodeReportType.ALL ||
+                            type == DatanodeReportType.DEAD;
+
+    HashMap<String, String> mustList = new HashMap<String, String>();
+    
+    if (listDeadNodes) {
+      //first load all the nodes listed in include and exclude files.
+      for (Iterator<String> it = hostsReader.getHosts().iterator(); 
+           it.hasNext();) {
+        mustList.put(it.next(), "");
+      }
+      for (Iterator<String> it = hostsReader.getExcludedHosts().iterator(); 
+           it.hasNext();) {
+        mustList.put(it.next(), "");
+      }
+    }
+   
+    ArrayList<DatanodeDescriptor> nodes = null;
+    
+    synchronized (datanodeMap) {
+      nodes = new ArrayList<DatanodeDescriptor>(datanodeMap.size() + 
+                                                mustList.size());
+      
+      for(Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator(); 
+                                                               it.hasNext();) {
+        DatanodeDescriptor dn = it.next();
+        boolean isDead = isDatanodeDead(dn);
+        if ( (isDead && listDeadNodes) || (!isDead && listLiveNodes) ) {
+          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());
+      }
+    }
+    
+    if (listDeadNodes) {
+      for (Iterator<String> it = mustList.keySet().iterator(); it.hasNext();) {
+        DatanodeDescriptor dn = 
+            new DatanodeDescriptor(new DatanodeID(it.next()));
+        dn.setLastUpdate(0);
+        nodes.add(dn);
+      }
+    }
+    
+    return nodes;
+  }
+
+  public synchronized DatanodeInfo[] datanodeReport( DatanodeReportType type
+      ) throws AccessControlException {
+    checkSuperuserPrivilege();
+
+    ArrayList<DatanodeDescriptor> results = getDatanodeListForReport(type);
+    DatanodeInfo[] arr = new DatanodeInfo[results.size()];
+    for (int i=0; i<arr.length; i++) {
+      arr[i] = new DatanodeInfo(results.get(i));
+    }
+    return arr;
+  }
+
+  /**
+   * Save namespace image.
+   * This will save current namespace into fsimage file and empty edits file.
+   * Requires superuser privilege and safe mode.
+   * 
+   * @throws AccessControlException if superuser privilege is violated.
+   * @throws IOException if 
+   */
+  synchronized void saveNamespace() throws AccessControlException, IOException {
+    checkSuperuserPrivilege();
+    if(!isInSafeMode()) {
+      throw new IOException("Safe mode should be turned ON " +
+                            "in order to create namespace image.");
+    }
+    getFSImage().saveFSImage();
+    LOG.info("New namespace image has been created.");
+  }
+  
+  /**
+   * Enables/Disables/Checks restoring failed storage replicas if the storage becomes available again.
+   * Requires superuser privilege.
+   * 
+   * @throws AccessControlException if superuser privilege is violated.
+   */
+  synchronized boolean restoreFailedStorage(String arg) throws AccessControlException {
+    checkSuperuserPrivilege();
+    
+    // if it is disabled - enable it and vice versa.
+    if(arg.equals("check"))
+      return getFSImage().getRestoreFailedStorage();
+    
+    boolean val = arg.equals("true");  // false if not
+    getFSImage().setRestoreFailedStorage(val);
+    
+    return val;
+  }
+
+  /**
+   */
+  public synchronized void DFSNodesStatus(ArrayList<DatanodeDescriptor> live, 
+                                          ArrayList<DatanodeDescriptor> dead) {
+
+    ArrayList<DatanodeDescriptor> results = 
+                            getDatanodeListForReport(DatanodeReportType.ALL);    
+    for(Iterator<DatanodeDescriptor> it = results.iterator(); it.hasNext();) {
+      DatanodeDescriptor node = it.next();
+      if (isDatanodeDead(node))
+        dead.add(node);
+      else
+        live.add(node);
+    }
+  }
+
+  /**
+   * Prints information about all datanodes.
+   */
+  private synchronized void datanodeDump(PrintWriter out) {
+    synchronized (datanodeMap) {
+      out.println("Metasave: Number of datanodes: " + datanodeMap.size());
+      for(Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator(); it.hasNext();) {
+        DatanodeDescriptor node = it.next();
+        out.println(node.dumpDatanode());
+      }
+    }
+  }
+
+  /**
+   * Start decommissioning the specified datanode. 
+   */
+  private void startDecommission (DatanodeDescriptor node) 
+    throws IOException {
+
+    if (!node.isDecommissionInProgress() && !node.isDecommissioned()) {
+      LOG.info("Start Decommissioning node " + node.getName());
+      node.startDecommission();
+      //
+      // all the blocks that reside on this node have to be 
+      // replicated.
+      Iterator<Block> decommissionBlocks = node.getBlockIterator();
+      while(decommissionBlocks.hasNext()) {
+        Block block = decommissionBlocks.next();
+        blockManager.updateNeededReplications(block, -1, 0);
+      }
+    }
+  }
+
+  /**
+   * Stop decommissioning the specified datanodes.
+   */
+  public void stopDecommission (DatanodeDescriptor node) 
+    throws IOException {
+    LOG.info("Stop Decommissioning node " + node.getName());
+    node.stopDecommission();
+  }
+
+  /** 
+   */
+  public DatanodeInfo getDataNodeInfo(String name) {
+    return datanodeMap.get(name);
+  }
+
+  public Date getStartTime() {
+    return new Date(systemStart); 
+  }
+    
+  short getMaxReplication()     { return (short)blockManager.maxReplication; }
+  short getMinReplication()     { return (short)blockManager.minReplication; }
+  short getDefaultReplication() { return (short)blockManager.defaultReplication; }
+    
+  /**
+   * 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.
+   */
+  boolean checkDecommissionStateInternal(DatanodeDescriptor node) {
+    //
+    // Check to see if all blocks in this decommissioned
+    // node has reached their target replication factor.
+    //
+    if (node.isDecommissionInProgress()) {
+      if (!blockManager.isReplicationInProgress(node)) {
+        node.setDecommissioned();
+        LOG.info("Decommission complete for node " + node.getName());
+      }
+    }
+    if (node.isDecommissioned()) {
+      return true;
+    }
+    return false;
+  }
+
+  /** 
+   * Keeps track of which datanodes/ipaddress are allowed to connect to the namenode.
+   */
+  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())));
+  }
+  
+  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())));
+  }
+
+  /**
+   * Rereads the config to get hosts and exclude list file names.
+   * Rereads the files to update the hosts and exclude lists.  It
+   * checks if any of the hosts have changed states:
+   * 1. Added to hosts  --> no further work needed here.
+   * 2. Removed from hosts --> mark AdminState as decommissioned. 
+   * 3. Added to exclude --> start decommission.
+   * 4. Removed from exclude --> stop decommission.
+   */
+  public void refreshNodes(Configuration conf) throws IOException {
+    checkSuperuserPrivilege();
+    // Reread the config to get dfs.hosts and dfs.hosts.exclude filenames.
+    // Update the file names and refresh internal includes and excludes list
+    if (conf == null)
+      conf = new Configuration();
+    hostsReader.updateFileNames(conf.get("dfs.hosts",""), 
+                                conf.get("dfs.hosts.exclude", ""));
+    hostsReader.refresh();
+    synchronized (this) {
+      for (Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator();
+           it.hasNext();) {
+        DatanodeDescriptor node = it.next();
+        // Check if not include.
+        if (!inHostsList(node, null)) {
+          node.setDecommissioned();  // case 2.
+        } else {
+          if (inExcludedHostsList(node, null)) {
+            if (!node.isDecommissionInProgress() && 
+                !node.isDecommissioned()) {
+              startDecommission(node);   // case 3.
+            }
+          } else {
+            if (node.isDecommissionInProgress() || 
+                node.isDecommissioned()) {
+              stopDecommission(node);   // case 4.
+            } 
+          }
+        }
+      }
+    } 
+      
+  }
+    
+  void finalizeUpgrade() throws IOException {
+    checkSuperuserPrivilege();
+    getFSImage().finalizeUpgrade();
+  }
+
+  /**
+   * Checks if the node is not on the hosts list.  If it is not, then
+   * it will be ignored.  If the node is in the hosts list, but is also 
+   * on the exclude list, then it will be decommissioned.
+   * Returns FALSE if node is rejected for registration. 
+   * Returns TRUE if node is registered (including when it is on the 
+   * exclude list and is being decommissioned). 
+   */
+  private synchronized boolean verifyNodeRegistration(DatanodeID nodeReg, String ipAddr) 
+    throws IOException {
+    if (!inHostsList(nodeReg, ipAddr)) {
+      return false;    
+    }
+    if (inExcludedHostsList(nodeReg, ipAddr)) {
+      DatanodeDescriptor node = getDatanode(nodeReg);
+      if (node == null) {
+        throw new IOException("verifyNodeRegistration: unknown datanode " +
+                              nodeReg.getName());
+      }
+      if (!checkDecommissionStateInternal(node)) {
+        startDecommission(node);
+      }
+    } 
+    return true;
+  }
+    
+  /**
+   * Checks if the Admin state bit is DECOMMISSIONED.  If so, then 
+   * we should shut it down. 
+   * 
+   * Returns true if the node should be shutdown.
+   */
+  private boolean shouldNodeShutdown(DatanodeDescriptor node) {
+    return (node.isDecommissioned());
+  }
+    
+  /**
+   * Get data node by storage ID.
+   * 
+   * @param nodeID
+   * @return DatanodeDescriptor or null if the node is not found.
+   * @throws IOException
+   */
+  public DatanodeDescriptor getDatanode(DatanodeID nodeID) throws IOException {
+    UnregisteredNodeException e = null;
+    DatanodeDescriptor node = datanodeMap.get(nodeID.getStorageID());
+    if (node == null) 
+      return null;
+    if (!node.getName().equals(nodeID.getName())) {
+      e = new UnregisteredNodeException(nodeID, node);
+      NameNode.stateChangeLog.fatal("BLOCK* NameSystem.getDatanode: "
+                                    + e.getLocalizedMessage());
+      throw e;
+    }
+    return node;
+  }
+    
+  /** Stop at and return the datanode at index (used for content browsing)*/
+  @Deprecated
+  private DatanodeDescriptor getDatanodeByIndex(int index) {
+    int i = 0;
+    for (DatanodeDescriptor node : datanodeMap.values()) {
+      if (i == index) {
+        return node;
+      }
+      i++;
+    }
+    return null;
+  }
+    
+  @Deprecated
+  public String randomDataNode() {
+    int size = datanodeMap.size();
+    int index = 0;
+    if (size != 0) {
+      index = r.nextInt(size);
+      for(int i=0; i<size; i++) {
+        DatanodeDescriptor d = getDatanodeByIndex(index);
+        if (d != null && !d.isDecommissioned() && !isDatanodeDead(d) &&
+            !d.isDecommissionInProgress()) {
+          return d.getHost() + ":" + d.getInfoPort();
+        }
+        index = (index + 1) % size;
+      }
+    }
+    return null;
+  }
+
+  /** Choose a random datanode
+   * 
+   * @return a randomly chosen datanode
+   */
+  public DatanodeDescriptor getRandomDatanode() {
+    return (DatanodeDescriptor)clusterMap.chooseRandom(NodeBase.ROOT);
+  }
+
+  /**
+   * SafeModeInfo contains information related to the safe mode.
+   * <p>
+   * An instance of {@link SafeModeInfo} is created when the name node
+   * enters safe mode.
+   * <p>
+   * During name node startup {@link SafeModeInfo} counts the number of
+   * <em>safe blocks</em>, those that have at least the minimal number of
+   * replicas, and calculates the ratio of safe blocks to the total number
+   * of blocks in the system, which is the size of blocks in
+   * {@link FSNamesystem#blockManager}. When the ratio reaches the
+   * {@link #threshold} it starts the {@link SafeModeMonitor} daemon in order
+   * to monitor whether the safe mode {@link #extension} is passed.
+   * Then it leaves safe mode and destroys itself.
+   * <p>
+   * If safe mode is turned on manually then the number of safe blocks is
+   * not tracked because the name node is not intended to leave safe mode
+   * automatically in the case.
+   *
+   * @see ClientProtocol#setSafeMode(FSConstants.SafeModeAction)
+   * @see SafeModeMonitor
+   */
+  class SafeModeInfo {
+    // configuration fields
+    /** Safe mode threshold condition %.*/
+    private double threshold;
+    /** Safe mode extension after the threshold. */
+    private int extension;
+    /** Min replication required by safe mode. */
+    private int safeReplication;
+      
+    // internal fields
+    /** Time when threshold was reached.
+     * 
+     * <br>-1 safe mode is off
+     * <br> 0 safe mode is on, but threshold is not reached yet 
+     */
+    private long reached = -1;  
+    /** Total number of blocks. */
+    int blockTotal; 
+    /** Number of safe blocks. */
+    private int blockSafe;
+    /** Number of blocks needed to satisfy safe mode threshold condition */
+    private int blockThreshold;
+    /** time of the last status printout */
+    private long lastStatusReport = 0;
+      
+    /**
+     * Creates SafeModeInfo when the name node enters
+     * automatic safe mode at startup.
+     *  
+     * @param conf configuration
+     */
+    SafeModeInfo(Configuration conf) {
+      this.threshold = conf.getFloat("dfs.safemode.threshold.pct", 0.95f);
+      this.extension = conf.getInt("dfs.safemode.extension", 0);
+      this.safeReplication = conf.getInt("dfs.replication.min", 1);
+      this.blockTotal = 0; 
+      this.blockSafe = 0;
+    }
+
+    /**
+     * Creates SafeModeInfo when safe mode is entered manually.
+     *
+     * The {@link #threshold} is set to 1.5 so that it could never be reached.
+     * {@link #blockTotal} is set to -1 to indicate that safe mode is manual.
+     * 
+     * @see SafeModeInfo
+     */
+    private SafeModeInfo() {
+      this.threshold = 1.5f;  // this threshold can never be reached
+      this.extension = Integer.MAX_VALUE;
+      this.safeReplication = Short.MAX_VALUE + 1; // more than maxReplication
+      this.blockTotal = -1;
+      this.blockSafe = -1;
+      this.reached = -1;
+      enter();
+      reportStatus("STATE* Safe mode is ON.", true);
+    }
+      
+    /**
+     * Check if safe mode is on.
+     * @return true if in safe mode
+     */
+    synchronized boolean isOn() {
+      try {
+        assert isConsistent() : " SafeMode: Inconsistent filesystem state: "
+          + "Total num of blocks, active blocks, or "
+          + "total safe blocks don't match.";
+      } catch(IOException e) {
+        System.err.print(StringUtils.stringifyException(e));
+      }
+      return this.reached >= 0;
+    }
+      
+    /**
+     * Enter safe mode.
+     */
+    void enter() {
+      this.reached = 0;
+    }
+      
+    /**
+     * Leave safe mode.
+     * <p>
+     * Switch to manual safe mode if distributed upgrade is required.<br>
+     * Check for invalid, under- & over-replicated blocks in the end of startup.
+     */
+    synchronized void leave(boolean checkForUpgrades) {
+      if(checkForUpgrades) {
+        // verify whether a distributed upgrade needs to be started
+        boolean needUpgrade = false;
+        try {
+          needUpgrade = startDistributedUpgradeIfNeeded();
+        } catch(IOException e) {
+          FSNamesystem.LOG.error(StringUtils.stringifyException(e));
+        }
+        if(needUpgrade) {
+          // switch to manual safe mode
+          safeMode = new SafeModeInfo();
+          return;
+        }
+      }
+      // verify blocks replications
+      blockManager.processMisReplicatedBlocks();
+      long timeInSafemode = now() - systemStart;
+      NameNode.stateChangeLog.info("STATE* Leaving safe mode after " 
+                                    + timeInSafemode/1000 + " secs.");
+      NameNode.getNameNodeMetrics().safeModeTime.set((int) timeInSafemode);
+      
+      if (reached >= 0) {
+        NameNode.stateChangeLog.info("STATE* Safe mode is OFF."); 
+      }
+      reached = -1;
+      safeMode = null;
+      NameNode.stateChangeLog.info("STATE* Network topology has "
+                                   +clusterMap.getNumOfRacks()+" racks and "
+                                   +clusterMap.getNumOfLeaves()+ " datanodes");
+      NameNode.stateChangeLog.info("STATE* UnderReplicatedBlocks has "
+                                   +blockManager.neededReplications.size()+" blocks");
+    }
+      
+    /** 
+     * Safe mode can be turned off iff 
+     * the threshold is reached and 
+     * the extension time have passed.
+     * @return true if can leave or false otherwise.
+     */
+    synchronized boolean canLeave() {
+      if (reached == 0)
+        return false;
+      if (now() - reached < extension) {
+        reportStatus("STATE* Safe mode ON.", false);
+        return false;
+      }
+      return !needEnter();
+    }
+      
+    /** 
+     * There is no need to enter safe mode 
+     * if DFS is empty or {@link #threshold} == 0
+     */
+    boolean needEnter() {
+      return threshold != 0 && blockSafe < blockThreshold;
+    }
+      
+    /**
+     * Check and trigger safe mode if needed. 
+     */
+    private void checkMode() {
+      if (needEnter()) {
+        enter();
+        reportStatus("STATE* Safe mode ON.", false);
+        return;
+      }
+      // the threshold is reached
+      if (!isOn() ||                           // safe mode is off
+          extension <= 0 || threshold <= 0) {  // don't need to wait
+        this.leave(true); // leave safe mode
+        return;
+      }
+      if (reached > 0) {  // threshold has already been reached before
+        reportStatus("STATE* Safe mode ON.", false);
+        return;
+      }
+      // start monitor
+      reached = now();
+      smmthread = new Daemon(new SafeModeMonitor());
+      smmthread.start();
+      reportStatus("STATE* Safe mode extension entered.", true);
+    }
+      
+    /**
+     * Set total number of blocks.
+     */
+    synchronized void setBlockTotal(int total) {
+      this.blockTotal = total;
+      this.blockThreshold = (int) (blockTotal * threshold);
+      checkMode();
+    }
+      
+    /**
+     * Increment number of safe blocks if current block has 
+     * reached minimal replication.
+     * @param replication current replication 
+     */
+    synchronized void incrementSafeBlockCount(short replication) {
+      if ((int)replication == safeReplication)
+        this.blockSafe++;
+      checkMode();
+    }
+      
+    /**
+     * Decrement number of safe blocks if current block has 
+     * fallen below minimal replication.
+     * @param replication current replication 
+     */
+    synchronized void decrementSafeBlockCount(short replication) {
+      if (replication == safeReplication-1)
+        this.blockSafe--;
+      checkMode();
+    }
+
+    /**
+     * Check if safe mode was entered manually or at startup.
+     */
+    boolean isManual() {
+      return extension == Integer.MAX_VALUE;
+    }
+
+    /**
+     * Set manual safe mode.
+     */
+    synchronized void setManual() {
+      extension = Integer.MAX_VALUE;
+    }
+
+    /**
+     * A tip on how safe mode is to be turned off: manually or automatically.
+     */
+    String getTurnOffTip() {
+      if(reached < 0)
+        return "Safe mode is OFF.";
+      String leaveMsg = "Safe mode will be turned off automatically";
+      if(isManual()) {
+        if(getDistributedUpgradeState())
+          return leaveMsg + " upon completion of " + 
+            "the distributed upgrade: upgrade progress = " + 
+            getDistributedUpgradeStatus() + "%";
+        leaveMsg = "Use \"hadoop dfs -safemode leave\" to turn safe mode off";
+      }
+      if(blockTotal < 0)
+        return leaveMsg + ".";
+      
+      String msg = null;
+      if (reached == 0) {
+        msg = String.format("The reported blocks %d needs additional %d"
+            + " blocks to reach the threshold %.4f of total blocks %d. %s",
+            blockSafe, (blockThreshold - blockSafe), threshold, blockTotal,
+            leaveMsg);
+      } else {
+        msg = String.format("The reported blocks %d has reached the threshold"
+            + " %.4f of total blocks %d. %s", blockSafe, threshold, 
+            blockTotal, leaveMsg);
+      }
+      if(reached == 0 || isManual()) {  // threshold is not reached or manual       
+        return msg + ".";
+      }
+      // extension period is in progress
+      return msg + " in " + Math.abs(reached + extension - now()) / 1000
+          + " seconds.";
+    }
+
+    /**
+     * Print status every 20 seconds.
+     */
+    private void reportStatus(String msg, boolean rightNow) {
+      long curTime = now();
+      if(!rightNow && (curTime - lastStatusReport < 20 * 1000))
+        return;
+      NameNode.stateChangeLog.info(msg + " \n" + getTurnOffTip());
+      lastStatusReport = curTime;
+    }
+
+    /**
+     * Returns printable state of the class.
+     */
+    public String toString() {
+      String resText = "Current safe blocks = " 
+        + blockSafe 
+        + ". Target blocks = " + blockThreshold + " for threshold = %" + threshold
+        + ". Minimal replication = " + safeReplication + ".";
+      if (reached > 0) 
+        resText += " Threshold was reached " + new Date(reached) + ".";
+      return resText;
+    }
+      
+    /**
+     * Checks consistency of the class state.
+     * This is costly and currently called only in assert.
+     */
+    boolean isConsistent() throws IOException {
+      if (blockTotal == -1 && blockSafe == -1) {
+        return true; // manual safe mode
+      }
+      int activeBlocks = blockManager.getActiveBlockCount();
+      return (blockTotal == activeBlocks) ||
+        (blockSafe >= 0 && blockSafe <= blockTotal);
+    }
+  }
+    
+  /**
+   * Periodically check whether it is time to leave safe mode.
+   * This thread starts when the threshold level is reached.
+   *
+   */
+  class SafeModeMonitor implements Runnable {
+    /** interval in msec for checking safe mode: {@value} */
+    private static final long recheckInterval = 1000;
+      
+    /**
+     */
+    public void run() {
+      while (fsRunning && (safeMode != null && !safeMode.canLeave())) {
+        try {
+          Thread.sleep(recheckInterval);
+        } catch (InterruptedException ie) {
+        }
+      }
+      // leave safe mode and stop the monitor
+      try {
+        leaveSafeMode(true);
+      } catch(SafeModeException es) { // should never happen
+        String msg = "SafeModeMonitor may not run during distributed upgrade.";
+        assert false : msg;
+        throw new RuntimeException(msg, es);
+      }
+      smmthread = null;
+    }
+  }
+    
+  /**
+   * Current system time.
+   * @return current time in msec.
+   */
+  static long now() {
+    return System.currentTimeMillis();
+  }
+    
+  boolean setSafeMode(SafeModeAction action) throws IOException {
+    if (action != SafeModeAction.SAFEMODE_GET) {
+      checkSuperuserPrivilege();
+      switch(action) {
+      case SAFEMODE_LEAVE: // leave safe mode
+        leaveSafeMode(false);
+        break;
+      case SAFEMODE_ENTER: // enter safe mode
+        enterSafeMode();
+        break;
+      }
+    }
+    return isInSafeMode();
+  }
+
+  /**
+   * Check whether the name node is in safe mode.
+   * @return true if safe mode is ON, false otherwise
+   */
+  boolean isInSafeMode() {
+    if (safeMode == null)
+      return false;
+    return safeMode.isOn();
+  }
+    
+  /**
+   * Increment number of blocks that reached minimal replication.
+   * @param replication current replication 
+   */
+  void incrementSafeBlockCount(int replication) {
+    if (safeMode == null)
+      return;
+    safeMode.incrementSafeBlockCount((short)replication);
+  }
+
+  /**
+   * Decrement number of blocks that reached minimal replication.
+   */
+  void decrementSafeBlockCount(Block b) {
+    if (safeMode == null) // mostly true
+      return;
+    safeMode.decrementSafeBlockCount((short)blockManager.countNodes(b).liveReplicas());
+  }
+
+  /**
+   * Set the total number of blocks in the system. 
+   */
+  void setBlockTotal() {
+    if (safeMode == null)
+      return;
+    safeMode.setBlockTotal((int)getBlocksTotal());
+  }
+
+  /**
+   * Get the total number of blocks in the system. 
+   */
+  public long getBlocksTotal() {
+    return blockManager.getTotalBlocks();
+  }
+
+  /**
+   * Enter safe mode manually.
+   * @throws IOException
+   */
+  synchronized void enterSafeMode() throws IOException {
+    if (!isInSafeMode()) {
+      safeMode = new SafeModeInfo();
+      return;
+    }
+    safeMode.setManual();
+    NameNode.stateChangeLog.info("STATE* Safe mode is ON. " 
+                                + safeMode.getTurnOffTip());
+  }
+
+  /**
+   * Leave safe mode.
+   * @throws IOException
+   */
+  synchronized void leaveSafeMode(boolean checkForUpgrades) throws SafeModeException {
+    if (!isInSafeMode()) {
+      NameNode.stateChangeLog.info("STATE* Safe mode is already OFF."); 
+      return;
+    }
+    if(getDistributedUpgradeState())
+      throw new SafeModeException("Distributed upgrade is in progress",
+                                  safeMode);
+    safeMode.leave(checkForUpgrades);
+  }
+    
+  synchronized String getSafeModeTip() {
+    if (!isInSafeMode())
+      return "";
+    return safeMode.getTurnOffTip();
+  }
+
+  long getEditLogSize() throws IOException {
+    return getEditLog().getEditLogSize();
+  }
+
+  synchronized CheckpointSignature rollEditLog() throws IOException {
+    if (isInSafeMode()) {
+      throw new SafeModeException("Checkpoint not created",
+                                  safeMode);
+    }
+    LOG.info("Roll Edit Log from " + Server.getRemoteAddress());
+    return getFSImage().rollEditLog();
+  }
+
+  synchronized void rollFSImage() throws IOException {
+    if (isInSafeMode()) {
+      throw new SafeModeException("Checkpoint not created",
+                                  safeMode);
+    }
+    LOG.info("Roll FSImage from " + Server.getRemoteAddress());
+    getFSImage().rollFSImage();
+  }
+
+  NamenodeCommand startCheckpoint(NamenodeRegistration bnReg, // backup node
+                                  NamenodeRegistration nnReg) // active name-node
+  throws IOException {
+    NamenodeCommand cmd;
+    synchronized(this) {
+      cmd = getFSImage().startCheckpoint(bnReg, nnReg);
+    }
+    LOG.info("Start checkpoint for " + bnReg.getAddress());
+    getEditLog().logSync();
+    return cmd;
+  }
+
+  synchronized void endCheckpoint(NamenodeRegistration registration,
+                            CheckpointSignature sig) throws IOException {
+    LOG.info("End checkpoint for " + registration.getAddress());
+    getFSImage().endCheckpoint(sig, registration.getRole());
+  }
+
+  /**
+   * Returns whether the given block is one pointed-to by a file.
+   */
+  private boolean isValidBlock(Block b) {
+    return (blockManager.getINode(b) != null);
+  }
+
+  // Distributed upgrade manager
+  final UpgradeManagerNamenode upgradeManager = new UpgradeManagerNamenode(this);
+
+  UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action 
+                                                 ) throws IOException {
+    return upgradeManager.distributedUpgradeProgress(action);
+  }
+
+  UpgradeCommand processDistributedUpgradeCommand(UpgradeCommand comm) throws IOException {
+    return upgradeManager.processUpgradeCommand(comm);
+  }
+
+  int getDistributedUpgradeVersion() {
+    return upgradeManager.getUpgradeVersion();
+  }
+
+  UpgradeCommand getDistributedUpgradeCommand() throws IOException {
+    return upgradeManager.getBroadcastCommand();
+  }
+
+  boolean getDistributedUpgradeState() {
+    return upgradeManager.getUpgradeState();
+  }
+
+  short getDistributedUpgradeStatus() {
+    return upgradeManager.getUpgradeStatus();
+  }
+
+  boolean startDistributedUpgradeIfNeeded() throws IOException {
+    return upgradeManager.startUpgrade();
+  }
+
+  PermissionStatus createFsOwnerPermissions(FsPermission permission) {
+    return new PermissionStatus(fsOwner.getUserName(), supergroup, permission);
+  }
+
+  private FSPermissionChecker checkOwner(String path) throws AccessControlException {
+    return checkPermission(path, true, null, null, null, null);
+  }
+
+  private FSPermissionChecker checkPathAccess(String path, FsAction access
+      ) throws AccessControlException {
+    return checkPermission(path, false, null, null, access, null);
+  }
+
+  private FSPermissionChecker checkParentAccess(String path, FsAction access
+      ) throws AccessControlException {
+    return checkPermission(path, false, null, access, null, null);
+  }
+
+  private FSPermissionChecker checkAncestorAccess(String path, FsAction access
+      ) throws AccessControlException {
+    return checkPermission(path, false, access, null, null, null);
+  }
+
+  private FSPermissionChecker checkTraverse(String path
+      ) throws AccessControlException {
+    return checkPermission(path, false, null, null, null, null);
+  }
+
+  private void checkSuperuserPrivilege() throws AccessControlException {
+    if (isPermissionEnabled) {
+      PermissionChecker.checkSuperuserPrivilege(fsOwner, supergroup);
+    }
+  }
+
+  /**
+   * Check whether current user have permissions to access the path.
+   * For more details of the parameters, see
+   * {@link FSPermissionChecker#checkPermission(String, INodeDirectory, boolean, FsAction, FsAction, FsAction, FsAction)}.
+   */
+  private FSPermissionChecker checkPermission(String path, boolean doCheckOwner,
+      FsAction ancestorAccess, FsAction parentAccess, FsAction access,
+      FsAction subAccess) throws AccessControlException {
+    FSPermissionChecker pc = new FSPermissionChecker(
+        fsOwner.getUserName(), supergroup);
+    if (!pc.isSuper) {
+      dir.waitForReady();
+      pc.checkPermission(path, dir.rootDir, doCheckOwner,
+          ancestorAccess, parentAccess, access, subAccess);
+    }
+    return pc;
+  }
+
+  /**
+   * Check to see if we have exceeded the limit on the number
+   * of inodes.
+   */
+  void checkFsObjectLimit() throws IOException {
+    if (maxFsObjects != 0 &&
+        maxFsObjects <= dir.totalInodes() + getBlocksTotal()) {
+      throw new IOException("Exceeded the configured number of objects " +
+                             maxFsObjects + " in the filesystem.");
+    }
+  }
+
+  /**
+   * Get the total number of objects in the system. 
+   */
+  long getMaxObjects() {
+    return maxFsObjects;
+  }
+
+  public long getFilesTotal() {
+    return this.dir.totalInodes();
+  }
+
+  public long getPendingReplicationBlocks() {
+    return blockManager.pendingReplicationBlocksCount;
+  }
+
+  public long getUnderReplicatedBlocks() {
+    return blockManager.underReplicatedBlocksCount;
+  }
+
+  /** Returns number of blocks with corrupt replicas */
+  public long getCorruptReplicaBlocksCount() {
+    return blockManager.corruptReplicaBlocksCount;
+  }
+
+  public long getScheduledReplicationBlocks() {
+    return blockManager.scheduledReplicationBlocksCount;
+  }
+
+  public String getFSState() {
+    return isInSafeMode() ? "safeMode" : "Operational";
+  }
+  
+  private ObjectName mbeanName;
+  /**
+   * Register the FSNamesystem MBean using the name
+   *        "hadoop:service=NameNode,name=FSNamesystemState"
+   */
+  void registerMBean(Configuration conf) {
+    // We wrap to bypass standard mbean naming convention.
+    // This wraping can be removed in java 6 as it is more flexible in 
+    // package naming for mbeans and their impl.
+    StandardMBean bean;
+    try {
+      myFSMetrics = new FSNamesystemMetrics(this, conf);
+      bean = new StandardMBean(this,FSNamesystemMBean.class);
+      mbeanName = MBeanUtil.registerMBean("NameNode", "FSNamesystemState", bean);
+    } catch (NotCompliantMBeanException e) {
+      e.printStackTrace();
+    }
+
+    LOG.info("Registered FSNamesystemStatusMBean");
+  }
+
+  /**
+   * get FSNamesystemMetrics
+   */
+  public FSNamesystemMetrics getFSNamesystemMetrics() {
+    return myFSMetrics;
+  }
+
+  /**
+   * shutdown FSNamesystem
+   */
+  public void shutdown() {
+    if (mbeanName != null)
+      MBeanUtil.unregisterMBean(mbeanName);
+  }
+  
+
+  /**
+   * Number of live data nodes
+   * @return Number of live data nodes
+   */
+  public int getNumLiveDataNodes() {
+    int numLive = 0;
+    synchronized (datanodeMap) {   
+      for(Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator(); 
+                                                               it.hasNext();) {
+        DatanodeDescriptor dn = it.next();
+        if (!isDatanodeDead(dn) ) {
+          numLive++;
+        }
+      }
+    }
+    return numLive;
+  }
+  
+
+  /**
+   * Number of dead data nodes
+   * @return Number of dead data nodes
+   */
+  public int getNumDeadDataNodes() {
+    int numDead = 0;
+    synchronized (datanodeMap) {   
+      for(Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator(); 
+                                                               it.hasNext();) {
+        DatanodeDescriptor dn = it.next();
+        if (isDatanodeDead(dn) ) {
+          numDead++;
+        }
+      }
+    }
+    return numDead;
+  }
+
+  /**
+   * Sets the generation stamp for this filesystem
+   */
+  public void setGenerationStamp(long stamp) {
+    generationStamp.setStamp(stamp);
+  }
+
+  /**
+   * Gets the generation stamp for this filesystem
+   */
+  public long getGenerationStamp() {
+    return generationStamp.getStamp();
+  }
+
+  /**
+   * Increments, logs and then returns the stamp
+   */
+  long nextGenerationStamp() {
+    long gs = generationStamp.nextStamp();
+    getEditLog().logGenerationStamp(gs);
+    return gs;
+  }
+
+  /**
+   * Verifies that the block is associated with a file that has a lease.
+   * Increments, logs and then returns the stamp
+   */
+  synchronized long nextGenerationStampForBlock(Block block) throws IOException {
+    BlockInfo storedBlock = blockManager.getStoredBlock(block);
+    if (storedBlock == null) {
+      String msg = block + " is already commited, storedBlock == null.";
+      LOG.info(msg);
+      throw new IOException(msg);
+    }
+    INodeFile fileINode = storedBlock.getINode();
+    if (!fileINode.isUnderConstruction()) {
+      String msg = block + " is already commited, !fileINode.isUnderConstruction().";
+      LOG.info(msg);
+      throw new IOException(msg);
+    }
+    if (!((INodeFileUnderConstruction)fileINode).setLastRecoveryTime(now())) {
+      String msg = block + " is beening recovered, ignoring this request.";
+      LOG.info(msg);
+      throw new IOException(msg);
+    }
+    return nextGenerationStamp();
+  }
+
+  // rename was successful. If any part of the renamed subtree had
+  // files that were being written to, update with new filename.
+  //
+  void changeLease(String src, String dst, FileStatus dinfo) 
+                   throws IOException {
+    String overwrite;
+    String replaceBy;
+
+    boolean destinationExisted = true;
+    if (dinfo == null) {
+      destinationExisted = false;
+    }
+
+    if (destinationExisted && dinfo.isDir()) {
+      Path spath = new Path(src);
+      overwrite = spath.getParent().toString() + Path.SEPARATOR;
+      replaceBy = dst + Path.SEPARATOR;
+    } else {
+      overwrite = src;
+      replaceBy = dst;
+    }
+
+    leaseManager.changeLease(src, dst, overwrite, replaceBy);
+  }
+           
+  /**
+   * Serializes leases. 
+   */
+  void saveFilesUnderConstruction(DataOutputStream out) throws IOException {
+    synchronized (leaseManager) {
+      out.writeInt(leaseManager.countPath()); // write the size
+
+      for (Lease lease : leaseManager.getSortedLeases()) {
+        for(String path : lease.getPaths()) {
+          // verify that path exists in namespace
+          INode node = dir.getFileINode(path);
+          if (node == null) {
+            throw new IOException("saveLeases found path " + path +
+                                  " but no matching entry in namespace.");
+          }
+          if (!node.isUnderConstruction()) {
+            throw new IOException("saveLeases found path " + path +
+                                  " but is not under construction.");
+          }
+          INodeFileUnderConstruction cons = (INodeFileUnderConstruction) node;
+          FSImage.writeINodeUnderConstruction(out, cons, path);
+        }
+      }
+    }
+  }
+
+  /**
+   * Register a name-node.
+   * <p>
+   * Registration is allowed if there is no ongoing streaming to
+   * another backup node.
+   * We currently allow only one backup node, but multiple chackpointers 
+   * if there are no backups.
+   * 
+   * @param registration
+   * @throws IOException
+   */
+  synchronized void registerBackupNode(NamenodeRegistration registration)
+  throws IOException {
+    if(getFSImage().getNamespaceID() != registration.getNamespaceID())
+      throw new IOException("Incompatible namespaceIDs: " 
+          + " Namenode namespaceID = " + getFSImage().getNamespaceID() 
+          + "; " + registration.getRole() +
+              " node namespaceID = " + registration.getNamespaceID());
+    boolean regAllowed = getEditLog().checkBackupRegistration(registration);
+    if(!regAllowed)
+      throw new IOException("Registration is not allowed. " +
+      		"Another node is registered as a backup.");
+  }
+
+  /**
+   * Release (unregister) backup node.
+   * <p>
+   * Find and remove the backup stream corresponding to the node.
+   * @param registration
+   * @throws IOException
+   */
+  synchronized void releaseBackupNode(NamenodeRegistration registration)
+  throws IOException {
+    if(getFSImage().getNamespaceID() != registration.getNamespaceID())
+      throw new IOException("Incompatible namespaceIDs: " 
+          + " Namenode namespaceID = " + getFSImage().getNamespaceID() 
+          + "; " + registration.getRole() +
+              " node namespaceID = " + registration.getNamespaceID());
+    getEditLog().releaseBackupStream(registration);
+  }
+
+  public int numCorruptReplicas(Block blk) {
+    return blockManager.numCorruptReplicas(blk);
+  }
+
+  /** Get a datanode descriptor given corresponding storageID */
+  DatanodeDescriptor getDatanode(String nodeID) {
+    return datanodeMap.get(nodeID);
+  }
+}

+ 185 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java

@@ -0,0 +1,185 @@
+/**
+ * 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 java.util.Stack;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.PermissionChecker;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/** Perform permission checking in {@link FSNamesystem}. */
+class FSPermissionChecker extends PermissionChecker {
+  static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
+
+  FSPermissionChecker(String fsOwner, String supergroup
+      ) throws AccessControlException{
+    super(fsOwner, supergroup);
+  }
+
+  /**
+   * Check whether current user have permissions to access the path.
+   * Traverse is always checked.
+   *
+   * Parent path means the parent directory for the path.
+   * Ancestor path means the last (the closest) existing ancestor directory
+   * of the path.
+   * Note that if the parent path exists,
+   * then the parent path and the ancestor path are the same.
+   *
+   * For example, suppose the path is "/foo/bar/baz".
+   * No matter baz is a file or a directory,
+   * the parent path is "/foo/bar".
+   * If bar exists, then the ancestor path is also "/foo/bar".
+   * If bar does not exist and foo exists,
+   * then the ancestor path is "/foo".
+   * Further, if both foo and bar do not exist,
+   * then the ancestor path is "/".
+   *
+   * @param doCheckOwner Require user to be the owner of the path?
+   * @param ancestorAccess The access required by the ancestor of the path.
+   * @param parentAccess The access required by the parent of the path.
+   * @param access The access required by the path.
+   * @param subAccess If path is a directory,
+   * it is the access required of the path and all the sub-directories.
+   * If path is not a directory, there is no effect.
+   * @return a PermissionChecker object which caches data for later use.
+   * @throws AccessControlException
+   */
+  void checkPermission(String path, INodeDirectory root, boolean doCheckOwner,
+      FsAction ancestorAccess, FsAction parentAccess, FsAction access,
+      FsAction subAccess) throws AccessControlException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("ACCESS CHECK: " + this
+          + ", doCheckOwner=" + doCheckOwner
+          + ", ancestorAccess=" + ancestorAccess
+          + ", parentAccess=" + parentAccess
+          + ", access=" + access
+          + ", subAccess=" + subAccess);
+    }
+    // check if (parentAccess != null) && file exists, then check sb
+    synchronized(root) {
+      INode[] inodes = root.getExistingPathINodes(path);
+      int ancestorIndex = inodes.length - 2;
+      for(; ancestorIndex >= 0 && inodes[ancestorIndex] == null;
+          ancestorIndex--);
+      checkTraverse(inodes, ancestorIndex);
+
+      if(parentAccess != null && parentAccess.implies(FsAction.WRITE)
+          && inodes[inodes.length - 1] != null)
+        checkStickyBit(inodes[inodes.length - 2], inodes[inodes.length - 1]);
+
+      if (ancestorAccess != null && inodes.length > 1) {
+        check(inodes, ancestorIndex, ancestorAccess);
+      }
+      if (parentAccess != null && inodes.length > 1) {
+        check(inodes, inodes.length - 2, parentAccess);
+      }
+      if (access != null) {
+        check(inodes[inodes.length - 1], access);
+      }
+      if (subAccess != null) {
+        checkSubAccess(inodes[inodes.length - 1], subAccess);
+      }
+      if (doCheckOwner) {
+        checkOwner(inodes[inodes.length - 1]);
+      }
+    }
+  }
+
+  private void checkOwner(INode inode) throws AccessControlException {
+    if (inode != null && user.equals(inode.getUserName())) {
+      return;
+    }
+    throw new AccessControlException("Permission denied");
+  }
+
+  private void checkTraverse(INode[] inodes, int last
+      ) throws AccessControlException {
+    for(int j = 0; j <= last; j++) {
+      check(inodes[j], FsAction.EXECUTE);
+    }
+  }
+
+  private void checkSubAccess(INode inode, FsAction access
+      ) throws AccessControlException {
+    if (inode == null || !inode.isDirectory()) {
+      return;
+    }
+
+    Stack<INodeDirectory> directories = new Stack<INodeDirectory>();
+    for(directories.push((INodeDirectory)inode); !directories.isEmpty(); ) {
+      INodeDirectory d = directories.pop();
+      check(d, access);
+
+      for(INode child : d.getChildren()) {
+        if (child.isDirectory()) {
+          directories.push((INodeDirectory)child);
+        }
+      }
+    }
+  }
+
+  private void check(INode[] inodes, int i, FsAction access
+      ) throws AccessControlException {
+    check(i >= 0? inodes[i]: null, access);
+  }
+
+  private void check(INode inode, FsAction access
+      ) throws AccessControlException {
+    if (inode == null) {
+      return;
+    }
+    FsPermission mode = inode.getFsPermission();
+
+    if (user.equals(inode.getUserName())) { //user class
+      if (mode.getUserAction().implies(access)) { return; }
+    }
+    else if (groups.contains(inode.getGroupName())) { //group class
+      if (mode.getGroupAction().implies(access)) { return; }
+    }
+    else { //other class
+      if (mode.getOtherAction().implies(access)) { return; }
+    }
+    throw new AccessControlException("Permission denied: user=" + user
+        + ", access=" + access + ", inode=" + inode);
+  }
+
+  private void checkStickyBit(INode parent, INode inode) throws AccessControlException {
+    if(!parent.getFsPermission().getStickyBit()) {
+      return;
+    }
+
+    // If this user is the directory owner, return
+    if(parent.getUserName().equals(user)) {
+      return;
+    }
+
+    // if this user is the file owner, return
+    if(inode.getUserName().equals(user)) {
+      return;
+    }
+
+    throw new AccessControlException("Permission denied by sticky bit setting:" +
+      " user=" + user + ", inode=" + inode);
+  }
+}

+ 102 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java

@@ -0,0 +1,102 @@
+/**
+ * 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 java.io.IOException;
+import java.io.PrintWriter;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import javax.net.SocketFactory;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.znerd.xmlenc.XMLOutputter;
+
+/** Servlets for file checksum */
+public class FileChecksumServlets {
+  /** Redirect file checksum queries to an appropriate datanode. */
+  public static class RedirectServlet extends DfsServlet {
+    /** For java.io.Serializable */
+    private static final long serialVersionUID = 1L;
+  
+    /** {@inheritDoc} */
+    public void doGet(HttpServletRequest request, HttpServletResponse response
+        ) throws ServletException, IOException {
+      final UserGroupInformation ugi = getUGI(request);
+      final ServletContext context = getServletContext();
+      final NameNode namenode = (NameNode)context.getAttribute("name.node");
+      final DatanodeID datanode = namenode.getNamesystem().getRandomDatanode();
+      try {
+        final URI uri = createRedirectUri("/getFileChecksum", ugi, datanode, request); 
+        response.sendRedirect(uri.toURL().toString());
+      } catch(URISyntaxException e) {
+        throw new ServletException(e); 
+        //response.getWriter().println(e.toString());
+      } catch (IOException e) {
+        response.sendError(400, e.getMessage());
+      }
+    }
+  }
+  
+  /** Get FileChecksum */
+  public static class GetServlet extends DfsServlet {
+    /** For java.io.Serializable */
+    private static final long serialVersionUID = 1L;
+    
+    /** {@inheritDoc} */
+    public void doGet(HttpServletRequest request, HttpServletResponse response
+        ) throws ServletException, IOException {
+      final UnixUserGroupInformation ugi = getUGI(request);
+      final PrintWriter out = response.getWriter();
+      final String filename = getFilename(request, response);
+      final XMLOutputter xml = new XMLOutputter(out, "UTF-8");
+      xml.declaration();
+
+      final Configuration conf = new Configuration(DataNode.getDataNode().getConf());
+      final int socketTimeout = conf.getInt("dfs.socket.timeout", HdfsConstants.READ_TIMEOUT);
+      final SocketFactory socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
+      UnixUserGroupInformation.saveToConf(conf,
+          UnixUserGroupInformation.UGI_PROPERTY_NAME, ugi);
+      final ClientProtocol nnproxy = DFSClient.createNamenode(conf);
+
+      try {
+        final MD5MD5CRC32FileChecksum checksum = DFSClient.getFileChecksum(
+            filename, nnproxy, socketFactory, socketTimeout);
+        MD5MD5CRC32FileChecksum.write(xml, checksum);
+      } catch(IOException ioe) {
+        new RemoteException(ioe.getClass().getName(), ioe.getMessage()
+            ).writeXml(filename, xml);
+      }
+      xml.endDocument();
+    }
+  }
+}

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

@@ -0,0 +1,107 @@
+/**
+ * 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 java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+
+/** Redirect queries about the hosted filesystem to an appropriate datanode.
+ * @see org.apache.hadoop.hdfs.HftpFileSystem
+ */
+public class FileDataServlet extends DfsServlet {
+  /** For java.io.Serializable */
+  private static final long serialVersionUID = 1L;
+
+  /** Create a redirection URI */
+  protected URI createUri(FileStatus i, UnixUserGroupInformation ugi,
+      ClientProtocol nnproxy, HttpServletRequest request)
+      throws IOException, URISyntaxException {
+    String scheme = request.getScheme();
+    final DatanodeID host = pickSrcDatanode(i, nnproxy);
+    final String hostname;
+    if (host instanceof DatanodeInfo) {
+      hostname = ((DatanodeInfo)host).getHostName();
+    } else {
+      hostname = host.getHost();
+    }
+    return new URI(scheme, null, hostname,
+        "https".equals(scheme)
+          ? (Integer)getServletContext().getAttribute("datanode.https.port")
+          : host.getInfoPort(),
+        "/streamFile", "filename=" + i.getPath() + "&ugi=" + ugi, null);
+  }
+
+  /** Select a datanode to service this request.
+   * Currently, this looks at no more than the first five blocks of a file,
+   * selecting a datanode randomly from the most represented.
+   */
+  private DatanodeID pickSrcDatanode(FileStatus i,
+      ClientProtocol nnproxy) throws IOException {
+    final LocatedBlocks blks = nnproxy.getBlockLocations(
+        i.getPath().toUri().getPath(), 0, 1);
+    if (i.getLen() == 0 || blks.getLocatedBlocks().size() <= 0) {
+      // pick a random datanode
+      NameNode nn = (NameNode)getServletContext().getAttribute("name.node");
+      return nn.getNamesystem().getRandomDatanode();
+    }
+    return JspHelper.bestNode(blks.get(0));
+  }
+
+  /**
+   * Service a GET request as described below.
+   * Request:
+   * {@code
+   * GET http://<nn>:<port>/data[/<path>] HTTP/1.1
+   * }
+   */
+  public void doGet(HttpServletRequest request, HttpServletResponse response)
+    throws IOException {
+    final UnixUserGroupInformation ugi = getUGI(request);
+    final ClientProtocol nnproxy = createNameNodeProxy(ugi);
+
+    try {
+      final String path = request.getPathInfo() != null
+        ? request.getPathInfo() : "/";
+      FileStatus info = nnproxy.getFileInfo(path);
+      if ((info != null) && !info.isDir()) {
+        response.sendRedirect(createUri(info, ugi, nnproxy,
+              request).toURL().toString());
+      } else if (info == null){
+        response.sendError(400, "cat: File not found " + path);
+      } else {
+        response.sendError(400, "cat: " + path + ": is a directory");
+      }
+    } catch (URISyntaxException e) {
+      response.getWriter().println(e.toString());
+    } catch (IOException e) {
+      response.sendError(400, e.getMessage());
+    }
+  }
+
+}
+

+ 63 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java

@@ -0,0 +1,63 @@
+/**
+ * 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 java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Map;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * This class is used in Namesystem's web server to do fsck on namenode.
+ */
+public class FsckServlet extends DfsServlet {
+  /** for java.io.Serializable */
+  private static final long serialVersionUID = 1L;
+
+  /** Handle fsck request */
+  public void doGet(HttpServletRequest request, HttpServletResponse response
+      ) throws IOException {
+    @SuppressWarnings("unchecked")
+    final Map<String,String[]> pmap = request.getParameterMap();
+    final PrintWriter out = response.getWriter();
+
+    final UnixUserGroupInformation ugi = getUGI(request);
+    UserGroupInformation.setCurrentUser(ugi);
+
+    final ServletContext context = getServletContext();
+    final Configuration conf = new Configuration((Configuration) context.getAttribute("name.conf"));
+    UnixUserGroupInformation.saveToConf(conf,
+        UnixUserGroupInformation.UGI_PROPERTY_NAME, ugi);
+
+    final NameNode nn = (NameNode) context.getAttribute("name.node");
+    final FSNamesystem namesystem = nn.getNamesystem();
+    final int totalDatanodes = namesystem.getNumberOfDatanodes(DatanodeReportType.LIVE); 
+    final short minReplication = namesystem.getMinReplication();
+
+    new NamenodeFsck(conf, nn, nn.getNetworkTopology(), pmap, out,
+        totalDatanodes, minReplication).fsck();
+  }
+}

+ 70 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java

@@ -0,0 +1,70 @@
+/**
+ * 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 java.util.*;
+import java.io.*;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * This class is used in Namesystem's jetty to retrieve a file.
+ * Typically used by the Secondary NameNode to retrieve image and
+ * edit file for periodic checkpointing.
+ */
+public class GetImageServlet extends HttpServlet {
+  private static final long serialVersionUID = -7669068179452648952L;
+
+  @SuppressWarnings("unchecked")
+  public void doGet(HttpServletRequest request,
+                    HttpServletResponse response
+                    ) throws ServletException, IOException {
+    Map<String,String[]> pmap = request.getParameterMap();
+    try {
+      ServletContext context = getServletContext();
+      FSImage nnImage = (FSImage)context.getAttribute("name.system.image");
+      TransferFsImage ff = new TransferFsImage(pmap, request, response);
+      if (ff.getImage()) {
+        // send fsImage
+        TransferFsImage.getFileServer(response.getOutputStream(),
+                                      nnImage.getFsImageName()); 
+      } else if (ff.getEdit()) {
+        // send edits
+        TransferFsImage.getFileServer(response.getOutputStream(),
+                                      nnImage.getFsEditName());
+      } else if (ff.putImage()) {
+        // issue a HTTP get request to download the new fsimage 
+        nnImage.validateCheckpointUpload(ff.getToken());
+        TransferFsImage.getFileClient(ff.getInfoServer(), "getimage=1", 
+                                      nnImage.getFsImageNameCheckpoint());
+        nnImage.checkpointUploadDone();
+      }
+    } catch (Exception ie) {
+      String errMsg = "GetImage failed. " + StringUtils.stringifyException(ie);
+      response.sendError(HttpServletResponse.SC_GONE, errMsg);
+      throw new IOException(errMsg);
+    } finally {
+      response.getOutputStream().close();
+    }
+  }
+}

+ 188 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/Host2NodesMap.java

@@ -0,0 +1,188 @@
+/**
+ * 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 java.util.*;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+class Host2NodesMap {
+  private HashMap<String, DatanodeDescriptor[]> map
+    = new HashMap<String, DatanodeDescriptor[]>();
+  private Random r = new Random();
+  private ReadWriteLock hostmapLock = new ReentrantReadWriteLock();
+                      
+  /** Check if node is already in the map. */
+  boolean contains(DatanodeDescriptor node) {
+    if (node==null) {
+      return false;
+    }
+      
+    String host = node.getHost();
+    hostmapLock.readLock().lock();
+    try {
+      DatanodeDescriptor[] nodes = map.get(host);
+      if (nodes != null) {
+        for(DatanodeDescriptor containedNode:nodes) {
+          if (node==containedNode) {
+            return true;
+          }
+        }
+      }
+    } finally {
+      hostmapLock.readLock().unlock();
+    }
+    return false;
+  }
+    
+  /** add node to the map 
+   * return true if the node is added; false otherwise.
+   */
+  boolean add(DatanodeDescriptor node) {
+    hostmapLock.writeLock().lock();
+    try {
+      if (node==null || contains(node)) {
+        return false;
+      }
+      
+      String host = node.getHost();
+      DatanodeDescriptor[] nodes = map.get(host);
+      DatanodeDescriptor[] newNodes;
+      if (nodes==null) {
+        newNodes = new DatanodeDescriptor[1];
+        newNodes[0]=node;
+      } else { // rare case: more than one datanode on the host
+        newNodes = new DatanodeDescriptor[nodes.length+1];
+        System.arraycopy(nodes, 0, newNodes, 0, nodes.length);
+        newNodes[nodes.length] = node;
+      }
+      map.put(host, newNodes);
+      return true;
+    } finally {
+      hostmapLock.writeLock().unlock();
+    }
+  }
+    
+  /** remove node from the map 
+   * return true if the node is removed; false otherwise.
+   */
+  boolean remove(DatanodeDescriptor node) {
+    if (node==null) {
+      return false;
+    }
+      
+    String host = node.getHost();
+    hostmapLock.writeLock().lock();
+    try {
+
+      DatanodeDescriptor[] nodes = map.get(host);
+      if (nodes==null) {
+        return false;
+      }
+      if (nodes.length==1) {
+        if (nodes[0]==node) {
+          map.remove(host);
+          return true;
+        } else {
+          return false;
+        }
+      }
+      //rare case
+      int i=0;
+      for(; i<nodes.length; i++) {
+        if (nodes[i]==node) {
+          break;
+        }
+      }
+      if (i==nodes.length) {
+        return false;
+      } else {
+        DatanodeDescriptor[] newNodes;
+        newNodes = new DatanodeDescriptor[nodes.length-1];
+        System.arraycopy(nodes, 0, newNodes, 0, i);
+        System.arraycopy(nodes, i+1, newNodes, i, nodes.length-i-1);
+        map.put(host, newNodes);
+        return true;
+      }
+    } finally {
+      hostmapLock.writeLock().unlock();
+    }
+  }
+    
+  /** get a data node by its host.
+   * @return DatanodeDescriptor if found; otherwise null.
+   */
+  DatanodeDescriptor getDatanodeByHost(String host) {
+    if (host==null) {
+      return null;
+    }
+      
+    hostmapLock.readLock().lock();
+    try {
+      DatanodeDescriptor[] nodes = map.get(host);
+      // no entry
+      if (nodes== null) {
+        return null;
+      }
+      // one node
+      if (nodes.length == 1) {
+        return nodes[0];
+      }
+      // more than one node
+      return nodes[r.nextInt(nodes.length)];
+    } finally {
+      hostmapLock.readLock().unlock();
+    }
+  }
+    
+  /**
+   * Find data node by its name.
+   * 
+   * @return DatanodeDescriptor if found or null otherwise 
+   */
+  public DatanodeDescriptor getDatanodeByName(String name) {
+    if (name==null) {
+      return null;
+    }
+      
+    int colon = name.indexOf(":");
+    String host;
+    if (colon < 0) {
+      host = name;
+    } else {
+      host = name.substring(0, colon);
+    }
+
+    hostmapLock.readLock().lock();
+    try {
+      DatanodeDescriptor[] nodes = map.get(host);
+      // no entry
+      if (nodes== null) {
+        return null;
+      }
+      for(DatanodeDescriptor containedNode:nodes) {
+        if (name.equals(containedNode.getName())) {
+          return containedNode;
+        }
+      }
+      return null;
+    } finally {
+      hostmapLock.readLock().unlock();
+    }
+  }
+}

+ 426 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -0,0 +1,426 @@
+/**
+ * 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 java.io.UnsupportedEncodingException;
+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.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+
+/**
+ * We keep an in-memory representation of the file/block hierarchy.
+ * This is a base INode class containing common fields for file and 
+ * directory inodes.
+ */
+abstract class INode implements Comparable<byte[]> {
+  protected byte[] name;
+  protected INodeDirectory parent;
+  protected long modificationTime;
+  protected long accessTime;
+
+  /** Simple wrapper for two counters : 
+   *  nsCount (namespace consumed) and dsCount (diskspace consumed).
+   */
+  static class DirCounts {
+    long nsCount = 0;
+    long dsCount = 0;
+    
+    /** returns namespace count */
+    long getNsCount() {
+      return nsCount;
+    }
+    /** returns diskspace count */
+    long getDsCount() {
+      return dsCount;
+    }
+  }
+  
+  //Only updated by updatePermissionStatus(...).
+  //Other codes should not modify it.
+  private long permission;
+
+  private static enum PermissionStatusFormat {
+    MODE(0, 16),
+    GROUP(MODE.OFFSET + MODE.LENGTH, 25),
+    USER(GROUP.OFFSET + GROUP.LENGTH, 23);
+
+    final int OFFSET;
+    final int LENGTH; //bit length
+    final long MASK;
+
+    PermissionStatusFormat(int offset, int length) {
+      OFFSET = offset;
+      LENGTH = length;
+      MASK = ((-1L) >>> (64 - LENGTH)) << OFFSET;
+    }
+
+    long retrieve(long record) {
+      return (record & MASK) >>> OFFSET;
+    }
+
+    long combine(long bits, long record) {
+      return (record & ~MASK) | (bits << OFFSET);
+    }
+  }
+
+  protected INode() {
+    name = null;
+    parent = null;
+    modificationTime = 0;
+    accessTime = 0;
+  }
+
+  INode(PermissionStatus permissions, long mTime, long atime) {
+    this.name = null;
+    this.parent = null;
+    this.modificationTime = mTime;
+    setAccessTime(atime);
+    setPermissionStatus(permissions);
+  }
+
+  protected INode(String name, PermissionStatus permissions) {
+    this(permissions, 0L, 0L);
+    setLocalName(name);
+  }
+  
+  /** copy constructor
+   * 
+   * @param other Other node to be copied
+   */
+  INode(INode other) {
+    setLocalName(other.getLocalName());
+    this.parent = other.getParent();
+    setPermissionStatus(other.getPermissionStatus());
+    setModificationTime(other.getModificationTime());
+    setAccessTime(other.getAccessTime());
+  }
+
+  /**
+   * Check whether this is the root inode.
+   */
+  boolean isRoot() {
+    return name.length == 0;
+  }
+
+  /** Set the {@link PermissionStatus} */
+  protected void setPermissionStatus(PermissionStatus ps) {
+    setUser(ps.getUserName());
+    setGroup(ps.getGroupName());
+    setPermission(ps.getPermission());
+  }
+  /** Get the {@link PermissionStatus} */
+  protected PermissionStatus getPermissionStatus() {
+    return new PermissionStatus(getUserName(),getGroupName(),getFsPermission());
+  }
+  private synchronized void updatePermissionStatus(
+      PermissionStatusFormat f, long n) {
+    permission = f.combine(n, permission);
+  }
+  /** Get user name */
+  public String getUserName() {
+    int n = (int)PermissionStatusFormat.USER.retrieve(permission);
+    return SerialNumberManager.INSTANCE.getUser(n);
+  }
+  /** Set user */
+  protected void setUser(String user) {
+    int n = SerialNumberManager.INSTANCE.getUserSerialNumber(user);
+    updatePermissionStatus(PermissionStatusFormat.USER, n);
+  }
+  /** Get group name */
+  public String getGroupName() {
+    int n = (int)PermissionStatusFormat.GROUP.retrieve(permission);
+    return SerialNumberManager.INSTANCE.getGroup(n);
+  }
+  /** Set group */
+  protected void setGroup(String group) {
+    int n = SerialNumberManager.INSTANCE.getGroupSerialNumber(group);
+    updatePermissionStatus(PermissionStatusFormat.GROUP, n);
+  }
+  /** Get the {@link FsPermission} */
+  public FsPermission getFsPermission() {
+    return new FsPermission(
+        (short)PermissionStatusFormat.MODE.retrieve(permission));
+  }
+  protected short getFsPermissionShort() {
+    return (short)PermissionStatusFormat.MODE.retrieve(permission);
+  }
+  /** Set the {@link FsPermission} of this {@link INode} */
+  protected void setPermission(FsPermission permission) {
+    updatePermissionStatus(PermissionStatusFormat.MODE, permission.toShort());
+  }
+
+  /**
+   * Check whether it's a directory
+   */
+  public abstract boolean isDirectory();
+  /**
+   * Collect all the blocks in all children of this INode.
+   * Count and return the number of files in the sub tree.
+   * Also clears references since this INode is deleted.
+   */
+  abstract int collectSubtreeBlocksAndClear(List<Block> v);
+
+  /** Compute {@link ContentSummary}. */
+  public final ContentSummary computeContentSummary() {
+    long[] a = computeContentSummary(new long[]{0,0,0,0});
+    return new ContentSummary(a[0], a[1], a[2], getNsQuota(), 
+                              a[3], getDsQuota());
+  }
+  /**
+   * @return an array of three longs. 
+   * 0: length, 1: file count, 2: directory count 3: disk space
+   */
+  abstract long[] computeContentSummary(long[] summary);
+  
+  /**
+   * Get the quota set for this inode
+   * @return the quota if it is set; -1 otherwise
+   */
+  long getNsQuota() {
+    return -1;
+  }
+
+  long getDsQuota() {
+    return -1;
+  }
+  
+  boolean isQuotaSet() {
+    return getNsQuota() >= 0 || getDsQuota() >= 0;
+  }
+  
+  /**
+   * Adds total nubmer of names and total disk space taken under 
+   * this tree to counts.
+   * Returns updated counts object.
+   */
+  abstract DirCounts spaceConsumedInTree(DirCounts counts);
+  
+  /**
+   * Get local file name
+   * @return local file name
+   */
+  String getLocalName() {
+    return bytes2String(name);
+  }
+
+  /**
+   * Get local file name
+   * @return local file name
+   */
+  byte[] getLocalNameBytes() {
+    return name;
+  }
+
+  /**
+   * Set local file name
+   */
+  void setLocalName(String name) {
+    this.name = string2Bytes(name);
+  }
+
+  /**
+   * Set local file name
+   */
+  void setLocalName(byte[] name) {
+    this.name = name;
+  }
+
+  /** {@inheritDoc} */
+  public String toString() {
+    return "\"" + getLocalName() + "\":" + getPermissionStatus();
+  }
+
+  /**
+   * Get parent directory 
+   * @return parent INode
+   */
+  INodeDirectory getParent() {
+    return this.parent;
+  }
+
+  /** 
+   * Get last modification time of inode.
+   * @return access time
+   */
+  public long getModificationTime() {
+    return this.modificationTime;
+  }
+
+  /**
+   * Set last modification time of inode.
+   */
+  void setModificationTime(long modtime) {
+    assert isDirectory();
+    if (this.modificationTime <= modtime) {
+      this.modificationTime = modtime;
+    }
+  }
+
+  /**
+   * Always set the last modification time of inode.
+   */
+  void setModificationTimeForce(long modtime) {
+    assert !isDirectory();
+    this.modificationTime = modtime;
+  }
+
+  /**
+   * Get access time of inode.
+   * @return access time
+   */
+  public long getAccessTime() {
+    return accessTime;
+  }
+
+  /**
+   * Set last access time of inode.
+   */
+  void setAccessTime(long atime) {
+    accessTime = atime;
+  }
+
+  /**
+   * Is this inode being constructed?
+   */
+  boolean isUnderConstruction() {
+    return false;
+  }
+
+  /**
+   * Breaks file path into components.
+   * @param path
+   * @return array of byte arrays each of which represents 
+   * a single path component.
+   */
+  static byte[][] getPathComponents(String path) {
+    return getPathComponents(getPathNames(path));
+  }
+
+  /** Convert strings to byte arrays for path components. */
+  static byte[][] getPathComponents(String[] strings) {
+    if (strings.length == 0) {
+      return new byte[][]{null};
+    }
+    byte[][] bytes = new byte[strings.length][];
+    for (int i = 0; i < strings.length; i++)
+      bytes[i] = string2Bytes(strings[i]);
+    return bytes;
+  }
+
+  /**
+   * Breaks file path into names.
+   * @param path
+   * @return array of names 
+   */
+  static String[] getPathNames(String path) {
+    if (path == null || !path.startsWith(Path.SEPARATOR)) {
+      return null;
+    }
+    return path.split(Path.SEPARATOR);
+  }
+
+  boolean removeNode() {
+    if (parent == null) {
+      return false;
+    } else {
+      
+      parent.removeChild(this);
+      parent = null;
+      return true;
+    }
+  }
+
+  //
+  // Comparable interface
+  //
+  public int compareTo(byte[] o) {
+    return compareBytes(name, o);
+  }
+
+  public boolean equals(Object o) {
+    if (!(o instanceof INode)) {
+      return false;
+    }
+    return Arrays.equals(this.name, ((INode)o).name);
+  }
+
+  public int hashCode() {
+    return Arrays.hashCode(this.name);
+  }
+
+  //
+  // static methods
+  //
+  /**
+   * Compare two byte arrays.
+   * 
+   * @return a negative integer, zero, or a positive integer 
+   * as defined by {@link #compareTo(byte[])}.
+   */
+  static int compareBytes(byte[] a1, byte[] a2) {
+    if (a1==a2)
+        return 0;
+    int len1 = (a1==null ? 0 : a1.length);
+    int len2 = (a2==null ? 0 : a2.length);
+    int n = Math.min(len1, len2);
+    byte b1, b2;
+    for (int i=0; i<n; i++) {
+      b1 = a1[i];
+      b2 = a2[i];
+      if (b1 != b2)
+        return b1 - b2;
+    }
+    return len1 - len2;
+  }
+
+  /**
+   * Converts a byte array to a string using UTF8 encoding.
+   */
+  static String bytes2String(byte[] bytes) {
+    try {
+      return new String(bytes, "UTF8");
+    } catch(UnsupportedEncodingException e) {
+      assert false : "UTF8 encoding is not supported ";
+    }
+    return null;
+  }
+
+  /**
+   * Converts a string to a byte array using UTF8 encoding.
+   */
+  static byte[] string2Bytes(String str) {
+    try {
+      return str.getBytes("UTF8");
+    } catch(UnsupportedEncodingException e) {
+      assert false : "UTF8 encoding is not supported ";
+    }
+    return null;
+  }
+  
+  
+  LocatedBlocks createLocatedBlocks(List<LocatedBlock> blocks) {
+    return new LocatedBlocks(computeContentSummary().getLength(), blocks,
+        isUnderConstruction());
+  }
+}

+ 348 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -0,0 +1,348 @@
+/**
+ * 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 java.io.FileNotFoundException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+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;
+
+/**
+ * Directory INode class.
+ */
+class INodeDirectory extends INode {
+  protected static final int DEFAULT_FILES_PER_DIRECTORY = 5;
+  final static String ROOT_NAME = "";
+
+  private List<INode> children;
+
+  INodeDirectory(String name, PermissionStatus permissions) {
+    super(name, permissions);
+    this.children = null;
+  }
+
+  public INodeDirectory(PermissionStatus permissions, long mTime) {
+    super(permissions, mTime, 0);
+    this.children = null;
+  }
+
+  /** constructor */
+  INodeDirectory(byte[] localName, PermissionStatus permissions, long mTime) {
+    this(permissions, mTime);
+    this.name = localName;
+  }
+  
+  /** copy constructor
+   * 
+   * @param other
+   */
+  INodeDirectory(INodeDirectory other) {
+    super(other);
+    this.children = other.getChildren();
+  }
+  
+  /**
+   * Check whether it's a directory
+   */
+  public boolean isDirectory() {
+    return true;
+  }
+
+  INode removeChild(INode node) {
+    assert children != null;
+    int low = Collections.binarySearch(children, node.name);
+    if (low >= 0) {
+      return children.remove(low);
+    } else {
+      return null;
+    }
+  }
+
+  /** Replace a child that has the same name as newChild by newChild.
+   * 
+   * @param newChild Child node to be added
+   */
+  void replaceChild(INode newChild) {
+    if ( children == null ) {
+      throw new IllegalArgumentException("The directory is empty");
+    }
+    int low = Collections.binarySearch(children, newChild.name);
+    if (low>=0) { // an old child exists so replace by the newChild
+      children.set(low, newChild);
+    } else {
+      throw new IllegalArgumentException("No child exists to be replaced");
+    }
+  }
+  
+  INode getChild(String name) {
+    return getChildINode(string2Bytes(name));
+  }
+
+  private INode getChildINode(byte[] name) {
+    if (children == null) {
+      return null;
+    }
+    int low = Collections.binarySearch(children, name);
+    if (low >= 0) {
+      return children.get(low);
+    }
+    return null;
+  }
+
+  /**
+   */
+  private INode getNode(byte[][] components) {
+    INode[] inode  = new INode[1];
+    getExistingPathINodes(components, inode);
+    return inode[0];
+  }
+
+  /**
+   * This is the external interface
+   */
+  INode getNode(String path) {
+    return getNode(getPathComponents(path));
+  }
+
+  /**
+   * Retrieve existing INodes from a path. If existing is big enough to store
+   * all path components (existing and non-existing), then existing INodes
+   * will be stored starting from the root INode into existing[0]; if
+   * existing is not big enough to store all path components, then only the
+   * last existing and non existing INodes will be stored so that
+   * existing[existing.length-1] refers to the target INode.
+   * 
+   * <p>
+   * Example: <br>
+   * Given the path /c1/c2/c3 where only /c1/c2 exists, resulting in the
+   * following path components: ["","c1","c2","c3"],
+   * 
+   * <p>
+   * <code>getExistingPathINodes(["","c1","c2"], [?])</code> should fill the
+   * array with [c2] <br>
+   * <code>getExistingPathINodes(["","c1","c2","c3"], [?])</code> should fill the
+   * array with [null]
+   * 
+   * <p>
+   * <code>getExistingPathINodes(["","c1","c2"], [?,?])</code> should fill the
+   * array with [c1,c2] <br>
+   * <code>getExistingPathINodes(["","c1","c2","c3"], [?,?])</code> should fill
+   * the array with [c2,null]
+   * 
+   * <p>
+   * <code>getExistingPathINodes(["","c1","c2"], [?,?,?,?])</code> should fill
+   * the array with [rootINode,c1,c2,null], <br>
+   * <code>getExistingPathINodes(["","c1","c2","c3"], [?,?,?,?])</code> should
+   * fill the array with [rootINode,c1,c2,null]
+   * @param components array of path component name
+   * @param existing INode array to fill with existing INodes
+   * @return number of existing INodes in the path
+   */
+  int getExistingPathINodes(byte[][] components, INode[] existing) {
+    assert compareBytes(this.name, components[0]) == 0 :
+      "Incorrect name " + getLocalName() + " expected " + components[0];
+
+    INode curNode = this;
+    int count = 0;
+    int index = existing.length - components.length;
+    if (index > 0)
+      index = 0;
+    while ((count < components.length) && (curNode != null)) {
+      if (index >= 0)
+        existing[index] = curNode;
+      if (!curNode.isDirectory() || (count == components.length - 1))
+        break; // no more child, stop here
+      INodeDirectory parentDir = (INodeDirectory)curNode;
+      curNode = parentDir.getChildINode(components[count + 1]);
+      count += 1;
+      index += 1;
+    }
+    return count;
+  }
+
+  /**
+   * Retrieve the existing INodes along the given path. The first INode
+   * always exist and is this INode.
+   * 
+   * @param path the path to explore
+   * @return INodes array containing the existing INodes in the order they
+   *         appear when following the path from the root INode to the
+   *         deepest INodes. The array size will be the number of expected
+   *         components in the path, and non existing components will be
+   *         filled with null
+   *         
+   * @see #getExistingPathINodes(byte[][], INode[])
+   */
+  INode[] getExistingPathINodes(String path) {
+    byte[][] components = getPathComponents(path);
+    INode[] inodes = new INode[components.length];
+
+    this.getExistingPathINodes(components, inodes);
+    
+    return inodes;
+  }
+
+  /**
+   * Add a child inode to the directory.
+   * 
+   * @param node INode to insert
+   * @param inheritPermission inherit permission from parent?
+   * @return  null if the child with this name already exists; 
+   *          node, otherwise
+   */
+  <T extends INode> T addChild(final T node, boolean inheritPermission) {
+    if (inheritPermission) {
+      FsPermission p = getFsPermission();
+      //make sure the  permission has wx for the user
+      if (!p.getUserAction().implies(FsAction.WRITE_EXECUTE)) {
+        p = new FsPermission(p.getUserAction().or(FsAction.WRITE_EXECUTE),
+            p.getGroupAction(), p.getOtherAction());
+      }
+      node.setPermission(p);
+    }
+
+    if (children == null) {
+      children = new ArrayList<INode>(DEFAULT_FILES_PER_DIRECTORY);
+    }
+    int low = Collections.binarySearch(children, node.name);
+    if(low >= 0)
+      return null;
+    node.parent = this;
+    children.add(-low - 1, node);
+    // update modification time of the parent directory
+    setModificationTime(node.getModificationTime());
+    if (node.getGroupName() == null) {
+      node.setGroup(getGroupName());
+    }
+    return node;
+  }
+
+  /**
+   * Equivalent to addNode(path, newNode, false).
+   * @see #addNode(String, INode, boolean)
+   */
+  <T extends INode> T addNode(String path, T newNode) throws FileNotFoundException {
+    return addNode(path, newNode, false);
+  }
+  /**
+   * Add new INode to the file tree.
+   * Find the parent and insert 
+   * 
+   * @param path file path
+   * @param newNode INode to be added
+   * @param inheritPermission If true, copy the parent's permission to newNode.
+   * @return null if the node already exists; inserted INode, otherwise
+   * @throws FileNotFoundException if parent does not exist or 
+   * is not a directory.
+   */
+  <T extends INode> T addNode(String path, T newNode, boolean inheritPermission
+      ) throws FileNotFoundException {
+    if(addToParent(path, newNode, null, inheritPermission) == null)
+      return null;
+    return newNode;
+  }
+
+  /**
+   * Add new inode to the parent if specified.
+   * Optimized version of addNode() if parent is not null.
+   * 
+   * @return  parent INode if new inode is inserted
+   *          or null if it already exists.
+   * @throws  FileNotFoundException if parent does not exist or 
+   *          is not a directory.
+   */
+  <T extends INode> INodeDirectory addToParent(
+                                      String path,
+                                      T newNode,
+                                      INodeDirectory parent,
+                                      boolean inheritPermission
+                                    ) throws FileNotFoundException {
+    byte[][] pathComponents = getPathComponents(path);
+    assert pathComponents != null : "Incorrect path " + path;
+    int pathLen = pathComponents.length;
+    if (pathLen < 2)  // add root
+      return null;
+    if(parent == null) {
+      // Gets the parent INode
+      INode[] inodes  = new INode[2];
+      getExistingPathINodes(pathComponents, inodes);
+      INode inode = inodes[0];
+      if (inode == null) {
+        throw new FileNotFoundException("Parent path does not exist: "+path);
+      }
+      if (!inode.isDirectory()) {
+        throw new FileNotFoundException("Parent path is not a directory: "+path);
+      }
+      parent = (INodeDirectory)inode;
+    }
+    // insert into the parent children list
+    newNode.name = pathComponents[pathLen-1];
+    if(parent.addChild(newNode, inheritPermission) == null)
+      return null;
+    return parent;
+  }
+
+  /** {@inheritDoc} */
+  DirCounts spaceConsumedInTree(DirCounts counts) {
+    counts.nsCount += 1;
+    if (children != null) {
+      for (INode child : children) {
+        child.spaceConsumedInTree(counts);
+      }
+    }
+    return counts;    
+  }
+
+  /** {@inheritDoc} */
+  long[] computeContentSummary(long[] summary) {
+    if (children != null) {
+      for (INode child : children) {
+        child.computeContentSummary(summary);
+      }
+    }
+    summary[2]++;
+    return summary;
+  }
+
+  /**
+   */
+  List<INode> getChildren() {
+    return children==null ? new ArrayList<INode>() : children;
+  }
+  List<INode> getChildrenRaw() {
+    return children;
+  }
+
+  int collectSubtreeBlocksAndClear(List<Block> v) {
+    int total = 1;
+    if (children == null) {
+      return total;
+    }
+    for (INode child : children) {
+      total += child.collectSubtreeBlocksAndClear(v);
+    }
+    parent = null;
+    children = null;
+    return total;
+  }
+}

+ 164 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java

@@ -0,0 +1,164 @@
+/**
+ * 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 org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+
+/**
+ * Directory INode class that has a quota restriction
+ */
+class INodeDirectoryWithQuota extends INodeDirectory {
+  private long nsQuota; /// NameSpace quota
+  private long nsCount;
+  private long dsQuota; /// disk space quota
+  private long diskspace;
+  
+  /** Convert an existing directory inode to one with the given quota
+   * 
+   * @param nsQuota Namespace quota to be assigned to this inode
+   * @param dsQuota Diskspace quota to be assigned to this indoe
+   * @param other The other inode from which all other properties are copied
+   */
+  INodeDirectoryWithQuota(long nsQuota, long dsQuota, INodeDirectory other)
+  throws QuotaExceededException {
+    super(other);
+    INode.DirCounts counts = new INode.DirCounts();
+    other.spaceConsumedInTree(counts);
+    this.nsCount= counts.getNsCount();
+    this.diskspace = counts.getDsCount();
+    setQuota(nsQuota, dsQuota);
+  }
+  
+  /** constructor with no quota verification */
+  INodeDirectoryWithQuota(
+      PermissionStatus permissions, long modificationTime, 
+      long nsQuota, long dsQuota)
+  {
+    super(permissions, modificationTime);
+    this.nsQuota = nsQuota;
+    this.dsQuota = dsQuota;
+    this.nsCount = 1;
+  }
+  
+  /** constructor with no quota verification */
+  INodeDirectoryWithQuota(String name, PermissionStatus permissions, 
+                          long nsQuota, long dsQuota)
+  {
+    super(name, permissions);
+    this.nsQuota = nsQuota;
+    this.dsQuota = dsQuota;
+    this.nsCount = 1;
+  }
+  
+  /** Get this directory's namespace quota
+   * @return this directory's namespace quota
+   */
+  long getNsQuota() {
+    return nsQuota;
+  }
+  
+  /** Get this directory's diskspace quota
+   * @return this directory's diskspace quota
+   */
+  long getDsQuota() {
+    return dsQuota;
+  }
+  
+  /** Set this directory's quota
+   * 
+   * @param nsQuota Namespace quota to be set
+   * @param dsQuota diskspace quota to be set
+   * @throws QuotaExceededException if quota is modified and the modified quota
+   *         is too low.
+   *                                
+   */
+  void setQuota(long newNsQuota, long newDsQuota) throws QuotaExceededException {
+    // if a quota is not chaged, ignore that in verification.
+    if ((newNsQuota >=0 && newNsQuota != nsQuota && newNsQuota < nsCount)  ||
+        (newDsQuota >=0 && newDsQuota != dsQuota && newDsQuota < diskspace)) {
+      throw new QuotaExceededException(newNsQuota, nsCount, 
+                                       newDsQuota, diskspace);
+    }
+
+    nsQuota = newNsQuota;
+    dsQuota = newDsQuota;
+  }
+  
+  
+  @Override
+  DirCounts spaceConsumedInTree(DirCounts counts) {
+    counts.nsCount += nsCount;
+    counts.dsCount += diskspace;
+    return counts;
+  }
+
+  /** Get the number of names in the subtree rooted at this directory
+   * @return the size of the subtree rooted at this directory
+   */
+  long numItemsInTree() {
+    return nsCount;
+  }
+  
+  long diskspaceConsumed() {
+    return diskspace;
+  }
+  
+  /** Update the size of the tree
+   * 
+   * @param nsDelta the change of the tree size
+   * @param dsDelta change to disk space occupied
+   * @throws QuotaExceededException if the changed size is greater 
+   *                                than the quota
+   */
+  void updateNumItemsInTree(long nsDelta, long dsDelta) throws 
+                            QuotaExceededException {
+    long newCount = nsCount + nsDelta;
+    long newDiskspace = diskspace + dsDelta;
+    if (nsDelta>0 || dsDelta>0) {
+      verifyQuota(nsQuota, newCount, dsQuota, newDiskspace);
+    }
+    nsCount = newCount;
+    diskspace = newDiskspace;
+  }
+  
+  /** 
+   * Sets namespace and diskspace take by the directory rooted 
+   * at this INode. This should be used carefully. It does not check 
+   * for quota violations.
+   * 
+   * @param namespace size of the directory to be set
+   * @param diskspace disk space take by all the nodes under this directory
+   */
+  void setSpaceConsumed(long namespace, long diskspace) {
+    this.nsCount = namespace;
+    this.diskspace = diskspace;
+  }
+  
+  /** Verify if the namespace count disk space satisfies the quota restriction 
+   * @throws QuotaExceededException if the given quota is less than the count
+   */
+  private static void verifyQuota(long nsQuota, long nsCount, 
+                                  long dsQuota, long diskspace)
+                                  throws QuotaExceededException {
+    if ((nsQuota >= 0 && nsQuota < nsCount) || 
+        (dsQuota >= 0 && dsQuota < diskspace)) {
+      throw new QuotaExceededException(nsQuota, nsCount, dsQuota, diskspace);
+    }
+  }
+}

+ 194 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -0,0 +1,194 @@
+/**
+ * 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 java.io.IOException;
+import java.util.List;
+
+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.namenode.BlocksMap.BlockInfo;
+
+class INodeFile extends INode {
+  static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
+
+  protected BlockInfo blocks[] = null;
+  protected short blockReplication;
+  protected long preferredBlockSize;
+
+  INodeFile(PermissionStatus permissions,
+            int nrBlocks, short replication, long modificationTime,
+            long atime, long preferredBlockSize) {
+    this(permissions, new BlockInfo[nrBlocks], replication,
+        modificationTime, atime, preferredBlockSize);
+  }
+
+  protected INodeFile() {
+    blocks = null;
+    blockReplication = 0;
+    preferredBlockSize = 0;
+  }
+
+  protected INodeFile(PermissionStatus permissions, BlockInfo[] blklist,
+                      short replication, long modificationTime,
+                      long atime, long preferredBlockSize) {
+    super(permissions, modificationTime, atime);
+    this.blockReplication = replication;
+    this.preferredBlockSize = preferredBlockSize;
+    blocks = blklist;
+  }
+
+  /**
+   * Set the {@link FsPermission} of this {@link INodeFile}.
+   * Since this is a file,
+   * the {@link FsAction#EXECUTE} action, if any, is ignored.
+   */
+  protected void setPermission(FsPermission permission) {
+    super.setPermission(permission.applyUMask(UMASK));
+  }
+
+  public boolean isDirectory() {
+    return false;
+  }
+
+  /**
+   * Get block replication for the file 
+   * @return block replication
+   */
+  public short getReplication() {
+    return this.blockReplication;
+  }
+
+  void setReplication(short replication) {
+    this.blockReplication = replication;
+  }
+
+  /**
+   * Get file blocks 
+   * @return file blocks
+   */
+  BlockInfo[] getBlocks() {
+    return this.blocks;
+  }
+
+  /**
+   * add a block to the block list
+   */
+  void addBlock(BlockInfo newblock) {
+    if (this.blocks == null) {
+      this.blocks = new BlockInfo[1];
+      this.blocks[0] = newblock;
+    } else {
+      int size = this.blocks.length;
+      BlockInfo[] newlist = new BlockInfo[size + 1];
+      System.arraycopy(this.blocks, 0, newlist, 0, size);
+      newlist[size] = newblock;
+      this.blocks = newlist;
+    }
+  }
+
+  /**
+   * Set file block
+   */
+  void setBlock(int idx, BlockInfo blk) {
+    this.blocks[idx] = blk;
+  }
+
+  int collectSubtreeBlocksAndClear(List<Block> v) {
+    parent = null;
+    for (Block blk : blocks) {
+      v.add(blk);
+    }
+    blocks = null;
+    return 1;
+  }
+
+  /** {@inheritDoc} */
+  long[] computeContentSummary(long[] summary) {
+    long bytes = 0;
+    for(Block blk : blocks) {
+      bytes += blk.getNumBytes();
+    }
+    summary[0] += bytes;
+    summary[1]++;
+    summary[3] += diskspaceConsumed();
+    return summary;
+  }
+
+  
+
+  @Override
+  DirCounts spaceConsumedInTree(DirCounts counts) {
+    counts.nsCount += 1;
+    counts.dsCount += diskspaceConsumed();
+    return counts;
+  }
+
+  long diskspaceConsumed() {
+    return diskspaceConsumed(blocks);
+  }
+  
+  long diskspaceConsumed(Block[] blkArr) {
+    long size = 0;
+    for (Block blk : blkArr) {
+      if (blk != null) {
+        size += blk.getNumBytes();
+      }
+    }
+    /* If the last block is being written to, use prefferedBlockSize
+     * rather than the actual block size.
+     */
+    if (blkArr.length > 0 && blkArr[blkArr.length-1] != null && 
+        isUnderConstruction()) {
+      size += preferredBlockSize - blocks[blocks.length-1].getNumBytes();
+    }
+    return size * blockReplication;
+  }
+  
+  /**
+   * Get the preferred block size of the file.
+   * @return the number of bytes
+   */
+  public long getPreferredBlockSize() {
+    return preferredBlockSize;
+  }
+
+  /**
+   * Return the penultimate allocated block for this file.
+   */
+  Block getPenultimateBlock() {
+    if (blocks == null || blocks.length <= 1) {
+      return null;
+    }
+    return blocks[blocks.length - 2];
+  }
+
+  INodeFileUnderConstruction toINodeFileUnderConstruction(
+      String clientName, String clientMachine, DatanodeDescriptor clientNode
+      ) throws IOException {
+    if (isUnderConstruction()) {
+      return (INodeFileUnderConstruction)this;
+    }
+    return new INodeFileUnderConstruction(name,
+        blockReplication, modificationTime, preferredBlockSize,
+        blocks, getPermissionStatus(),
+        clientName, clientMachine, clientNode);
+  }
+}

+ 181 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java

@@ -0,0 +1,181 @@
+/**
+ * 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 java.io.IOException;
+
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
+
+
+class INodeFileUnderConstruction extends INodeFile {
+  final String clientName;         // lease holder
+  private final String clientMachine;
+  private final DatanodeDescriptor clientNode; // if client is a cluster node too.
+
+  private int primaryNodeIndex = -1; //the node working on lease recovery
+  private DatanodeDescriptor[] targets = null;   //locations for last block
+  private long lastRecoveryTime = 0;
+  
+  INodeFileUnderConstruction(PermissionStatus permissions,
+                             short replication,
+                             long preferredBlockSize,
+                             long modTime,
+                             String clientName,
+                             String clientMachine,
+                             DatanodeDescriptor clientNode) {
+    super(permissions.applyUMask(UMASK), 0, replication, modTime, modTime,
+        preferredBlockSize);
+    this.clientName = clientName;
+    this.clientMachine = clientMachine;
+    this.clientNode = clientNode;
+  }
+
+  public INodeFileUnderConstruction(byte[] name,
+                             short blockReplication,
+                             long modificationTime,
+                             long preferredBlockSize,
+                             BlockInfo[] blocks,
+                             PermissionStatus perm,
+                             String clientName,
+                             String clientMachine,
+                             DatanodeDescriptor clientNode) {
+    super(perm, blocks, blockReplication, modificationTime, modificationTime,
+          preferredBlockSize);
+    setLocalName(name);
+    this.clientName = clientName;
+    this.clientMachine = clientMachine;
+    this.clientNode = clientNode;
+  }
+
+  String getClientName() {
+    return clientName;
+  }
+
+  String getClientMachine() {
+    return clientMachine;
+  }
+
+  DatanodeDescriptor getClientNode() {
+    return clientNode;
+  }
+
+  /**
+   * Is this inode being constructed?
+   */
+  @Override
+  boolean isUnderConstruction() {
+    return true;
+  }
+
+  DatanodeDescriptor[] getTargets() {
+    return targets;
+  }
+
+  void setTargets(DatanodeDescriptor[] targets) {
+    this.targets = targets;
+    this.primaryNodeIndex = -1;
+  }
+
+  //
+  // converts a INodeFileUnderConstruction into a INodeFile
+  // use the modification time as the access time
+  //
+  INodeFile convertToInodeFile() {
+    INodeFile obj = new INodeFile(getPermissionStatus(),
+                                  getBlocks(),
+                                  getReplication(),
+                                  getModificationTime(),
+                                  getModificationTime(),
+                                  getPreferredBlockSize());
+    return obj;
+    
+  }
+
+  /**
+   * remove a block from the block list. This block should be
+   * the last one on the list.
+   */
+  void removeBlock(Block oldblock) throws IOException {
+    if (blocks == null) {
+      throw new IOException("Trying to delete non-existant block " + oldblock);
+    }
+    int size_1 = blocks.length - 1;
+    if (!blocks[size_1].equals(oldblock)) {
+      throw new IOException("Trying to delete non-last block " + oldblock);
+    }
+
+    //copy to a new list
+    BlockInfo[] newlist = new BlockInfo[size_1];
+    System.arraycopy(blocks, 0, newlist, 0, size_1);
+    blocks = newlist;
+    
+    // Remove the block locations for the last block.
+    targets = null;
+  }
+
+  synchronized void setLastBlock(BlockInfo newblock, DatanodeDescriptor[] newtargets
+      ) throws IOException {
+    if (blocks == null) {
+      throw new IOException("Trying to update non-existant block (newblock="
+          + newblock + ")");
+    }
+    blocks[blocks.length - 1] = newblock;
+    setTargets(newtargets);
+    lastRecoveryTime = 0;
+  }
+
+  /**
+   * Initialize lease recovery for this object
+   */
+  void assignPrimaryDatanode() {
+    //assign the first alive datanode as the primary datanode
+
+    if (targets.length == 0) {
+      NameNode.stateChangeLog.warn("BLOCK*"
+        + " INodeFileUnderConstruction.initLeaseRecovery:"
+        + " No blocks found, lease removed.");
+    }
+
+    int previous = primaryNodeIndex;
+    //find an alive datanode beginning from previous
+    for(int i = 1; i <= targets.length; i++) {
+      int j = (previous + i)%targets.length;
+      if (targets[j].isAlive) {
+        DatanodeDescriptor primary = targets[primaryNodeIndex = j]; 
+        primary.addBlockToBeRecovered(blocks[blocks.length - 1], targets);
+        NameNode.stateChangeLog.info("BLOCK* " + blocks[blocks.length - 1]
+          + " recovery started, primary=" + primary);
+        return;
+      }
+    }
+  }
+  
+  /**
+   * Update lastRecoveryTime if expired.
+   * @return true if lastRecoveryTimeis updated. 
+   */
+  synchronized boolean setLastRecoveryTime(long now) {
+    boolean expired = now - lastRecoveryTime > NameNode.LEASE_RECOVER_PERIOD;
+    if (expired) {
+      lastRecoveryTime = now;
+    }
+    return expired;
+  }
+}

+ 56 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/JournalStream.java

@@ -0,0 +1,56 @@
+/**
+ * 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;
+
+/**
+ * A generic interface for journal input and output streams.
+ */
+interface JournalStream {
+  /**
+   * Type of the underlying persistent storage type the stream is based upon.
+   * <ul>
+   * <li>{@link JournalType#FILE} - streams edits into a local file, see
+   * {@link FSEditLog.EditLogFileOutputStream} and 
+   * {@link FSEditLog.EditLogFileInputStream}</li>
+   * <li>{@link JournalType#BACKUP} - streams edits to a backup node, see
+   * {@link EditLogBackupOutputStream} and {@link EditLogBackupInputStream}</li>
+   * </ul>
+   */
+  static enum JournalType {
+    FILE,
+    BACKUP;
+    boolean isOfType(JournalType other) {
+      return other == null || this == other;
+    }
+  };
+
+  /**
+   * Get this stream name.
+   * 
+   * @return name of the stream
+   */
+  String getName();
+
+  /**
+   * Get the type of the stream.
+   * Determines the underlying persistent storage type.
+   * @see JournalType
+   * @return type
+   */
+  JournalType getType();
+}

+ 429 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/JspHelper.java

@@ -0,0 +1,429 @@
+/**
+ * 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 java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.net.Socket;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Random;
+import java.util.TreeSet;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.jsp.JspWriter;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessToken;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.VersionInfo;
+
+public class JspHelper {
+  final static public String WEB_UGI_PROPERTY_NAME = "dfs.web.ugi";
+
+  public static final Configuration conf = new Configuration();
+  public static final UnixUserGroupInformation webUGI
+  = UnixUserGroupInformation.createImmutable(
+      conf.getStrings(WEB_UGI_PROPERTY_NAME));
+
+  private static final int defaultChunkSizeToView = 
+    conf.getInt("dfs.default.chunk.view.size", 32 * 1024);
+  static final Random rand = new Random();
+
+  static {
+    UnixUserGroupInformation.saveToConf(conf,
+        UnixUserGroupInformation.UGI_PROPERTY_NAME, webUGI);
+  }
+
+  /** Private constructor for preventing creating JspHelper object. */
+  private JspHelper() {} 
+
+  public static DatanodeInfo bestNode(LocatedBlock blk) throws IOException {
+    TreeSet<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>();
+    DatanodeInfo chosenNode = null;
+    int failures = 0;
+    Socket s = null;
+    DatanodeInfo [] nodes = blk.getLocations();
+    if (nodes == null || nodes.length == 0) {
+      throw new IOException("No nodes contain this block");
+    }
+    while (s == null) {
+      if (chosenNode == null) {
+        do {
+          chosenNode = nodes[rand.nextInt(nodes.length)];
+        } while (deadNodes.contains(chosenNode));
+      }
+      int index = rand.nextInt(nodes.length);
+      chosenNode = nodes[index];
+
+      //just ping to check whether the node is alive
+      InetSocketAddress targetAddr = NetUtils.createSocketAddr(
+          chosenNode.getHost() + ":" + chosenNode.getInfoPort());
+        
+      try {
+        s = new Socket();
+        s.connect(targetAddr, HdfsConstants.READ_TIMEOUT);
+        s.setSoTimeout(HdfsConstants.READ_TIMEOUT);
+      } catch (IOException e) {
+        deadNodes.add(chosenNode);
+        s.close();
+        s = null;
+        failures++;
+      }
+      if (failures == nodes.length)
+        throw new IOException("Could not reach the block containing the data. Please try again");
+        
+    }
+    s.close();
+    return chosenNode;
+  }
+
+  public static void streamBlockInAscii(InetSocketAddress addr, long blockId, 
+                                 AccessToken accessToken, long genStamp, long blockSize, 
+                                 long offsetIntoBlock, long chunkSizeToView, JspWriter out) 
+    throws IOException {
+    if (chunkSizeToView == 0) return;
+    Socket s = new Socket();
+    s.connect(addr, HdfsConstants.READ_TIMEOUT);
+    s.setSoTimeout(HdfsConstants.READ_TIMEOUT);
+      
+      long amtToRead = Math.min(chunkSizeToView, blockSize - offsetIntoBlock);     
+      
+      // Use the block name for file name. 
+      DFSClient.BlockReader blockReader = 
+        DFSClient.BlockReader.newBlockReader(s, addr.toString() + ":" + blockId,
+                                             blockId, accessToken, genStamp ,offsetIntoBlock, 
+                                             amtToRead, 
+                                             conf.getInt("io.file.buffer.size",
+                                                         4096));
+        
+    byte[] buf = new byte[(int)amtToRead];
+    int readOffset = 0;
+    int retries = 2;
+    while ( amtToRead > 0 ) {
+      int numRead;
+      try {
+        numRead = blockReader.readAll(buf, readOffset, (int)amtToRead);
+      }
+      catch (IOException e) {
+        retries--;
+        if (retries == 0)
+          throw new IOException("Could not read data from datanode");
+        continue;
+      }
+      amtToRead -= numRead;
+      readOffset += numRead;
+    }
+    blockReader = null;
+    s.close();
+    out.print(new String(buf));
+  }
+
+  public static void addTableHeader(JspWriter out) throws IOException {
+    out.print("<table border=\"1\""+
+              " cellpadding=\"2\" cellspacing=\"2\">");
+    out.print("<tbody>");
+  }
+  public static void addTableRow(JspWriter out, String[] columns) throws IOException {
+    out.print("<tr>");
+    for (int i = 0; i < columns.length; i++) {
+      out.print("<td style=\"vertical-align: top;\"><B>"+columns[i]+"</B><br></td>");
+    }
+    out.print("</tr>");
+  }
+  public static void addTableRow(JspWriter out, String[] columns, int row) throws IOException {
+    out.print("<tr>");
+      
+    for (int i = 0; i < columns.length; i++) {
+      if (row/2*2 == row) {//even
+        out.print("<td style=\"vertical-align: top;background-color:LightGrey;\"><B>"+columns[i]+"</B><br></td>");
+      } else {
+        out.print("<td style=\"vertical-align: top;background-color:LightBlue;\"><B>"+columns[i]+"</B><br></td>");
+          
+      }
+    }
+    out.print("</tr>");
+  }
+  public static void addTableFooter(JspWriter out) throws IOException {
+    out.print("</tbody></table>");
+  }
+
+  public static String getSafeModeText(FSNamesystem fsn) {
+    if (!fsn.isInSafeMode())
+      return "";
+    return "Safe mode is ON. <em>" + fsn.getSafeModeTip() + "</em><br>";
+  }
+
+  public static String getWarningText(FSNamesystem fsn) {
+    // Ideally this should be displayed in RED
+    long missingBlocks = fsn.getMissingBlocksCount();
+    if (missingBlocks > 0) {
+      return "<br> WARNING :" + 
+             " There are about " + missingBlocks +
+             " missing blocks. Please check the log or run fsck. <br><br>";
+    }
+    return "";
+  }
+  
+  public static String getInodeLimitText(FSNamesystem fsn) {
+    long inodes = fsn.dir.totalInodes();
+    long blocks = fsn.getBlocksTotal();
+    long maxobjects = fsn.getMaxObjects();
+    long totalMemory = Runtime.getRuntime().totalMemory();   
+    long maxMemory = Runtime.getRuntime().maxMemory();   
+
+    long used = (totalMemory * 100)/maxMemory;
+ 
+    String str = inodes + " files and directories, " +
+                 blocks + " blocks = " +
+                 (inodes + blocks) + " total";
+    if (maxobjects != 0) {
+      long pct = ((inodes + blocks) * 100)/maxobjects;
+      str += " / " + maxobjects + " (" + pct + "%)";
+    }
+    str += ".  Heap Size is " + StringUtils.byteDesc(totalMemory) + " / " + 
+           StringUtils.byteDesc(maxMemory) + 
+           " (" + used + "%) <br>";
+    return str;
+  }
+
+  public static String getUpgradeStatusText(FSNamesystem fsn) {
+    String statusText = "";
+    try {
+      UpgradeStatusReport status = 
+        fsn.distributedUpgradeProgress(UpgradeAction.GET_STATUS);
+      statusText = (status == null ? 
+          "There are no upgrades in progress." :
+            status.getStatusText(false));
+    } catch(IOException e) {
+      statusText = "Upgrade status unknown.";
+    }
+    return statusText;
+  }
+
+  public static void sortNodeList(ArrayList<DatanodeDescriptor> nodes,
+                           String field, String order) {
+        
+    class NodeComapare implements Comparator<DatanodeDescriptor> {
+      static final int 
+        FIELD_NAME              = 1,
+        FIELD_LAST_CONTACT      = 2,
+        FIELD_BLOCKS            = 3,
+        FIELD_CAPACITY          = 4,
+        FIELD_USED              = 5,
+        FIELD_PERCENT_USED      = 6,
+        FIELD_NONDFS_USED       = 7,
+        FIELD_REMAINING         = 8,
+        FIELD_PERCENT_REMAINING = 9,
+        SORT_ORDER_ASC          = 1,
+        SORT_ORDER_DSC          = 2;
+
+      int sortField = FIELD_NAME;
+      int sortOrder = SORT_ORDER_ASC;
+            
+      public NodeComapare(String field, String order) {
+        if (field.equals("lastcontact")) {
+          sortField = FIELD_LAST_CONTACT;
+        } else if (field.equals("capacity")) {
+          sortField = FIELD_CAPACITY;
+        } else if (field.equals("used")) {
+          sortField = FIELD_USED;
+        } else if (field.equals("nondfsused")) {
+          sortField = FIELD_NONDFS_USED;
+        } else if (field.equals("remaining")) {
+          sortField = FIELD_REMAINING;
+        } else if (field.equals("pcused")) {
+          sortField = FIELD_PERCENT_USED;
+        } else if (field.equals("pcremaining")) {
+          sortField = FIELD_PERCENT_REMAINING;
+        } else if (field.equals("blocks")) {
+          sortField = FIELD_BLOCKS;
+        } else {
+          sortField = FIELD_NAME;
+        }
+                
+        if (order.equals("DSC")) {
+          sortOrder = SORT_ORDER_DSC;
+        } else {
+          sortOrder = SORT_ORDER_ASC;
+        }
+      }
+
+      public int compare(DatanodeDescriptor d1,
+                         DatanodeDescriptor d2) {
+        int ret = 0;
+        switch (sortField) {
+        case FIELD_LAST_CONTACT:
+          ret = (int) (d2.getLastUpdate() - d1.getLastUpdate());
+          break;
+        case FIELD_CAPACITY:
+          long  dlong = d1.getCapacity() - d2.getCapacity();
+          ret = (dlong < 0) ? -1 : ((dlong > 0) ? 1 : 0);
+          break;
+        case FIELD_USED:
+          dlong = d1.getDfsUsed() - d2.getDfsUsed();
+          ret = (dlong < 0) ? -1 : ((dlong > 0) ? 1 : 0);
+          break;
+        case FIELD_NONDFS_USED:
+          dlong = d1.getNonDfsUsed() - d2.getNonDfsUsed();
+          ret = (dlong < 0) ? -1 : ((dlong > 0) ? 1 : 0);
+          break;
+        case FIELD_REMAINING:
+          dlong = d1.getRemaining() - d2.getRemaining();
+          ret = (dlong < 0) ? -1 : ((dlong > 0) ? 1 : 0);
+          break;
+        case FIELD_PERCENT_USED:
+          double ddbl =((d1.getDfsUsedPercent())-
+                        (d2.getDfsUsedPercent()));
+          ret = (ddbl < 0) ? -1 : ((ddbl > 0) ? 1 : 0);
+          break;
+        case FIELD_PERCENT_REMAINING:
+          ddbl =((d1.getRemainingPercent())-
+                 (d2.getRemainingPercent()));
+          ret = (ddbl < 0) ? -1 : ((ddbl > 0) ? 1 : 0);
+          break;
+        case FIELD_BLOCKS:
+          ret = d1.numBlocks() - d2.numBlocks();
+          break;
+        case FIELD_NAME: 
+          ret = d1.getHostName().compareTo(d2.getHostName());
+          break;
+        }
+        return (sortOrder == SORT_ORDER_DSC) ? -ret : ret;
+      }
+    }
+        
+    Collections.sort(nodes, new NodeComapare(field, order));
+  }
+
+  public static void printPathWithLinks(String dir, JspWriter out, int namenodeInfoPort ) throws IOException {
+    try {
+      String[] parts = dir.split(Path.SEPARATOR);
+      StringBuilder tempPath = new StringBuilder(dir.length());
+      out.print("<a href=\"browseDirectory.jsp" + "?dir="+ Path.SEPARATOR
+          + "&namenodeInfoPort=" + namenodeInfoPort
+          + "\">" + Path.SEPARATOR + "</a>");
+      tempPath.append(Path.SEPARATOR);
+      for (int i = 0; i < parts.length-1; i++) {
+        if (!parts[i].equals("")) {
+          tempPath.append(parts[i]);
+          out.print("<a href=\"browseDirectory.jsp" + "?dir="
+              + tempPath.toString() + "&namenodeInfoPort=" + namenodeInfoPort);
+          out.print("\">" + parts[i] + "</a>" + Path.SEPARATOR);
+          tempPath.append(Path.SEPARATOR);
+        }
+      }
+      if(parts.length > 0) {
+        out.print(parts[parts.length-1]);
+      }
+    }
+    catch (UnsupportedEncodingException ex) {
+      ex.printStackTrace();
+    }
+  }
+
+  public static void printGotoForm(JspWriter out, int namenodeInfoPort, String file) throws IOException {
+    out.print("<form action=\"browseDirectory.jsp\" method=\"get\" name=\"goto\">");
+    out.print("Goto : ");
+    out.print("<input name=\"dir\" type=\"text\" width=\"50\" id\"dir\" value=\""+ file+"\">");
+    out.print("<input name=\"go\" type=\"submit\" value=\"go\">");
+    out.print("<input name=\"namenodeInfoPort\" type=\"hidden\" "
+        + "value=\"" + namenodeInfoPort  + "\">");
+    out.print("</form>");
+  }
+  
+  public static void createTitle(JspWriter out, 
+      HttpServletRequest req, String  file) throws IOException{
+    if(file == null) file = "";
+    int start = Math.max(0,file.length() - 100);
+    if(start != 0)
+      file = "..." + file.substring(start, file.length());
+    out.print("<title>HDFS:" + file + "</title>");
+  }
+
+  /** Convert a String to chunk-size-to-view. */
+  public static int string2ChunkSizeToView(String s) {
+    int n = s == null? 0: Integer.parseInt(s);
+    return n > 0? n: defaultChunkSizeToView;
+  }
+
+  /** Return a table containing version information. */
+  public static String getVersionTable(FSNamesystem fsn) {
+    return "<div id='dfstable'><table>"       
+        + "\n  <tr><td id='col1'>Started:</td><td>" + fsn.getStartTime() + "</td></tr>\n"
+        + "\n  <tr><td id='col1'>Version:</td><td>" + VersionInfo.getVersion() + ", " + VersionInfo.getRevision()
+        + "\n  <tr><td id='col1'>Compiled:</td><td>" + VersionInfo.getDate() + " by " + VersionInfo.getUser() + " from " + VersionInfo.getBranch()
+        + "\n  <tr><td id='col1'>Upgrades:</td><td>" + getUpgradeStatusText(fsn)
+        + "\n</table></div>";
+  }
+
+  /** Return a table containing version information. */
+  public static String getVersionTable() {
+    return "<div id='dfstable'><table>"       
+        + "\n  <tr><td id='col1'>Version:</td><td>" + VersionInfo.getVersion() + ", " + VersionInfo.getRevision()
+        + "\n  <tr><td id='col1'>Compiled:</td><td>" + VersionInfo.getDate() + " by " + VersionInfo.getUser() + " from " + VersionInfo.getBranch()
+        + "\n</table></div>";
+  }
+
+  /**
+   * Validate filename. 
+   * @return null if the filename is invalid.
+   *         Otherwise, return the validated filename.
+   */
+  public static String validatePath(String p) {
+    return p == null || p.length() == 0?
+        null: new Path(p).toUri().getPath();
+  }
+
+  /**
+   * Validate a long value. 
+   * @return null if the value is invalid.
+   *         Otherwise, return the validated Long object.
+   */
+  public static Long validateLong(String value) {
+    return value == null? null: Long.parseLong(value);
+  }
+
+  /**
+   * Validate a URL.
+   * @return null if the value is invalid.
+   *         Otherwise, return the validated URL String.
+   */
+  public static String validateURL(String value) {
+    try {
+      return URLEncoder.encode(new URL(value).toString(), "UTF-8");
+    } catch (IOException e) {
+      return null;
+    }
+  }
+}

+ 32 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/LeaseExpiredException.java

@@ -0,0 +1,32 @@
+/**
+ * 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 java.io.IOException;
+
+/**
+ * The lease that was being used to create this file has expired.
+ */
+public class LeaseExpiredException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public LeaseExpiredException(String msg) {
+    super(msg);
+  }
+}

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

@@ -0,0 +1,399 @@
+/**
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+
+/**
+ * LeaseManager does the lease housekeeping for writing on files.   
+ * This class also provides useful static methods for lease recovery.
+ * 
+ * Lease Recovery Algorithm
+ * 1) Namenode retrieves lease information
+ * 2) For each file f in the lease, consider the last block b of f
+ * 2.1) Get the datanodes which contains b
+ * 2.2) Assign one of the datanodes as the primary datanode p
+
+ * 2.3) p obtains a new generation stamp form the namenode
+ * 2.4) p get the block info from each datanode
+ * 2.5) p computes the minimum block length
+ * 2.6) p updates the datanodes, which have a valid generation stamp,
+ *      with the new generation stamp and the minimum block length 
+ * 2.7) p acknowledges the namenode the update results
+
+ * 2.8) Namenode updates the BlockInfo
+ * 2.9) Namenode removes f from the lease
+ *      and removes the lease once all files have been removed
+ * 2.10) Namenode commit changes to edit log
+ */
+public class LeaseManager {
+  public static final Log LOG = LogFactory.getLog(LeaseManager.class);
+
+  private final FSNamesystem fsnamesystem;
+
+  private long softLimit = FSConstants.LEASE_SOFTLIMIT_PERIOD;
+  private long hardLimit = FSConstants.LEASE_HARDLIMIT_PERIOD;
+
+  //
+  // Used for handling lock-leases
+  // Mapping: leaseHolder -> Lease
+  //
+  private SortedMap<String, Lease> leases = new TreeMap<String, Lease>();
+  // Set of: Lease
+  private SortedSet<Lease> sortedLeases = new TreeSet<Lease>();
+
+  // 
+  // Map path names to leases. It is protected by the sortedLeases lock.
+  // The map stores pathnames in lexicographical order.
+  //
+  private SortedMap<String, Lease> sortedLeasesByPath = new TreeMap<String, Lease>();
+
+  LeaseManager(FSNamesystem fsnamesystem) {this.fsnamesystem = fsnamesystem;}
+
+  Lease getLease(String holder) {
+    return leases.get(holder);
+  }
+  
+  SortedSet<Lease> getSortedLeases() {return sortedLeases;}
+
+  /** @return the lease containing src */
+  public Lease getLeaseByPath(String src) {return sortedLeasesByPath.get(src);}
+
+  /** @return the number of leases currently in the system */
+  public synchronized int countLease() {return sortedLeases.size();}
+
+  /** @return the number of paths contained in all leases */
+  synchronized int countPath() {
+    int count = 0;
+    for(Lease lease : sortedLeases) {
+      count += lease.getPaths().size();
+    }
+    return count;
+  }
+  
+  /**
+   * Adds (or re-adds) the lease for the specified file.
+   */
+  synchronized void addLease(String holder, String src) {
+    Lease lease = getLease(holder);
+    if (lease == null) {
+      lease = new Lease(holder);
+      leases.put(holder, lease);
+      sortedLeases.add(lease);
+    } else {
+      renewLease(lease);
+    }
+    sortedLeasesByPath.put(src, lease);
+    lease.paths.add(src);
+  }
+
+  /**
+   * Remove the specified lease and src.
+   */
+  synchronized void removeLease(Lease lease, String src) {
+    sortedLeasesByPath.remove(src);
+    if (!lease.removePath(src)) {
+      LOG.error(src + " not found in lease.paths (=" + lease.paths + ")");
+    }
+
+    if (!lease.hasPath()) {
+      leases.remove(lease.holder);
+      if (!sortedLeases.remove(lease)) {
+        LOG.error(lease + " not found in sortedLeases");
+      }
+    }
+  }
+
+  /**
+   * Remove the lease for the specified holder and src
+   */
+  synchronized void removeLease(String holder, String src) {
+    Lease lease = getLease(holder);
+    if (lease != null) {
+      removeLease(lease, src);
+    }
+  }
+
+  /**
+   * Finds the pathname for the specified pendingFile
+   */
+  synchronized String findPath(INodeFileUnderConstruction pendingFile
+      ) throws IOException {
+    Lease lease = getLease(pendingFile.clientName);
+    if (lease != null) {
+      String src = lease.findPath(pendingFile);
+      if (src != null) {
+        return src;
+      }
+    }
+    throw new IOException("pendingFile (=" + pendingFile + ") not found."
+        + "(lease=" + lease + ")");
+  }
+
+  /**
+   * Renew the lease(s) held by the given client
+   */
+  synchronized void renewLease(String holder) {
+    renewLease(getLease(holder));
+  }
+  synchronized void renewLease(Lease lease) {
+    if (lease != null) {
+      sortedLeases.remove(lease);
+      lease.renew();
+      sortedLeases.add(lease);
+    }
+  }
+
+  /************************************************************
+   * A Lease governs all the locks held by a single client.
+   * For each client there's a corresponding lease, whose
+   * timestamp is updated when the client periodically
+   * checks in.  If the client dies and allows its lease to
+   * expire, all the corresponding locks can be released.
+   *************************************************************/
+  class Lease implements Comparable<Lease> {
+    private final String holder;
+    private long lastUpdate;
+    private final Collection<String> paths = new TreeSet<String>();
+  
+    /** Only LeaseManager object can create a lease */
+    private Lease(String holder) {
+      this.holder = holder;
+      renew();
+    }
+    /** Only LeaseManager object can renew a lease */
+    private void renew() {
+      this.lastUpdate = FSNamesystem.now();
+    }
+
+    /** @return true if the Hard Limit Timer has expired */
+    public boolean expiredHardLimit() {
+      return FSNamesystem.now() - lastUpdate > hardLimit;
+    }
+
+    /** @return true if the Soft Limit Timer has expired */
+    public boolean expiredSoftLimit() {
+      return FSNamesystem.now() - lastUpdate > softLimit;
+    }
+
+    /**
+     * @return the path associated with the pendingFile and null if not found.
+     */
+    private String findPath(INodeFileUnderConstruction pendingFile) {
+      for(String src : paths) {
+        if (fsnamesystem.dir.getFileINode(src) == pendingFile) {
+          return src;
+        }
+      }
+      return null;
+    }
+
+    /** Does this lease contain any path? */
+    boolean hasPath() {return !paths.isEmpty();}
+
+    boolean removePath(String src) {
+      return paths.remove(src);
+    }
+
+    /** {@inheritDoc} */
+    public String toString() {
+      return "[Lease.  Holder: " + holder
+          + ", pendingcreates: " + paths.size() + "]";
+    }
+  
+    /** {@inheritDoc} */
+    public int compareTo(Lease o) {
+      Lease l1 = this;
+      Lease l2 = o;
+      long lu1 = l1.lastUpdate;
+      long lu2 = l2.lastUpdate;
+      if (lu1 < lu2) {
+        return -1;
+      } else if (lu1 > lu2) {
+        return 1;
+      } else {
+        return l1.holder.compareTo(l2.holder);
+      }
+    }
+  
+    /** {@inheritDoc} */
+    public boolean equals(Object o) {
+      if (!(o instanceof Lease)) {
+        return false;
+      }
+      Lease obj = (Lease) o;
+      if (lastUpdate == obj.lastUpdate &&
+          holder.equals(obj.holder)) {
+        return true;
+      }
+      return false;
+    }
+  
+    /** {@inheritDoc} */
+    public int hashCode() {
+      return holder.hashCode();
+    }
+    
+    Collection<String> getPaths() {
+      return paths;
+    }
+    
+    void replacePath(String oldpath, String newpath) {
+      paths.remove(oldpath);
+      paths.add(newpath);
+    }
+  }
+
+  synchronized void changeLease(String src, String dst,
+      String overwrite, String replaceBy) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(getClass().getSimpleName() + ".changelease: " +
+               " src=" + src + ", dest=" + dst + 
+               ", overwrite=" + overwrite +
+               ", replaceBy=" + replaceBy);
+    }
+
+    for(Map.Entry<String, Lease> entry : findLeaseWithPrefixPath(src, sortedLeasesByPath)) {
+      final String oldpath = entry.getKey();
+      final Lease lease = entry.getValue();
+      final String newpath = oldpath.replaceFirst(
+          java.util.regex.Pattern.quote(overwrite), replaceBy);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("changeLease: replacing " + oldpath + " with " + newpath);
+      }
+      lease.replacePath(oldpath, newpath);
+      sortedLeasesByPath.remove(oldpath);
+      sortedLeasesByPath.put(newpath, lease);
+    }
+  }
+
+  synchronized void removeLeaseWithPrefixPath(String prefix) {
+    for(Map.Entry<String, Lease> entry : findLeaseWithPrefixPath(prefix, sortedLeasesByPath)) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(LeaseManager.class.getSimpleName()
+            + ".removeLeaseWithPrefixPath: entry=" + entry);
+      }
+      removeLease(entry.getValue(), entry.getKey());    
+    }
+  }
+
+  static private List<Map.Entry<String, Lease>> findLeaseWithPrefixPath(
+      String prefix, SortedMap<String, Lease> path2lease) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(LeaseManager.class.getSimpleName() + ".findLease: prefix=" + prefix);
+    }
+
+    List<Map.Entry<String, Lease>> entries = new ArrayList<Map.Entry<String, Lease>>();
+    final int srclen = prefix.length();
+
+    for(Map.Entry<String, Lease> entry : path2lease.tailMap(prefix).entrySet()) {
+      final String p = entry.getKey();
+      if (!p.startsWith(prefix)) {
+        return entries;
+      }
+      if (p.length() == srclen || p.charAt(srclen) == Path.SEPARATOR_CHAR) {
+        entries.add(entry);
+      }
+    }
+    return entries;
+  }
+
+  public void setLeasePeriod(long softLimit, long hardLimit) {
+    this.softLimit = softLimit;
+    this.hardLimit = hardLimit; 
+  }
+  
+  /******************************************************
+   * Monitor checks for leases that have expired,
+   * and disposes of them.
+   ******************************************************/
+  class Monitor implements Runnable {
+    final String name = getClass().getSimpleName();
+
+    /** Check leases periodically. */
+    public void run() {
+      for(; fsnamesystem.isRunning(); ) {
+        synchronized(fsnamesystem) {
+          checkLeases();
+        }
+
+        try {
+          Thread.sleep(2000);
+        } catch(InterruptedException ie) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(name + " is interrupted", ie);
+          }
+        }
+      }
+    }
+  }
+
+  /** Check the leases beginning from the oldest. */
+  private synchronized void checkLeases() {
+    for(; sortedLeases.size() > 0; ) {
+      final Lease oldest = sortedLeases.first();
+      if (!oldest.expiredHardLimit()) {
+        return;
+      }
+
+      LOG.info("Lease " + oldest + " has expired hard limit");
+
+      final List<String> removing = new ArrayList<String>();
+      // need to create a copy of the oldest lease paths, becuase 
+      // internalReleaseLease() removes paths corresponding to empty files,
+      // i.e. it needs to modify the collection being iterated over
+      // causing ConcurrentModificationException
+      String[] leasePaths = new String[oldest.getPaths().size()];
+      oldest.getPaths().toArray(leasePaths);
+      for(String p : leasePaths) {
+        try {
+          fsnamesystem.internalReleaseLease(oldest, p);
+        } catch (IOException e) {
+          LOG.error("Cannot release the path "+p+" in the lease "+oldest, e);
+          removing.add(p);
+        }
+      }
+
+      for(String p : removing) {
+        removeLease(oldest, p);
+      }
+    }
+  }
+
+  /** {@inheritDoc} */
+  public synchronized String toString() {
+    return getClass().getSimpleName() + "= {"
+        + "\n leases=" + leases
+        + "\n sortedLeases=" + sortedLeases
+        + "\n sortedLeasesByPath=" + sortedLeasesByPath
+        + "\n}";
+  }
+}

+ 178 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java

@@ -0,0 +1,178 @@
+/**
+ * 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 org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.util.VersionInfo;
+
+import org.znerd.xmlenc.*;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Stack;
+import java.util.TimeZone;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+/**
+ * Obtain meta-information about a filesystem.
+ * @see org.apache.hadoop.hdfs.HftpFileSystem
+ */
+public class ListPathsServlet extends DfsServlet {
+  /** For java.io.Serializable */
+  private static final long serialVersionUID = 1L;
+
+  public static final SimpleDateFormat df =
+    new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ");
+  static {
+    df.setTimeZone(TimeZone.getTimeZone("UTC"));
+  }
+
+  /**
+   * Write a node to output.
+   * Node information includes path, modification, permission, owner and group.
+   * For files, it also includes size, replication and block-size. 
+   */
+  static void writeInfo(FileStatus i, XMLOutputter doc) throws IOException {
+    doc.startTag(i.isDir() ? "directory" : "file");
+    doc.attribute("path", i.getPath().toUri().getPath());
+    doc.attribute("modified", df.format(new Date(i.getModificationTime())));
+    doc.attribute("accesstime", df.format(new Date(i.getAccessTime())));
+    if (!i.isDir()) {
+      doc.attribute("size", String.valueOf(i.getLen()));
+      doc.attribute("replication", String.valueOf(i.getReplication()));
+      doc.attribute("blocksize", String.valueOf(i.getBlockSize()));
+    }
+    doc.attribute("permission", (i.isDir()? "d": "-") + i.getPermission());
+    doc.attribute("owner", i.getOwner());
+    doc.attribute("group", i.getGroup());
+    doc.endTag();
+  }
+
+  /**
+   * Build a map from the query string, setting values and defaults.
+   */
+  protected Map<String,String> buildRoot(HttpServletRequest request,
+      XMLOutputter doc) {
+    final String path = request.getPathInfo() != null
+      ? request.getPathInfo() : "/";
+    final String exclude = request.getParameter("exclude") != null
+      ? request.getParameter("exclude") : "\\..*\\.crc";
+    final String filter = request.getParameter("filter") != null
+      ? request.getParameter("filter") : ".*";
+    final boolean recur = request.getParameter("recursive") != null
+      && "yes".equals(request.getParameter("recursive"));
+
+    Map<String, String> root = new HashMap<String, String>();
+    root.put("path", path);
+    root.put("recursive", recur ? "yes" : "no");
+    root.put("filter", filter);
+    root.put("exclude", exclude);
+    root.put("time", df.format(new Date()));
+    root.put("version", VersionInfo.getVersion());
+    return root;
+  }
+
+  /**
+   * Service a GET request as described below.
+   * Request:
+   * {@code
+   * GET http://<nn>:<port>/listPaths[/<path>][<?option>[&option]*] HTTP/1.1
+   * }
+   *
+   * Where <i>option</i> (default) in:
+   * recursive (&quot;no&quot;)
+   * filter (&quot;.*&quot;)
+   * exclude (&quot;\..*\.crc&quot;)
+   *
+   * Response: A flat list of files/directories in the following format:
+   * {@code
+   *   <listing path="..." recursive="(yes|no)" filter="..."
+   *            time="yyyy-MM-dd hh:mm:ss UTC" version="...">
+   *     <directory path="..." modified="yyyy-MM-dd hh:mm:ss"/>
+   *     <file path="..." modified="yyyy-MM-dd'T'hh:mm:ssZ" accesstime="yyyy-MM-dd'T'hh:mm:ssZ" 
+   *           blocksize="..."
+   *           replication="..." size="..."/>
+   *   </listing>
+   * }
+   */
+  public void doGet(HttpServletRequest request, HttpServletResponse response)
+    throws ServletException, IOException {
+    final UnixUserGroupInformation ugi = getUGI(request);
+    final PrintWriter out = response.getWriter();
+    final XMLOutputter doc = new XMLOutputter(out, "UTF-8");
+    try {
+      final Map<String, String> root = buildRoot(request, doc);
+      final String path = root.get("path");
+      final boolean recur = "yes".equals(root.get("recursive"));
+      final Pattern filter = Pattern.compile(root.get("filter"));
+      final Pattern exclude = Pattern.compile(root.get("exclude"));
+      ClientProtocol nnproxy = createNameNodeProxy(ugi);
+
+      doc.declaration();
+      doc.startTag("listing");
+      for (Map.Entry<String,String> m : root.entrySet()) {
+        doc.attribute(m.getKey(), m.getValue());
+      }
+
+      FileStatus base = nnproxy.getFileInfo(path);
+      if ((base != null) && base.isDir()) {
+        writeInfo(base, doc);
+      }
+
+      Stack<String> pathstack = new Stack<String>();
+      pathstack.push(path);
+      while (!pathstack.empty()) {
+        String p = pathstack.pop();
+        try {
+          for (FileStatus i : nnproxy.getListing(p)) {
+            if (exclude.matcher(i.getPath().getName()).matches()
+                || !filter.matcher(i.getPath().getName()).matches()) {
+              continue;
+            }
+            if (recur && i.isDir()) {
+              pathstack.push(i.getPath().toUri().getPath());
+            }
+            writeInfo(i, doc);
+          }
+        }
+        catch(RemoteException re) {re.writeXml(p, doc);}
+      }
+    } catch (PatternSyntaxException e) {
+      out.println(e.toString());
+    } finally {
+      if (doc != null) {
+        doc.endDocument();
+      }
+
+      if (out != null) {
+        out.close();
+      }
+    }
+  }
+}

+ 1159 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -0,0 +1,1159 @@
+/**
+ * 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 java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.Trash;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.HDFSPolicyProvider;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+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.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
+import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.CompleteFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.NodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.ExportedAccessKeys;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.authorize.ConfiguredPolicy;
+import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
+import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.ServicePlugin;
+import org.apache.hadoop.util.StringUtils;
+
+/**********************************************************
+ * NameNode serves as both directory namespace manager and
+ * "inode table" for the Hadoop DFS.  There is a single NameNode
+ * running in any DFS deployment.  (Well, except when there
+ * is a second backup/failover NameNode.)
+ *
+ * The NameNode controls two critical tables:
+ *   1)  filename->blocksequence (namespace)
+ *   2)  block->machinelist ("inodes")
+ *
+ * The first table is stored on disk and is very precious.
+ * The second table is rebuilt every time the NameNode comes
+ * up.
+ *
+ * 'NameNode' refers to both this class as well as the 'NameNode server'.
+ * The 'FSNamesystem' class actually performs most of the filesystem
+ * management.  The majority of the 'NameNode' class itself is concerned
+ * with exposing the IPC interface and the http server to the outside world,
+ * plus some configuration management.
+ *
+ * NameNode implements the ClientProtocol interface, which allows
+ * clients to ask for DFS services.  ClientProtocol is not
+ * designed for direct use by authors of DFS client code.  End-users
+ * should instead use the org.apache.nutch.hadoop.fs.FileSystem class.
+ *
+ * NameNode also implements the DatanodeProtocol interface, used by
+ * DataNode programs that actually store DFS data blocks.  These
+ * methods are invoked repeatedly and automatically by all the
+ * DataNodes in a DFS deployment.
+ *
+ * NameNode also implements the NamenodeProtocol interface, used by
+ * secondary namenodes or rebalancing processes to get partial namenode's
+ * state, for example partial blocksMap etc.
+ **********************************************************/
+public class NameNode implements ClientProtocol, DatanodeProtocol,
+                                 NamenodeProtocol, FSConstants,
+                                 RefreshAuthorizationPolicyProtocol {
+  static{
+    Configuration.addDefaultResource("hdfs-default.xml");
+    Configuration.addDefaultResource("hdfs-site.xml");
+  }
+  
+  public long getProtocolVersion(String protocol, 
+                                 long clientVersion) throws IOException { 
+    if (protocol.equals(ClientProtocol.class.getName())) {
+      return ClientProtocol.versionID; 
+    } else if (protocol.equals(DatanodeProtocol.class.getName())){
+      return DatanodeProtocol.versionID;
+    } else if (protocol.equals(NamenodeProtocol.class.getName())){
+      return NamenodeProtocol.versionID;
+    } else if (protocol.equals(RefreshAuthorizationPolicyProtocol.class.getName())){
+      return RefreshAuthorizationPolicyProtocol.versionID;
+    } else {
+      throw new IOException("Unknown protocol to name node: " + protocol);
+    }
+  }
+    
+  public static final int DEFAULT_PORT = 8020;
+
+  public static final Log LOG = LogFactory.getLog(NameNode.class.getName());
+  public static final Log stateChangeLog = LogFactory.getLog("org.apache.hadoop.hdfs.StateChange");
+
+  protected FSNamesystem namesystem; 
+  protected NamenodeRole role;
+  /** RPC server */
+  protected Server server;
+  /** RPC server address */
+  protected InetSocketAddress rpcAddress = null;
+  /** httpServer */
+  protected HttpServer httpServer;
+  /** HTTP server address */
+  protected InetSocketAddress httpAddress = null;
+  private Thread emptier;
+  /** only used for testing purposes  */
+  protected boolean stopRequested = false;
+  /** Registration information of this name-node  */
+  protected NamenodeRegistration nodeRegistration;
+  /** Is service level authorization enabled? */
+  private boolean serviceAuthEnabled = false;
+  /** Activated plug-ins. */
+  private List<ServicePlugin> plugins;
+  
+  /** Format a new filesystem.  Destroys any filesystem that may already
+   * exist at this location.  **/
+  public static void format(Configuration conf) throws IOException {
+    format(conf, false);
+  }
+
+  static NameNodeMetrics myMetrics;
+
+  /** Return the {@link FSNamesystem} object.
+   * @return {@link FSNamesystem} object.
+   */
+  public FSNamesystem getNamesystem() {
+    return namesystem;
+  }
+
+  private static void initMetrics(Configuration conf, NamenodeRole role) {
+    myMetrics = new NameNodeMetrics(conf, role);
+  }
+
+  public static NameNodeMetrics getNameNodeMetrics() {
+    return myMetrics;
+  }
+  
+  public static InetSocketAddress getAddress(String address) {
+    return NetUtils.createSocketAddr(address, DEFAULT_PORT);
+  }
+
+  public static InetSocketAddress getAddress(Configuration conf) {
+    return getAddress(FileSystem.getDefaultUri(conf).getAuthority());
+  }
+
+  public static URI getUri(InetSocketAddress namenode) {
+    int port = namenode.getPort();
+    String portString = port == DEFAULT_PORT ? "" : (":"+port);
+    return URI.create("hdfs://"+ namenode.getHostName()+portString);
+  }
+
+  /**
+   * Compose a "host:port" string from the address.
+   */
+  public static String getHostPortString(InetSocketAddress addr) {
+    return addr.getHostName() + ":" + addr.getPort();
+  }
+
+  //
+  // Common NameNode methods implementation for the active name-node role.
+  //
+  public NamenodeRole getRole() {
+    return role;
+  }
+
+  boolean isRole(NamenodeRole that) {
+    return role.equals(that);
+  }
+
+  protected InetSocketAddress getRpcServerAddress(Configuration conf) throws IOException {
+    return getAddress(conf);
+  }
+
+  protected void setRpcServerAddress(Configuration conf) {
+    FileSystem.setDefaultUri(conf, getUri(rpcAddress));
+  }
+
+  protected InetSocketAddress getHttpServerAddress(Configuration conf) {
+    return  NetUtils.createSocketAddr(
+        conf.get("dfs.http.address", "0.0.0.0:50070"));
+  }
+
+  protected void setHttpServerAddress(Configuration conf){
+    conf.set("dfs.http.address", getHostPortString(httpAddress));
+  }
+
+  protected void loadNamesystem(Configuration conf) throws IOException {
+    this.namesystem = new FSNamesystem(conf);
+  }
+
+  NamenodeRegistration getRegistration() {
+    return nodeRegistration;
+  }
+
+  NamenodeRegistration setRegistration() {
+    nodeRegistration = new NamenodeRegistration(
+        getHostPortString(rpcAddress),
+        getHostPortString(httpAddress),
+        getFSImage(), getRole(), getFSImage().getCheckpointTime());
+    return nodeRegistration;
+  }
+
+  /**
+   * Initialize name-node.
+   * 
+   * @param conf the configuration
+   */
+  protected void initialize(Configuration conf) throws IOException {
+    InetSocketAddress socAddr = getRpcServerAddress(conf);
+    int handlerCount = conf.getInt("dfs.namenode.handler.count", 10);
+    
+    // set service-level authorization security policy
+    if (serviceAuthEnabled = 
+          conf.getBoolean(
+            ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, false)) {
+      PolicyProvider policyProvider = 
+        (PolicyProvider)(ReflectionUtils.newInstance(
+            conf.getClass(PolicyProvider.POLICY_PROVIDER_CONFIG, 
+                HDFSPolicyProvider.class, PolicyProvider.class), 
+            conf));
+      SecurityUtil.setPolicy(new ConfiguredPolicy(conf, policyProvider));
+    }
+
+    // create rpc server 
+    this.server = RPC.getServer(this, socAddr.getHostName(), socAddr.getPort(),
+                                handlerCount, false, conf);
+
+    // The rpc-server port can be ephemeral... ensure we have the correct info
+    this.rpcAddress = this.server.getListenerAddress(); 
+    setRpcServerAddress(conf);
+
+    NameNode.initMetrics(conf, this.getRole());
+    loadNamesystem(conf);
+    activate(conf);
+    LOG.info(getRole() + " up at: " + rpcAddress);
+  }
+
+  /**
+   * Activate name-node servers and threads.
+   */
+  void activate(Configuration conf) throws IOException {
+    namesystem.activate(conf);
+    startHttpServer(conf);
+    server.start();  //start RPC server
+    startTrashEmptier(conf);
+    
+    plugins = conf.getInstances("dfs.namenode.plugins", ServicePlugin.class);
+    for (ServicePlugin p: plugins) {
+      try {
+        p.start(this);
+      } catch (Throwable t) {
+        LOG.warn("ServicePlugin " + p + " could not be started", t);
+      }
+    }
+  }
+
+  private void startTrashEmptier(Configuration conf) throws IOException {
+    long trashInterval = conf.getLong("fs.trash.interval", 0);
+    if(trashInterval == 0)
+      return;
+    this.emptier = new Thread(new Trash(conf).getEmptier(), "Trash Emptier");
+    this.emptier.setDaemon(true);
+    this.emptier.start();
+  }
+
+  private void startHttpServer(Configuration conf) throws IOException {
+    InetSocketAddress infoSocAddr = getHttpServerAddress(conf);
+    String infoHost = infoSocAddr.getHostName();
+    int infoPort = infoSocAddr.getPort();
+    this.httpServer = new HttpServer("hdfs", infoHost, infoPort, 
+        infoPort == 0, conf);
+    if (conf.getBoolean("dfs.https.enable", false)) {
+      boolean needClientAuth = conf.getBoolean("dfs.https.need.client.auth", false);
+      InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(conf.get(
+          "dfs.https.address", infoHost + ":" + 0));
+      Configuration sslConf = new Configuration(false);
+      sslConf.addResource(conf.get("dfs.https.server.keystore.resource",
+          "ssl-server.xml"));
+      this.httpServer.addSslListener(secInfoSocAddr, sslConf, needClientAuth);
+      // assume same ssl port for all datanodes
+      InetSocketAddress datanodeSslPort = NetUtils.createSocketAddr(conf.get(
+          "dfs.datanode.https.address", infoHost + ":" + 50475));
+      this.httpServer.setAttribute("datanode.https.port", datanodeSslPort
+          .getPort());
+    }
+    this.httpServer.setAttribute("name.node", this);
+    this.httpServer.setAttribute("name.node.address", getNameNodeAddress());
+    this.httpServer.setAttribute("name.system.image", getFSImage());
+    this.httpServer.setAttribute("name.conf", conf);
+    this.httpServer.addInternalServlet("fsck", "/fsck", FsckServlet.class);
+    this.httpServer.addInternalServlet("getimage", "/getimage", GetImageServlet.class);
+    this.httpServer.addInternalServlet("listPaths", "/listPaths/*", ListPathsServlet.class);
+    this.httpServer.addInternalServlet("data", "/data/*", FileDataServlet.class);
+    this.httpServer.addInternalServlet("checksum", "/fileChecksum/*",
+        FileChecksumServlets.RedirectServlet.class);
+    this.httpServer.start();
+
+    // The web-server port can be ephemeral... ensure we have the correct info
+    infoPort = this.httpServer.getPort();
+    this.httpAddress = new InetSocketAddress(infoHost, infoPort);
+    setHttpServerAddress(conf);
+    LOG.info(getRole() + " Web-server up at: " + httpAddress);
+  }
+
+  /**
+   * Start NameNode.
+   * <p>
+   * The name-node can be started with one of the following startup options:
+   * <ul> 
+   * <li>{@link StartupOption#REGULAR REGULAR} - normal name node startup</li>
+   * <li>{@link StartupOption#FORMAT FORMAT} - format name node</li>
+   * <li>{@link StartupOption#BACKUP BACKUP} - start backup node</li>
+   * <li>{@link StartupOption#CHECKPOINT CHECKPOINT} - start checkpoint node</li>
+   * <li>{@link StartupOption#UPGRADE UPGRADE} - start the cluster  
+   * upgrade and create a snapshot of the current file system state</li> 
+   * <li>{@link StartupOption#ROLLBACK ROLLBACK} - roll the  
+   *            cluster back to the previous state</li>
+   * <li>{@link StartupOption#FINALIZE FINALIZE} - finalize 
+   *            previous upgrade</li>
+   * <li>{@link StartupOption#IMPORT IMPORT} - import checkpoint</li>
+   * </ul>
+   * The option is passed via configuration field: 
+   * <tt>dfs.namenode.startup</tt>
+   * 
+   * The conf will be modified to reflect the actual ports on which 
+   * the NameNode is up and running if the user passes the port as
+   * <code>zero</code> in the conf.
+   * 
+   * @param conf  confirguration
+   * @throws IOException
+   */
+  public NameNode(Configuration conf) throws IOException {
+    this(conf, NamenodeRole.ACTIVE);
+  }
+
+  protected NameNode(Configuration conf, NamenodeRole role) throws IOException {
+    this.role = role;
+    try {
+      initialize(conf);
+    } catch (IOException e) {
+      this.stop();
+      throw e;
+    }
+  }
+
+  /**
+   * Wait for service to finish.
+   * (Normally, it runs forever.)
+   */
+  public void join() {
+    try {
+      this.server.join();
+    } catch (InterruptedException ie) {
+    }
+  }
+
+  /**
+   * Stop all NameNode threads and wait for all to finish.
+   */
+  public void stop() {
+    if (stopRequested)
+      return;
+    stopRequested = true;
+    if (plugins != null) {
+      for (ServicePlugin p : plugins) {
+        try {
+          p.stop();
+        } catch (Throwable t) {
+          LOG.warn("ServicePlugin " + p + " could not be stopped", t);
+        }
+      }
+    }
+    try {
+      if (httpServer != null) httpServer.stop();
+    } catch (Exception e) {
+      LOG.error(StringUtils.stringifyException(e));
+    }
+    if(namesystem != null) namesystem.close();
+    if(emptier != null) emptier.interrupt();
+    if(server != null) server.stop();
+    if (myMetrics != null) {
+      myMetrics.shutdown();
+    }
+    if (namesystem != null) {
+      namesystem.shutdown();
+    }
+  }
+  
+  /////////////////////////////////////////////////////
+  // NamenodeProtocol
+  /////////////////////////////////////////////////////
+  @Override // NamenodeProtocol
+  public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
+  throws IOException {
+    if(size <= 0) {
+      throw new IllegalArgumentException(
+        "Unexpected not positive size: "+size);
+    }
+
+    return namesystem.getBlocks(datanode, size); 
+  }
+
+  /** {@inheritDoc} */
+  public ExportedAccessKeys getAccessKeys() throws IOException {
+    return namesystem.getAccessKeys();
+  }
+
+  @Override // NamenodeProtocol
+  public void errorReport(NamenodeRegistration registration,
+                          int errorCode, 
+                          String msg) throws IOException {
+    verifyRequest(registration);
+    LOG.info("Error report from " + registration + ": " + msg);
+    if(errorCode == FATAL)
+      namesystem.releaseBackupNode(registration);
+  }
+
+  @Override // NamenodeProtocol
+  public NamenodeRegistration register(NamenodeRegistration registration)
+  throws IOException {
+    verifyVersion(registration.getVersion());
+    namesystem.registerBackupNode(registration);
+    return setRegistration();
+  }
+
+  @Override // NamenodeProtocol
+  public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
+  throws IOException {
+    verifyRequest(registration);
+    if(!isRole(NamenodeRole.ACTIVE))
+      throw new IOException("Only an ACTIVE node can invoke startCheckpoint.");
+    return namesystem.startCheckpoint(registration, setRegistration());
+  }
+
+  @Override // NamenodeProtocol
+  public void endCheckpoint(NamenodeRegistration registration,
+                            CheckpointSignature sig) throws IOException {
+    verifyRequest(registration);
+    if(!isRole(NamenodeRole.ACTIVE))
+      throw new IOException("Only an ACTIVE node can invoke endCheckpoint.");
+    namesystem.endCheckpoint(registration, sig);
+  }
+
+  @Override // NamenodeProtocol
+  public long journalSize(NamenodeRegistration registration)
+  throws IOException {
+    verifyRequest(registration);
+    return namesystem.getEditLogSize();
+  }
+
+  /*
+   * Active name-node cannot journal.
+   */
+  @Override // NamenodeProtocol
+  public void journal(NamenodeRegistration registration,
+                      int jAction,
+                      int length,
+                      byte[] args) throws IOException {
+    throw new UnsupportedActionException("journal");
+  }
+
+  /////////////////////////////////////////////////////
+  // ClientProtocol
+  /////////////////////////////////////////////////////
+  /** {@inheritDoc} */
+  public LocatedBlocks   getBlockLocations(String src, 
+                                          long offset, 
+                                          long length) throws IOException {
+    myMetrics.numGetBlockLocations.inc();
+    return namesystem.getBlockLocations(getClientMachine(), 
+                                        src, offset, length);
+  }
+  
+  private static String getClientMachine() {
+    String clientMachine = Server.getRemoteAddress();
+    if (clientMachine == null) {
+      clientMachine = "";
+    }
+    return clientMachine;
+  }
+
+  /** {@inheritDoc} */
+  public void create(String src, 
+                     FsPermission masked,
+                             String clientName, 
+                             boolean overwrite,
+                             short replication,
+                             long blockSize
+                             ) throws IOException {
+    String clientMachine = getClientMachine();
+    if (stateChangeLog.isDebugEnabled()) {
+      stateChangeLog.debug("*DIR* NameNode.create: file "
+                         +src+" for "+clientName+" at "+clientMachine);
+    }
+    if (!checkPathLength(src)) {
+      throw new IOException("create: Pathname too long.  Limit " 
+                            + MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
+    }
+    namesystem.startFile(src,
+        new PermissionStatus(UserGroupInformation.getCurrentUGI().getUserName(),
+            null, masked),
+        clientName, clientMachine, overwrite, replication, blockSize);
+    myMetrics.numFilesCreated.inc();
+    myMetrics.numCreateFileOps.inc();
+  }
+
+  /** {@inheritDoc} */
+  public LocatedBlock append(String src, String clientName) throws IOException {
+    String clientMachine = getClientMachine();
+    if (stateChangeLog.isDebugEnabled()) {
+      stateChangeLog.debug("*DIR* NameNode.append: file "
+          +src+" for "+clientName+" at "+clientMachine);
+    }
+    LocatedBlock info = namesystem.appendFile(src, clientName, clientMachine);
+    myMetrics.numFilesAppended.inc();
+    return info;
+  }
+
+  /** {@inheritDoc} */
+  public boolean setReplication(String src, 
+                                short replication
+                                ) throws IOException {
+    return namesystem.setReplication(src, replication);
+  }
+    
+  /** {@inheritDoc} */
+  public void setPermission(String src, FsPermission permissions
+      ) throws IOException {
+    namesystem.setPermission(src, permissions);
+  }
+
+  /** {@inheritDoc} */
+  public void setOwner(String src, String username, String groupname
+      ) throws IOException {
+    namesystem.setOwner(src, username, groupname);
+  }
+
+  /**
+   */
+  public LocatedBlock addBlock(String src, 
+                               String clientName) throws IOException {
+    stateChangeLog.debug("*BLOCK* NameNode.addBlock: file "
+                         +src+" for "+clientName);
+    LocatedBlock locatedBlock = namesystem.getAdditionalBlock(src, clientName);
+    if (locatedBlock != null)
+      myMetrics.numAddBlockOps.inc();
+    return locatedBlock;
+  }
+
+  /**
+   * The client needs to give up on the block.
+   */
+  public void abandonBlock(Block b, String src, String holder
+      ) throws IOException {
+    stateChangeLog.debug("*BLOCK* NameNode.abandonBlock: "
+                         +b+" of file "+src);
+    if (!namesystem.abandonBlock(b, src, holder)) {
+      throw new IOException("Cannot abandon block during write to " + src);
+    }
+  }
+
+  /** {@inheritDoc} */
+  public boolean complete(String src, String clientName) throws IOException {
+    stateChangeLog.debug("*DIR* NameNode.complete: " + src + " for " + clientName);
+    CompleteFileStatus returnCode = namesystem.completeFile(src, clientName);
+    if (returnCode == CompleteFileStatus.STILL_WAITING) {
+      return false;
+    } else if (returnCode == CompleteFileStatus.COMPLETE_SUCCESS) {
+      return true;
+    } else {
+      throw new IOException("Could not complete write to file " + src + " by " + clientName);
+    }
+  }
+
+  /**
+   * The client has detected an error on the specified located blocks 
+   * and is reporting them to the server.  For now, the namenode will 
+   * mark the block as corrupt.  In the future we might 
+   * check the blocks are actually corrupt. 
+   */
+  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+    stateChangeLog.info("*DIR* NameNode.reportBadBlocks");
+    for (int i = 0; i < blocks.length; i++) {
+      Block blk = blocks[i].getBlock();
+      DatanodeInfo[] nodes = blocks[i].getLocations();
+      for (int j = 0; j < nodes.length; j++) {
+        DatanodeInfo dn = nodes[j];
+        namesystem.markBlockAsCorrupt(blk, dn);
+      }
+    }
+  }
+
+  /** {@inheritDoc} */
+  public long nextGenerationStamp(Block block) throws IOException{
+    return namesystem.nextGenerationStampForBlock(block);
+  }
+
+  /** {@inheritDoc} */
+  public void commitBlockSynchronization(Block block,
+      long newgenerationstamp, long newlength,
+      boolean closeFile, boolean deleteblock, DatanodeID[] newtargets
+      ) throws IOException {
+    namesystem.commitBlockSynchronization(block,
+        newgenerationstamp, newlength, closeFile, deleteblock, newtargets);
+  }
+  
+  public long getPreferredBlockSize(String filename) throws IOException {
+    return namesystem.getPreferredBlockSize(filename);
+  }
+    
+  /**
+   */
+  public boolean rename(String src, String dst) throws IOException {
+    stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst);
+    if (!checkPathLength(dst)) {
+      throw new IOException("rename: Pathname too long.  Limit " 
+                            + MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
+    }
+    boolean ret = namesystem.renameTo(src, dst);
+    if (ret) {
+      myMetrics.numFilesRenamed.inc();
+    }
+    return ret;
+  }
+
+  /**
+   */
+  @Deprecated
+  public boolean delete(String src) throws IOException {
+    return delete(src, true);
+  }
+
+  /** {@inheritDoc} */
+  public boolean delete(String src, boolean recursive) throws IOException {
+    if (stateChangeLog.isDebugEnabled()) {
+      stateChangeLog.debug("*DIR* Namenode.delete: src=" + src
+          + ", recursive=" + recursive);
+    }
+    boolean ret = namesystem.delete(src, recursive);
+    if (ret) 
+      myMetrics.numDeleteFileOps.inc();
+    return ret;
+  }
+
+  /**
+   * Check path length does not exceed maximum.  Returns true if
+   * length and depth are okay.  Returns false if length is too long 
+   * or depth is too great.
+   * 
+   */
+  private boolean checkPathLength(String src) {
+    Path srcPath = new Path(src);
+    return (src.length() <= MAX_PATH_LENGTH &&
+            srcPath.depth() <= MAX_PATH_DEPTH);
+  }
+    
+  /** {@inheritDoc} */
+  public boolean mkdirs(String src, FsPermission masked) throws IOException {
+    stateChangeLog.debug("*DIR* NameNode.mkdirs: " + src);
+    if (!checkPathLength(src)) {
+      throw new IOException("mkdirs: Pathname too long.  Limit " 
+                            + MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
+    }
+    return namesystem.mkdirs(src,
+        new PermissionStatus(UserGroupInformation.getCurrentUGI().getUserName(),
+            null, masked));
+  }
+
+  /**
+   */
+  public void renewLease(String clientName) throws IOException {
+    namesystem.renewLease(clientName);        
+  }
+
+  /**
+   */
+  public FileStatus[] getListing(String src) throws IOException {
+    FileStatus[] files = namesystem.getListing(src);
+    if (files != null) {
+      myMetrics.numGetListingOps.inc();
+    }
+    return files;
+  }
+
+  /**
+   * Get the file info for a specific file.
+   * @param src The string representation of the path to the file
+   * @throws IOException if permission to access file is denied by the system
+   * @return object containing information regarding the file
+   *         or null if file not found
+   */
+  public FileStatus getFileInfo(String src)  throws IOException {
+    myMetrics.numFileInfoOps.inc();
+    return namesystem.getFileInfo(src);
+  }
+
+  /** @inheritDoc */
+  public long[] getStats() {
+    return namesystem.getStats();
+  }
+
+  /**
+   */
+  public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
+  throws IOException {
+    DatanodeInfo results[] = namesystem.datanodeReport(type);
+    if (results == null ) {
+      throw new IOException("Cannot find datanode report");
+    }
+    return results;
+  }
+    
+  /**
+   * @inheritDoc
+   */
+  public boolean setSafeMode(SafeModeAction action) throws IOException {
+    return namesystem.setSafeMode(action);
+  }
+
+  /**
+   * Is the cluster currently in safe mode?
+   */
+  public boolean isInSafeMode() {
+    return namesystem.isInSafeMode();
+  }
+
+  /**
+   * @throws AccessControlException 
+   * @inheritDoc
+   */
+  public boolean restoreFailedStorage(String arg) throws AccessControlException {
+    return namesystem.restoreFailedStorage(arg);
+  }
+
+  /**
+   * @inheritDoc
+   */
+  public void saveNamespace() throws IOException {
+    namesystem.saveNamespace();
+  }
+
+  /**
+   * Refresh the list of datanodes that the namenode should allow to  
+   * connect.  Re-reads conf by creating new Configuration object and 
+   * uses the files list in the configuration to update the list. 
+   */
+  public void refreshNodes() throws IOException {
+    namesystem.refreshNodes(new Configuration());
+  }
+
+  /**
+   * Returns the size of the current edit log.
+   */
+  public long getEditLogSize() throws IOException {
+    return namesystem.getEditLogSize();
+  }
+
+  /**
+   * Roll the edit log.
+   */
+  public CheckpointSignature rollEditLog() throws IOException {
+    return namesystem.rollEditLog();
+  }
+
+  /**
+   * Roll the image 
+   */
+  public void rollFsImage() throws IOException {
+    namesystem.rollFSImage();
+  }
+    
+  public void finalizeUpgrade() throws IOException {
+    namesystem.finalizeUpgrade();
+  }
+
+  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action
+                                                        ) throws IOException {
+    return namesystem.distributedUpgradeProgress(action);
+  }
+
+  /**
+   * Dumps namenode state into specified file
+   */
+  public void metaSave(String filename) throws IOException {
+    namesystem.metaSave(filename);
+  }
+
+  /** {@inheritDoc} */
+  public ContentSummary getContentSummary(String path) throws IOException {
+    return namesystem.getContentSummary(path);
+  }
+
+  /** {@inheritDoc} */
+  public void setQuota(String path, long namespaceQuota, long diskspaceQuota) 
+                       throws IOException {
+    namesystem.setQuota(path, namespaceQuota, diskspaceQuota);
+  }
+  
+  /** {@inheritDoc} */
+  public void fsync(String src, String clientName) throws IOException {
+    namesystem.fsync(src, clientName);
+  }
+
+  /** @inheritDoc */
+  public void setTimes(String src, long mtime, long atime) throws IOException {
+    namesystem.setTimes(src, mtime, atime);
+  }
+
+  ////////////////////////////////////////////////////////////////
+  // DatanodeProtocol
+  ////////////////////////////////////////////////////////////////
+  /** 
+   */
+  public DatanodeRegistration register(DatanodeRegistration nodeReg
+                                       ) throws IOException {
+    verifyVersion(nodeReg.getVersion());
+    namesystem.registerDatanode(nodeReg);
+      
+    return nodeReg;
+  }
+
+  /**
+   * Data node notify the name node that it is alive 
+   * Return an array of block-oriented commands for the datanode to execute.
+   * This will be either a transfer or a delete operation.
+   */
+  public DatanodeCommand[] sendHeartbeat(DatanodeRegistration nodeReg,
+                                       long capacity,
+                                       long dfsUsed,
+                                       long remaining,
+                                       int xmitsInProgress,
+                                       int xceiverCount) throws IOException {
+    verifyRequest(nodeReg);
+    return namesystem.handleHeartbeat(nodeReg, capacity, dfsUsed, remaining,
+        xceiverCount, xmitsInProgress);
+  }
+
+  public DatanodeCommand blockReport(DatanodeRegistration nodeReg,
+                                     long[] blocks) throws IOException {
+    verifyRequest(nodeReg);
+    BlockListAsLongs blist = new BlockListAsLongs(blocks);
+    stateChangeLog.debug("*BLOCK* NameNode.blockReport: "
+           +"from "+nodeReg.getName()+" "+blist.getNumberOfBlocks() +" blocks");
+
+    namesystem.processReport(nodeReg, blist);
+    if (getFSImage().isUpgradeFinalized())
+      return DatanodeCommand.FINALIZE;
+    return null;
+  }
+
+  public void blockReceived(DatanodeRegistration nodeReg, 
+                            Block blocks[],
+                            String delHints[]) throws IOException {
+    verifyRequest(nodeReg);
+    stateChangeLog.debug("*BLOCK* NameNode.blockReceived: "
+                         +"from "+nodeReg.getName()+" "+blocks.length+" blocks.");
+    for (int i = 0; i < blocks.length; i++) {
+      namesystem.blockReceived(nodeReg, blocks[i], delHints[i]);
+    }
+  }
+
+  /**
+   */
+  public void errorReport(DatanodeRegistration nodeReg,
+                          int errorCode, 
+                          String msg) throws IOException {
+    // Log error message from datanode
+    String dnName = (nodeReg == null ? "unknown DataNode" : nodeReg.getName());
+    LOG.info("Error report from " + dnName + ": " + msg);
+    if (errorCode == DatanodeProtocol.NOTIFY) {
+      return;
+    }
+    verifyRequest(nodeReg);
+    if (errorCode == DatanodeProtocol.DISK_ERROR) {
+      namesystem.removeDatanode(nodeReg);            
+    }
+  }
+    
+  public NamespaceInfo versionRequest() throws IOException {
+    return namesystem.getNamespaceInfo();
+  }
+
+  public UpgradeCommand processUpgradeCommand(UpgradeCommand comm) throws IOException {
+    return namesystem.processDistributedUpgradeCommand(comm);
+  }
+
+  /** 
+   * Verify request.
+   * 
+   * Verifies correctness of the datanode version, registration ID, and 
+   * if the datanode does not need to be shutdown.
+   * 
+   * @param nodeReg data node registration
+   * @throws IOException
+   */
+  public void verifyRequest(NodeRegistration nodeReg) throws IOException {
+    verifyVersion(nodeReg.getVersion());
+    if (!namesystem.getRegistrationID().equals(nodeReg.getRegistrationID()))
+      throw new UnregisteredNodeException(nodeReg);
+  }
+    
+  /**
+   * Verify version.
+   * 
+   * @param version
+   * @throws IOException
+   */
+  public void verifyVersion(int version) throws IOException {
+    if (version != LAYOUT_VERSION)
+      throw new IncorrectVersionException(version, "data node");
+  }
+
+  /**
+   * Returns the name of the fsImage file
+   */
+  public File getFsImageName() throws IOException {
+    return getFSImage().getFsImageName();
+  }
+    
+  public FSImage getFSImage() {
+    return namesystem.dir.fsImage;
+  }
+
+  /**
+   * Returns the name of the fsImage file uploaded by periodic
+   * checkpointing
+   */
+  public File[] getFsImageNameCheckpoint() throws IOException {
+    return getFSImage().getFsImageNameCheckpoint();
+  }
+
+  /**
+   * Returns the address on which the NameNodes is listening to.
+   * @return the address on which the NameNodes is listening to.
+   */
+  public InetSocketAddress getNameNodeAddress() {
+    return rpcAddress;
+  }
+
+  /**
+   * Returns the address of the NameNodes http server, 
+   * which is used to access the name-node web UI.
+   * 
+   * @return the http address.
+   */
+  public InetSocketAddress getHttpAddress() {
+    return httpAddress;
+  }
+
+  NetworkTopology getNetworkTopology() {
+    return this.namesystem.clusterMap;
+  }
+
+  /**
+   * Verify that configured directories exist, then
+   * Interactively confirm that formatting is desired 
+   * for each existing directory and format them.
+   * 
+   * @param conf
+   * @param isConfirmationNeeded
+   * @return true if formatting was aborted, false otherwise
+   * @throws IOException
+   */
+  private static boolean format(Configuration conf,
+                                boolean isConfirmationNeeded
+                                ) throws IOException {
+    Collection<File> dirsToFormat = FSNamesystem.getNamespaceDirs(conf);
+    Collection<File> editDirsToFormat = 
+                 FSNamesystem.getNamespaceEditsDirs(conf);
+    for(Iterator<File> it = dirsToFormat.iterator(); it.hasNext();) {
+      File curDir = it.next();
+      if (!curDir.exists())
+        continue;
+      if (isConfirmationNeeded) {
+        System.err.print("Re-format filesystem in " + curDir +" ? (Y or N) ");
+        if (!(System.in.read() == 'Y')) {
+          System.err.println("Format aborted in "+ curDir);
+          return true;
+        }
+        while(System.in.read() != '\n'); // discard the enter-key
+      }
+    }
+
+    FSNamesystem nsys = new FSNamesystem(new FSImage(dirsToFormat,
+                                         editDirsToFormat), conf);
+    nsys.dir.fsImage.format();
+    return false;
+  }
+
+  private static boolean finalize(Configuration conf,
+                               boolean isConfirmationNeeded
+                               ) throws IOException {
+    Collection<File> dirsToFormat = FSNamesystem.getNamespaceDirs(conf);
+    Collection<File> editDirsToFormat = 
+                               FSNamesystem.getNamespaceEditsDirs(conf);
+    FSNamesystem nsys = new FSNamesystem(new FSImage(dirsToFormat,
+                                         editDirsToFormat), conf);
+    System.err.print(
+        "\"finalize\" will remove the previous state of the files system.\n"
+        + "Recent upgrade will become permanent.\n"
+        + "Rollback option will not be available anymore.\n");
+    if (isConfirmationNeeded) {
+      System.err.print("Finalize filesystem state ? (Y or N) ");
+      if (!(System.in.read() == 'Y')) {
+        System.err.println("Finalize aborted.");
+        return true;
+      }
+      while(System.in.read() != '\n'); // discard the enter-key
+    }
+    nsys.dir.fsImage.finalizeUpgrade();
+    return false;
+  }
+
+  @Override
+  public void refreshServiceAcl() throws IOException {
+    if (!serviceAuthEnabled) {
+      throw new AuthorizationException("Service Level Authorization not enabled!");
+    }
+
+    SecurityUtil.getPolicy().refresh();
+  }
+
+  private static void printUsage() {
+    System.err.println(
+      "Usage: java NameNode [" +
+      StartupOption.BACKUP.getName() + "] | [" +
+      StartupOption.CHECKPOINT.getName() + "] | [" +
+      StartupOption.FORMAT.getName() + "] | [" +
+      StartupOption.UPGRADE.getName() + "] | [" +
+      StartupOption.ROLLBACK.getName() + "] | [" +
+      StartupOption.FINALIZE.getName() + "] | [" +
+      StartupOption.IMPORT.getName() + "]");
+  }
+
+  private static StartupOption parseArguments(String args[]) {
+    int argsLen = (args == null) ? 0 : args.length;
+    StartupOption startOpt = StartupOption.REGULAR;
+    for(int i=0; i < argsLen; i++) {
+      String cmd = args[i];
+      if (StartupOption.FORMAT.getName().equalsIgnoreCase(cmd)) {
+        startOpt = StartupOption.FORMAT;
+      } else if (StartupOption.REGULAR.getName().equalsIgnoreCase(cmd)) {
+        startOpt = StartupOption.REGULAR;
+      } else if (StartupOption.BACKUP.getName().equalsIgnoreCase(cmd)) {
+        startOpt = StartupOption.BACKUP;
+      } else if (StartupOption.CHECKPOINT.getName().equalsIgnoreCase(cmd)) {
+        startOpt = StartupOption.CHECKPOINT;
+      } else if (StartupOption.UPGRADE.getName().equalsIgnoreCase(cmd)) {
+        startOpt = StartupOption.UPGRADE;
+      } else if (StartupOption.ROLLBACK.getName().equalsIgnoreCase(cmd)) {
+        startOpt = StartupOption.ROLLBACK;
+      } else if (StartupOption.FINALIZE.getName().equalsIgnoreCase(cmd)) {
+        startOpt = StartupOption.FINALIZE;
+      } else if (StartupOption.IMPORT.getName().equalsIgnoreCase(cmd)) {
+        startOpt = StartupOption.IMPORT;
+      } else
+        return null;
+    }
+    return startOpt;
+  }
+
+  private static void setStartupOption(Configuration conf, StartupOption opt) {
+    conf.set("dfs.namenode.startup", opt.toString());
+  }
+
+  static StartupOption getStartupOption(Configuration conf) {
+    return StartupOption.valueOf(conf.get("dfs.namenode.startup",
+                                          StartupOption.REGULAR.toString()));
+  }
+
+  public static NameNode createNameNode(String argv[], 
+                                 Configuration conf) throws IOException {
+    if (conf == null)
+      conf = new Configuration();
+    StartupOption startOpt = parseArguments(argv);
+    if (startOpt == null) {
+      printUsage();
+      return null;
+    }
+    setStartupOption(conf, startOpt);
+
+    switch (startOpt) {
+      case FORMAT:
+        boolean aborted = format(conf, true);
+        System.exit(aborted ? 1 : 0);
+      case FINALIZE:
+        aborted = finalize(conf, true);
+        System.exit(aborted ? 1 : 0);
+      case BACKUP:
+      case CHECKPOINT:
+        return new BackupNode(conf, startOpt.toNodeRole());
+      default:
+        return new NameNode(conf);
+    }
+  }
+    
+  /**
+   */
+  public static void main(String argv[]) throws Exception {
+    try {
+      StringUtils.startupShutdownMessage(NameNode.class, argv, LOG);
+      NameNode namenode = createNameNode(argv, null);
+      if (namenode != null)
+        namenode.join();
+    } catch (Throwable e) {
+      LOG.error(StringUtils.stringifyException(e));
+      System.exit(-1);
+    }
+  }
+}

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

@@ -0,0 +1,618 @@
+/**
+ * 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 java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.TreeSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+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;
+
+/**
+ * This class provides rudimentary checking of DFS volumes for errors and
+ * sub-optimal conditions.
+ * <p>The tool scans all files and directories, starting from an indicated
+ *  root path. The following abnormal conditions are detected and handled:</p>
+ * <ul>
+ * <li>files with blocks that are completely missing from all datanodes.<br/>
+ * In this case the tool can perform one of the following actions:
+ *  <ul>
+ *      <li>none ({@link #FIXING_NONE})</li>
+ *      <li>move corrupted files to /lost+found directory on DFS
+ *      ({@link #FIXING_MOVE}). Remaining data blocks are saved as a
+ *      block chains, representing longest consecutive series of valid blocks.</li>
+ *      <li>delete corrupted files ({@link #FIXING_DELETE})</li>
+ *  </ul>
+ *  </li>
+ *  <li>detect files with under-replicated or over-replicated blocks</li>
+ *  </ul>
+ *  Additionally, the tool collects a detailed overall DFS statistics, and
+ *  optionally can print detailed statistics on block locations and replication
+ *  factors of each file.
+ */
+public class NamenodeFsck {
+  public static final Log LOG = LogFactory.getLog(NameNode.class.getName());
+  
+  // return string marking fsck status
+  public static final String CORRUPT_STATUS = "is CORRUPT";
+  public static final String HEALTHY_STATUS = "is HEALTHY";
+  public static final String NONEXISTENT_STATUS = "does not exist";
+  public static final String FAILURE_STATUS = "FAILED";
+  
+  /** Don't attempt any fixing . */
+  public static final int FIXING_NONE = 0;
+  /** Move corrupted files to /lost+found . */
+  public static final int FIXING_MOVE = 1;
+  /** Delete corrupted files. */
+  public static final int FIXING_DELETE = 2;
+  
+  private final ClientProtocol namenode;
+  private final NetworkTopology networktopology;
+  private final int totalDatanodes;
+  private final short minReplication;
+
+  private String lostFound = null;
+  private boolean lfInited = false;
+  private boolean lfInitedOk = false;
+  private boolean showFiles = false;
+  private boolean showOpenFiles = false;
+  private boolean showBlocks = false;
+  private boolean showLocations = false;
+  private boolean showRacks = false;
+  private int fixing = FIXING_NONE;
+  private String path = "/";
+  
+  private final Configuration conf;
+  private final PrintWriter out;
+  
+  /**
+   * Filesystem checker.
+   * @param conf configuration (namenode config)
+   * @param nn namenode that this fsck is going to use
+   * @param pmap key=value[] map that is passed to the http servlet as url parameters
+   * @param response the object into which  this servelet writes the url contents
+   * @throws IOException
+   */
+  NamenodeFsck(Configuration conf, ClientProtocol namenode,
+      NetworkTopology networktopology, 
+      Map<String,String[]> pmap, PrintWriter out,
+      int totalDatanodes, short minReplication) {
+    this.conf = conf;
+    this.namenode = namenode;
+    this.networktopology = networktopology;
+    this.out = out;
+    this.totalDatanodes = totalDatanodes;
+    this.minReplication = minReplication;
+
+    for (Iterator<String> it = pmap.keySet().iterator(); it.hasNext();) {
+      String key = it.next();
+      if (key.equals("path")) { this.path = pmap.get("path")[0]; }
+      else if (key.equals("move")) { this.fixing = FIXING_MOVE; }
+      else if (key.equals("delete")) { this.fixing = FIXING_DELETE; }
+      else if (key.equals("files")) { this.showFiles = true; }
+      else if (key.equals("blocks")) { this.showBlocks = true; }
+      else if (key.equals("locations")) { this.showLocations = true; }
+      else if (key.equals("racks")) { this.showRacks = true; }
+      else if (key.equals("openforwrite")) {this.showOpenFiles = true; }
+    }
+  }
+  
+  /**
+   * Check files on DFS, starting from the indicated path.
+   */
+  public void fsck() {
+    try {
+      Result res = new Result(conf);
+
+      final FileStatus[] files = namenode.getListing(path);
+      if (files != null) {
+        for (int i = 0; i < files.length; i++) {
+          check(files[i], res);
+        }
+        out.println(res);
+        out.println(" Number of data-nodes:\t\t" + totalDatanodes);
+        out.println(" Number of racks:\t\t" + networktopology.getNumOfRacks());
+
+        // DFSck client scans for the string HEALTHY/CORRUPT to check the status
+        // of file system and return appropriate code. Changing the output string
+        // might break testcases. 
+        if (res.isHealthy()) {
+          out.print("\n\nThe filesystem under path '" + path + "' " + HEALTHY_STATUS);
+        }  else {
+          out.print("\n\nThe filesystem under path '" + path + "' " + CORRUPT_STATUS);
+        }
+      } else {
+        out.print("\n\nPath '" + path + "' " + NONEXISTENT_STATUS);
+      }
+    } catch (Exception e) {
+      String errMsg = "Fsck on path '" + path + "' " + FAILURE_STATUS;
+      LOG.warn(errMsg, e);
+      out.println(e.getMessage());
+      out.print("\n\n"+errMsg);
+    } finally {
+      out.close();
+    }
+  }
+  
+  private void check(FileStatus file, Result res) throws IOException {
+    String path = file.getPath().toString();
+    boolean isOpen = false;
+
+    if (file.isDir()) {
+      final FileStatus[] files = namenode.getListing(path);
+      if (files == null) {
+        return;
+      }
+      if (showFiles) {
+        out.println(path + " <dir>");
+      }
+      res.totalDirs++;
+      for (int i = 0; i < files.length; i++) {
+        check(files[i], res);
+      }
+      return;
+    }
+    long fileLen = file.getLen();
+    LocatedBlocks blocks = namenode.getBlockLocations(path, 0, fileLen);
+    if (blocks == null) { // the file is deleted
+      return;
+    }
+    isOpen = blocks.isUnderConstruction();
+    if (isOpen && !showOpenFiles) {
+      // We collect these stats about open files to report with default options
+      res.totalOpenFilesSize += fileLen;
+      res.totalOpenFilesBlocks += blocks.locatedBlockCount();
+      res.totalOpenFiles++;
+      return;
+    }
+    res.totalFiles++;
+    res.totalSize += fileLen;
+    res.totalBlocks += blocks.locatedBlockCount();
+    if (showOpenFiles && isOpen) {
+      out.print(path + " " + fileLen + " bytes, " +
+        blocks.locatedBlockCount() + " block(s), OPENFORWRITE: ");
+    } else if (showFiles) {
+      out.print(path + " " + fileLen + " bytes, " +
+        blocks.locatedBlockCount() + " block(s): ");
+    } else {
+      out.print('.');
+    }
+    if (res.totalFiles % 100 == 0) { out.println(); out.flush(); }
+    int missing = 0;
+    int corrupt = 0;
+    long missize = 0;
+    int underReplicatedPerFile = 0;
+    int misReplicatedPerFile = 0;
+    StringBuffer report = new StringBuffer();
+    int i = 0;
+    for (LocatedBlock lBlk : blocks.getLocatedBlocks()) {
+      Block block = lBlk.getBlock();
+      boolean isCorrupt = lBlk.isCorrupt();
+      String blkName = block.toString();
+      DatanodeInfo[] locs = lBlk.getLocations();
+      res.totalReplicas += locs.length;
+      short targetFileReplication = file.getReplication();
+      if (locs.length > targetFileReplication) {
+        res.excessiveReplicas += (locs.length - targetFileReplication);
+        res.numOverReplicatedBlocks += 1;
+      }
+      // Check if block is Corrupt
+      if (isCorrupt) {
+        corrupt++;
+        res.corruptBlocks++;
+        out.print("\n" + path + ": CORRUPT block " + block.getBlockName()+"\n");
+      }
+      if (locs.length >= minReplication)
+        res.numMinReplicatedBlocks++;
+      if (locs.length < targetFileReplication && locs.length > 0) {
+        res.missingReplicas += (targetFileReplication - locs.length);
+        res.numUnderReplicatedBlocks += 1;
+        underReplicatedPerFile++;
+        if (!showFiles) {
+          out.print("\n" + path + ": ");
+        }
+        out.println(" Under replicated " + block +
+                    ". Target Replicas is " +
+                    targetFileReplication + " but found " +
+                    locs.length + " replica(s).");
+      }
+      // verify block placement policy
+      int missingRacks = ReplicationTargetChooser.verifyBlockPlacement(
+                    lBlk, targetFileReplication, networktopology);
+      if (missingRacks > 0) {
+        res.numMisReplicatedBlocks++;
+        misReplicatedPerFile++;
+        if (!showFiles) {
+          if(underReplicatedPerFile == 0)
+            out.println();
+          out.print(path + ": ");
+        }
+        out.println(" Replica placement policy is violated for " + 
+                    block +
+                    ". Block should be additionally replicated on " + 
+                    missingRacks + " more rack(s).");
+      }
+      report.append(i + ". " + blkName + " len=" + block.getNumBytes());
+      if (locs.length == 0) {
+        report.append(" MISSING!");
+        res.addMissing(block.toString(), block.getNumBytes());
+        missing++;
+        missize += block.getNumBytes();
+      } else {
+        report.append(" repl=" + locs.length);
+        if (showLocations || showRacks) {
+          StringBuffer sb = new StringBuffer("[");
+          for (int j = 0; j < locs.length; j++) {
+            if (j > 0) { sb.append(", "); }
+            if (showRacks)
+              sb.append(NodeBase.getPath(locs[j]));
+            else
+              sb.append(locs[j]);
+          }
+          sb.append(']');
+          report.append(" " + sb.toString());
+        }
+      }
+      report.append('\n');
+      i++;
+    }
+    if ((missing > 0) || (corrupt > 0)) {
+      if (!showFiles && (missing > 0)) {
+        out.print("\n" + path + ": MISSING " + missing
+            + " blocks of total size " + missize + " B.");
+      }
+      res.corruptFiles++;
+      switch(fixing) {
+      case FIXING_NONE:
+        break;
+      case FIXING_MOVE:
+        if (!isOpen)
+          lostFoundMove(file, blocks);
+        break;
+      case FIXING_DELETE:
+        if (!isOpen)
+          namenode.delete(path, true);
+      }
+    }
+    if (showFiles) {
+      if (missing > 0) {
+        out.print(" MISSING " + missing + " blocks of total size " + missize + " B\n");
+      }  else if (underReplicatedPerFile == 0 && misReplicatedPerFile == 0) {
+        out.print(" OK\n");
+      }
+      if (showBlocks) {
+        out.print(report.toString() + "\n");
+      }
+    }
+  }
+  
+  private void lostFoundMove(FileStatus file, LocatedBlocks blocks)
+    throws IOException {
+    DFSClient dfs = new DFSClient(conf);
+    try {
+    if (!lfInited) {
+      lostFoundInit(dfs);
+    }
+    if (!lfInitedOk) {
+      return;
+    }
+    String target = lostFound + file.getPath();
+    String errmsg = "Failed to move " + file.getPath() + " to /lost+found";
+    try {
+      if (!namenode.mkdirs(target, file.getPermission())) {
+        LOG.warn(errmsg);
+        return;
+      }
+      // create chains
+      int chain = 0;
+      OutputStream fos = null;
+      for (LocatedBlock lBlk : blocks.getLocatedBlocks()) {
+        LocatedBlock lblock = lBlk;
+        DatanodeInfo[] locs = lblock.getLocations();
+        if (locs == null || locs.length == 0) {
+          if (fos != null) {
+            fos.flush();
+            fos.close();
+            fos = null;
+          }
+          continue;
+        }
+        if (fos == null) {
+          fos = dfs.create(target + "/" + chain, true);
+          if (fos != null) chain++;
+          else {
+            LOG.warn(errmsg + ": could not store chain " + chain);
+            // perhaps we should bail out here...
+            // return;
+            continue;
+          }
+        }
+        
+        // copy the block. It's a pity it's not abstracted from DFSInputStream ...
+        try {
+          copyBlock(dfs, lblock, fos);
+        } catch (Exception e) {
+          e.printStackTrace();
+          // something went wrong copying this block...
+          LOG.warn(" - could not copy block " + lblock.getBlock() + " to " + target);
+          fos.flush();
+          fos.close();
+          fos = null;
+        }
+      }
+      if (fos != null) fos.close();
+      LOG.warn("\n - moved corrupted file " + file.getPath() + " to /lost+found");
+      dfs.delete(file.getPath().toString(), true);
+    }  catch (Exception e) {
+      e.printStackTrace();
+      LOG.warn(errmsg + ": " + e.getMessage());
+    }
+    } finally {
+      dfs.close();
+    }
+  }
+      
+  /*
+   * XXX (ab) Bulk of this method is copied verbatim from {@link DFSClient}, which is
+   * bad. Both places should be refactored to provide a method to copy blocks
+   * around.
+   */
+  private void copyBlock(DFSClient dfs, LocatedBlock lblock,
+                         OutputStream fos) throws Exception {
+    int failures = 0;
+    InetSocketAddress targetAddr = null;
+    TreeSet<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>();
+    Socket s = null;
+    DFSClient.BlockReader blockReader = null; 
+    Block block = lblock.getBlock(); 
+
+    while (s == null) {
+      DatanodeInfo chosenNode;
+      
+      try {
+        chosenNode = bestNode(dfs, lblock.getLocations(), deadNodes);
+        targetAddr = NetUtils.createSocketAddr(chosenNode.getName());
+      }  catch (IOException ie) {
+        if (failures >= DFSClient.MAX_BLOCK_ACQUIRE_FAILURES) {
+          throw new IOException("Could not obtain block " + lblock);
+        }
+        LOG.info("Could not obtain block from any node:  " + ie);
+        try {
+          Thread.sleep(10000);
+        }  catch (InterruptedException iex) {
+        }
+        deadNodes.clear();
+        failures++;
+        continue;
+      }
+      try {
+        s = new Socket();
+        s.connect(targetAddr, HdfsConstants.READ_TIMEOUT);
+        s.setSoTimeout(HdfsConstants.READ_TIMEOUT);
+        
+        blockReader = 
+          DFSClient.BlockReader.newBlockReader(s, targetAddr.toString() + ":" + 
+                                               block.getBlockId(), 
+                                               block.getBlockId(), 
+                                               lblock.getAccessToken(),
+                                               block.getGenerationStamp(), 
+                                               0, -1,
+                                               conf.getInt("io.file.buffer.size", 4096));
+        
+      }  catch (IOException ex) {
+        // Put chosen node into dead list, continue
+        LOG.info("Failed to connect to " + targetAddr + ":" + ex);
+        deadNodes.add(chosenNode);
+        if (s != null) {
+          try {
+            s.close();
+          } catch (IOException iex) {
+          }
+        }
+        s = null;
+      }
+    }
+    if (blockReader == null) {
+      throw new Exception("Could not open data stream for " + lblock.getBlock());
+    }
+    byte[] buf = new byte[1024];
+    int cnt = 0;
+    boolean success = true;
+    long bytesRead = 0;
+    try {
+      while ((cnt = blockReader.read(buf, 0, buf.length)) > 0) {
+        fos.write(buf, 0, cnt);
+        bytesRead += cnt;
+      }
+      if ( bytesRead != block.getNumBytes() ) {
+        throw new IOException("Recorded block size is " + block.getNumBytes() + 
+                              ", but datanode returned " +bytesRead+" bytes");
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+      success = false;
+    } finally {
+      try {s.close(); } catch (Exception e1) {}
+    }
+    if (!success)
+      throw new Exception("Could not copy block data for " + lblock.getBlock());
+  }
+      
+  /*
+   * XXX (ab) See comment above for copyBlock().
+   *
+   * Pick the best node from which to stream the data.
+   * That's the local one, if available.
+   */
+  Random r = new Random();
+  private DatanodeInfo bestNode(DFSClient dfs, DatanodeInfo[] nodes,
+                                TreeSet<DatanodeInfo> deadNodes) throws IOException {
+    if ((nodes == null) ||
+        (nodes.length - deadNodes.size() < 1)) {
+      throw new IOException("No live nodes contain current block");
+    }
+    DatanodeInfo chosenNode;
+    do {
+      chosenNode = nodes[r.nextInt(nodes.length)];
+    } while (deadNodes.contains(chosenNode));
+    return chosenNode;
+  }
+  
+  private void lostFoundInit(DFSClient dfs) {
+    lfInited = true;
+    try {
+      String lfName = "/lost+found";
+      
+      final FileStatus lfStatus = dfs.getFileInfo(lfName);
+      if (lfStatus == null) { // not exists
+        lfInitedOk = dfs.mkdirs(lfName);
+        lostFound = lfName;
+      } else if (!lfStatus.isDir()) { // exists but not a directory
+        LOG.warn("Cannot use /lost+found : a regular file with this name exists.");
+        lfInitedOk = false;
+      }  else { // exists and is a directory
+        lostFound = lfName;
+        lfInitedOk = true;
+      }
+    }  catch (Exception e) {
+      e.printStackTrace();
+      lfInitedOk = false;
+    }
+    if (lostFound == null) {
+      LOG.warn("Cannot initialize /lost+found .");
+      lfInitedOk = false;
+    }
+  }
+
+  /**
+   * FsckResult of checking, plus overall DFS statistics.
+   */
+  private static class Result {
+    private List<String> missingIds = new ArrayList<String>();
+    private long missingSize = 0L;
+    private long corruptFiles = 0L;
+    private long corruptBlocks = 0L;
+    private long excessiveReplicas = 0L;
+    private long missingReplicas = 0L;
+    private long numOverReplicatedBlocks = 0L;
+    private long numUnderReplicatedBlocks = 0L;
+    private long numMisReplicatedBlocks = 0L;  // blocks that do not satisfy block placement policy
+    private long numMinReplicatedBlocks = 0L;  // minimally replicatedblocks
+    private long totalBlocks = 0L;
+    private long totalOpenFilesBlocks = 0L;
+    private long totalFiles = 0L;
+    private long totalOpenFiles = 0L;
+    private long totalDirs = 0L;
+    private long totalSize = 0L;
+    private long totalOpenFilesSize = 0L;
+    private long totalReplicas = 0L;
+
+    final short replication;
+    
+    private Result(Configuration conf) {
+      this.replication = (short)conf.getInt("dfs.replication", 3);
+    }
+    
+    /**
+     * DFS is considered healthy if there are no missing blocks.
+     */
+    boolean isHealthy() {
+      return ((missingIds.size() == 0) && (corruptBlocks == 0));
+    }
+    
+    /** Add a missing block name, plus its size. */
+    void addMissing(String id, long size) {
+      missingIds.add(id);
+      missingSize += size;
+    }
+    
+    /** Return the actual replication factor. */
+    float getReplicationFactor() {
+      if (totalBlocks == 0)
+        return 0.0f;
+      return (float) (totalReplicas) / (float) totalBlocks;
+    }
+    
+    /** {@inheritDoc} */
+    public String toString() {
+      StringBuffer res = new StringBuffer();
+      res.append("Status: " + (isHealthy() ? "HEALTHY" : "CORRUPT"));
+      res.append("\n Total size:\t" + totalSize + " B");
+      if (totalOpenFilesSize != 0) 
+        res.append(" (Total open files size: " + totalOpenFilesSize + " B)");
+      res.append("\n Total dirs:\t" + totalDirs);
+      res.append("\n Total files:\t" + totalFiles);
+      if (totalOpenFiles != 0)
+        res.append(" (Files currently being written: " + 
+                   totalOpenFiles + ")");
+      res.append("\n Total blocks (validated):\t" + totalBlocks);
+      if (totalBlocks > 0) res.append(" (avg. block size "
+                                      + (totalSize / totalBlocks) + " B)");
+      if (totalOpenFilesBlocks != 0)
+        res.append(" (Total open file blocks (not validated): " + 
+                   totalOpenFilesBlocks + ")");
+      if (corruptFiles > 0) { 
+        res.append("\n  ********************************");
+        res.append("\n  CORRUPT FILES:\t" + corruptFiles);
+        if (missingSize > 0) {
+          res.append("\n  MISSING BLOCKS:\t" + missingIds.size());
+          res.append("\n  MISSING SIZE:\t\t" + missingSize + " B");
+        }
+        if (corruptBlocks > 0) {
+          res.append("\n  CORRUPT BLOCKS: \t" + corruptBlocks);
+        }
+        res.append("\n  ********************************");
+      }
+      res.append("\n Minimally replicated blocks:\t" + numMinReplicatedBlocks);
+      if (totalBlocks > 0)        res.append(" (" + ((float) (numMinReplicatedBlocks * 100) / (float) totalBlocks) + " %)");
+      res.append("\n Over-replicated blocks:\t" + numOverReplicatedBlocks);
+      if (totalBlocks > 0)        res.append(" (" + ((float) (numOverReplicatedBlocks * 100) / (float) totalBlocks) + " %)");
+      res.append("\n Under-replicated blocks:\t" + numUnderReplicatedBlocks);
+      if (totalBlocks > 0)        res.append(" (" + ((float) (numUnderReplicatedBlocks * 100) / (float) totalBlocks) + " %)");
+      res.append("\n Mis-replicated blocks:\t\t" + numMisReplicatedBlocks);
+      if (totalBlocks > 0)        res.append(" (" + ((float) (numMisReplicatedBlocks * 100) / (float) totalBlocks) + " %)");
+      res.append("\n Default replication factor:\t" + replication);
+      res.append("\n Average block replication:\t" + getReplicationFactor());
+      res.append("\n Corrupt blocks:\t\t" + corruptBlocks);
+      res.append("\n Missing replicas:\t\t" + missingReplicas);
+      if (totalReplicas > 0)        res.append(" (" + ((float) (missingReplicas * 100) / (float) totalReplicas) + " %)");
+      return res.toString();
+    }
+  }
+}

+ 32 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java

@@ -0,0 +1,32 @@
+/**
+ * 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 java.io.IOException;
+
+/**
+ * The file has not finished being written to enough datanodes yet.
+ */
+public class NotReplicatedYetException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public NotReplicatedYetException(String msg) {
+    super(msg);
+  }
+}

+ 249 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/PendingReplicationBlocks.java

@@ -0,0 +1,249 @@
+/**
+ * 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 org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.util.*;
+import java.io.*;
+import java.util.*;
+import java.sql.Time;
+
+/***************************************************
+ * PendingReplicationBlocks does the bookkeeping of all
+ * blocks that are getting replicated.
+ *
+ * It does the following:
+ * 1)  record blocks that are getting replicated at this instant.
+ * 2)  a coarse grain timer to track age of replication request
+ * 3)  a thread that periodically identifies replication-requests
+ *     that never made it.
+ *
+ ***************************************************/
+class PendingReplicationBlocks {
+  private Map<Block, PendingBlockInfo> pendingReplications;
+  private ArrayList<Block> timedOutItems;
+  Daemon timerThread = null;
+  private volatile boolean fsRunning = true;
+
+  //
+  // It might take anywhere between 5 to 10 minutes before
+  // a request is timed out.
+  //
+  private long timeout = 5 * 60 * 1000;
+  private long defaultRecheckInterval = 5 * 60 * 1000;
+
+  PendingReplicationBlocks(long timeoutPeriod) {
+    if ( timeoutPeriod > 0 ) {
+      this.timeout = timeoutPeriod;
+    }
+    pendingReplications = new HashMap<Block, PendingBlockInfo>();
+    timedOutItems = new ArrayList<Block>();
+  }
+
+  void start() {
+    timerThread = new Daemon(new PendingReplicationMonitor());
+    timerThread.start();
+  }
+
+  /**
+   * Add a block to the list of pending Replications
+   */
+  void add(Block block, int numReplicas) {
+    synchronized (pendingReplications) {
+      PendingBlockInfo found = pendingReplications.get(block);
+      if (found == null) {
+        pendingReplications.put(block, new PendingBlockInfo(numReplicas));
+      } else {
+        found.incrementReplicas(numReplicas);
+        found.setTimeStamp();
+      }
+    }
+  }
+
+  /**
+   * One replication request for this block has finished.
+   * Decrement the number of pending replication requests
+   * for this block.
+   */
+  void remove(Block block) {
+    synchronized (pendingReplications) {
+      PendingBlockInfo found = pendingReplications.get(block);
+      if (found != null) {
+      	FSNamesystem.LOG.debug("Removing pending replication for block" + block);
+        found.decrementReplicas();
+        if (found.getNumReplicas() <= 0) {
+          pendingReplications.remove(block);
+        }
+      }
+    }
+  }
+
+  /**
+   * The total number of blocks that are undergoing replication
+   */
+  int size() {
+    return pendingReplications.size();
+  } 
+
+  /**
+   * How many copies of this block is pending replication?
+   */
+  int getNumReplicas(Block block) {
+    synchronized (pendingReplications) {
+      PendingBlockInfo found = pendingReplications.get(block);
+      if (found != null) {
+        return found.getNumReplicas();
+      }
+    }
+    return 0;
+  }
+
+  /**
+   * Returns a list of blocks that have timed out their 
+   * replication requests. Returns null if no blocks have
+   * timed out.
+   */
+  Block[] getTimedOutBlocks() {
+    synchronized (timedOutItems) {
+      if (timedOutItems.size() <= 0) {
+        return null;
+      }
+      Block[] blockList = timedOutItems.toArray(
+                                                new Block[timedOutItems.size()]);
+      timedOutItems.clear();
+      return blockList;
+    }
+  }
+
+  /**
+   * An object that contains information about a block that 
+   * is being replicated. It records the timestamp when the 
+   * system started replicating the most recent copy of this
+   * block. It also records the number of replication
+   * requests that are in progress.
+   */
+  static class PendingBlockInfo {
+    private long timeStamp;
+    private int numReplicasInProgress;
+
+    PendingBlockInfo(int numReplicas) {
+      this.timeStamp = FSNamesystem.now();
+      this.numReplicasInProgress = numReplicas;
+    }
+
+    long getTimeStamp() {
+      return timeStamp;
+    }
+
+    void setTimeStamp() {
+      timeStamp = FSNamesystem.now();
+    }
+
+    void incrementReplicas(int increment) {
+      numReplicasInProgress += increment;
+    }
+
+    void decrementReplicas() {
+      numReplicasInProgress--;
+      assert(numReplicasInProgress >= 0);
+    }
+
+    int getNumReplicas() {
+      return numReplicasInProgress;
+    }
+  }
+
+  /*
+   * A periodic thread that scans for blocks that never finished
+   * their replication request.
+   */
+  class PendingReplicationMonitor implements Runnable {
+    public void run() {
+      while (fsRunning) {
+        long period = Math.min(defaultRecheckInterval, timeout);
+        try {
+          pendingReplicationCheck();
+          Thread.sleep(period);
+        } catch (InterruptedException ie) {
+          FSNamesystem.LOG.debug(
+                "PendingReplicationMonitor thread received exception. " + ie);
+        }
+      }
+    }
+
+    /**
+     * Iterate through all items and detect timed-out items
+     */
+    void pendingReplicationCheck() {
+      synchronized (pendingReplications) {
+        Iterator<Map.Entry<Block, PendingBlockInfo>> iter =
+                                    pendingReplications.entrySet().iterator();
+        long now = FSNamesystem.now();
+        FSNamesystem.LOG.debug("PendingReplicationMonitor checking Q");
+        while (iter.hasNext()) {
+          Map.Entry<Block, PendingBlockInfo> entry = iter.next();
+          PendingBlockInfo pendingBlock = entry.getValue();
+          if (now > pendingBlock.getTimeStamp() + timeout) {
+            Block block = entry.getKey();
+            synchronized (timedOutItems) {
+              timedOutItems.add(block);
+            }
+            FSNamesystem.LOG.warn(
+                "PendingReplicationMonitor timed out block " + block);
+            iter.remove();
+          }
+        }
+      }
+    }
+  }
+
+  /*
+   * Shuts down the pending replication monitor thread.
+   * Waits for the thread to exit.
+   */
+  void stop() {
+    fsRunning = false;
+    if(timerThread == null) return;
+    timerThread.interrupt();
+    try {
+      timerThread.join(3000);
+    } catch (InterruptedException ie) {
+    }
+  }
+
+  /**
+   * Iterate through all items and print them.
+   */
+  void metaSave(PrintWriter out) {
+    synchronized (pendingReplications) {
+      out.println("Metasave: Blocks being replicated: " +
+                  pendingReplications.size());
+      Iterator<Map.Entry<Block, PendingBlockInfo>> iter =
+                                  pendingReplications.entrySet().iterator();
+      while (iter.hasNext()) {
+        Map.Entry<Block, PendingBlockInfo> entry = iter.next();
+        PendingBlockInfo pendingBlock = entry.getValue();
+        Block block = entry.getKey();
+        out.println(block + 
+                    " StartTime: " + new Time(pendingBlock.timeStamp) +
+                    " NumReplicaInProgress: " + 
+                    pendingBlock.numReplicasInProgress);
+      }
+    }
+  }
+}

+ 514 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/ReplicationTargetChooser.java

@@ -0,0 +1,514 @@
+/**
+ * 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 org.apache.commons.logging.*;
+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.net.NetworkTopology;
+import org.apache.hadoop.net.Node;
+import org.apache.hadoop.net.NodeBase;
+import java.util.*;
+
+/** The class is responsible for choosing the desired number of targets
+ * for placing block replicas.
+ * The replica placement strategy is that if the writer is on a datanode,
+ * the 1st replica is placed on the local machine, 
+ * otherwise a random datanode. The 2nd replica is placed on a datanode
+ * that is on a different rack. The 3rd replica is placed on a datanode
+ * which is on a different node of the rack as the second replica.
+ */
+class ReplicationTargetChooser {
+  private final boolean considerLoad; 
+  private NetworkTopology clusterMap;
+  private FSNamesystem fs;
+    
+  ReplicationTargetChooser(boolean considerLoad,  FSNamesystem fs,
+                           NetworkTopology clusterMap) {
+    this.considerLoad = considerLoad;
+    this.fs = fs;
+    this.clusterMap = clusterMap;
+  }
+    
+  private static class NotEnoughReplicasException extends Exception {
+    private static final long serialVersionUID = 1L;
+
+    NotEnoughReplicasException(String msg) {
+      super(msg);
+    }
+  }
+    
+  /**
+   * choose <i>numOfReplicas</i> data nodes for <i>writer</i> to replicate
+   * a block with size <i>blocksize</i> 
+   * If not, return as many as we can.
+   * 
+   * @param numOfReplicas: number of replicas wanted.
+   * @param writer: the writer's machine, null if not in the cluster.
+   * @param excludedNodes: datanodes that should not be considered targets.
+   * @param blocksize: size of the data to be written.
+   * @return array of DatanodeDescriptor instances chosen as targets
+   * and sorted as a pipeline.
+   */
+  DatanodeDescriptor[] chooseTarget(int numOfReplicas,
+                                    DatanodeDescriptor writer,
+                                    HashMap<Node, Node> excludedNodes,
+                                    long blocksize) {
+    if (excludedNodes == null) {
+      excludedNodes = new HashMap<Node, Node>();
+    }
+      
+    return chooseTarget(numOfReplicas, writer, 
+                        new ArrayList<DatanodeDescriptor>(), excludedNodes, blocksize);
+  }
+    
+  /**
+   * choose <i>numOfReplicas</i> data nodes for <i>writer</i> 
+   * to re-replicate a block with size <i>blocksize</i> 
+   * If not, return as many as we can.
+   * 
+   * @param numOfReplicas: additional number of replicas wanted.
+   * @param writer: the writer's machine, null if not in the cluster.
+   * @param choosenNodes: datanodes that have been chosen as targets.
+   * @param excludedNodes: datanodes that should not be considered targets.
+   * @param blocksize: size of the data to be written.
+   * @return array of DatanodeDescriptor instances chosen as target 
+   * and sorted as a pipeline.
+   */
+  DatanodeDescriptor[] chooseTarget(int numOfReplicas,
+                                    DatanodeDescriptor writer,
+                                    List<DatanodeDescriptor> choosenNodes,
+                                    HashMap<Node, Node> excludedNodes,
+                                    long blocksize) {
+    if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
+      return new DatanodeDescriptor[0];
+    }
+      
+    if (excludedNodes == null) {
+      excludedNodes = new HashMap<Node, Node>();
+    }
+      
+    int clusterSize = clusterMap.getNumOfLeaves();
+    int totalNumOfReplicas = choosenNodes.size()+numOfReplicas;
+    if (totalNumOfReplicas > clusterSize) {
+      numOfReplicas -= (totalNumOfReplicas-clusterSize);
+      totalNumOfReplicas = clusterSize;
+    }
+      
+    int maxNodesPerRack = 
+      (totalNumOfReplicas-1)/clusterMap.getNumOfRacks()+2;
+      
+    List<DatanodeDescriptor> results = 
+      new ArrayList<DatanodeDescriptor>(choosenNodes);
+    for (Node node:choosenNodes) {
+      excludedNodes.put(node, node);
+    }
+      
+    if (!clusterMap.contains(writer)) {
+      writer=null;
+    }
+      
+    DatanodeDescriptor localNode = chooseTarget(numOfReplicas, writer, 
+                                                excludedNodes, blocksize, maxNodesPerRack, results);
+      
+    results.removeAll(choosenNodes);
+      
+    // sorting nodes to form a pipeline
+    return getPipeline((writer==null)?localNode:writer,
+                       results.toArray(new DatanodeDescriptor[results.size()]));
+  }
+    
+  /* choose <i>numOfReplicas</i> from all data nodes */
+  private DatanodeDescriptor chooseTarget(int numOfReplicas,
+                                          DatanodeDescriptor writer,
+                                          HashMap<Node, Node> excludedNodes,
+                                          long blocksize,
+                                          int maxNodesPerRack,
+                                          List<DatanodeDescriptor> results) {
+      
+    if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
+      return writer;
+    }
+      
+    int numOfResults = results.size();
+    boolean newBlock = (numOfResults==0);
+    if (writer == null && !newBlock) {
+      writer = results.get(0);
+    }
+      
+    try {
+      if (numOfResults == 0) {
+        writer = chooseLocalNode(writer, excludedNodes, 
+                                 blocksize, maxNodesPerRack, results);
+        if (--numOfReplicas == 0) {
+          return writer;
+        }
+      }
+      if (numOfResults <= 1) {
+        chooseRemoteRack(1, results.get(0), excludedNodes, 
+                         blocksize, maxNodesPerRack, results);
+        if (--numOfReplicas == 0) {
+          return writer;
+        }
+      }
+      if (numOfResults <= 2) {
+        if (clusterMap.isOnSameRack(results.get(0), results.get(1))) {
+          chooseRemoteRack(1, results.get(0), excludedNodes,
+                           blocksize, maxNodesPerRack, results);
+        } else if (newBlock){
+          chooseLocalRack(results.get(1), excludedNodes, blocksize, 
+                          maxNodesPerRack, results);
+        } else {
+          chooseLocalRack(writer, excludedNodes, blocksize,
+                          maxNodesPerRack, results);
+        }
+        if (--numOfReplicas == 0) {
+          return writer;
+        }
+      }
+      chooseRandom(numOfReplicas, NodeBase.ROOT, excludedNodes, 
+                   blocksize, maxNodesPerRack, results);
+    } catch (NotEnoughReplicasException e) {
+      FSNamesystem.LOG.warn("Not able to place enough replicas, still in need of "
+               + numOfReplicas);
+    }
+    return writer;
+  }
+    
+  /* choose <i>localMachine</i> as the target.
+   * if <i>localMachine</i> is not available, 
+   * choose a node on the same rack
+   * @return the chosen node
+   */
+  private DatanodeDescriptor chooseLocalNode(
+                                             DatanodeDescriptor localMachine,
+                                             HashMap<Node, Node> excludedNodes,
+                                             long blocksize,
+                                             int maxNodesPerRack,
+                                             List<DatanodeDescriptor> results)
+    throws NotEnoughReplicasException {
+    // if no local machine, randomly choose one node
+    if (localMachine == null)
+      return chooseRandom(NodeBase.ROOT, excludedNodes, 
+                          blocksize, maxNodesPerRack, results);
+      
+    // otherwise try local machine first
+    Node oldNode = excludedNodes.put(localMachine, localMachine);
+    if (oldNode == null) { // was not in the excluded list
+      if (isGoodTarget(localMachine, blocksize,
+                       maxNodesPerRack, false, results)) {
+        results.add(localMachine);
+        return localMachine;
+      }
+    } 
+      
+    // try a node on local rack
+    return chooseLocalRack(localMachine, excludedNodes, 
+                           blocksize, maxNodesPerRack, results);
+  }
+    
+  /* choose one node from the rack that <i>localMachine</i> is on.
+   * if no such node is available, choose one node from the rack where
+   * a second replica is on.
+   * if still no such node is available, choose a random node 
+   * in the cluster.
+   * @return the chosen node
+   */
+  private DatanodeDescriptor chooseLocalRack(
+                                             DatanodeDescriptor localMachine,
+                                             HashMap<Node, Node> excludedNodes,
+                                             long blocksize,
+                                             int maxNodesPerRack,
+                                             List<DatanodeDescriptor> results)
+    throws NotEnoughReplicasException {
+    // no local machine, so choose a random machine
+    if (localMachine == null) {
+      return chooseRandom(NodeBase.ROOT, excludedNodes, 
+                          blocksize, maxNodesPerRack, results);
+    }
+      
+    // choose one from the local rack
+    try {
+      return chooseRandom(
+                          localMachine.getNetworkLocation(),
+                          excludedNodes, blocksize, maxNodesPerRack, results);
+    } catch (NotEnoughReplicasException e1) {
+      // find the second replica
+      DatanodeDescriptor newLocal=null;
+      for(Iterator<DatanodeDescriptor> iter=results.iterator();
+          iter.hasNext();) {
+        DatanodeDescriptor nextNode = iter.next();
+        if (nextNode != localMachine) {
+          newLocal = nextNode;
+          break;
+        }
+      }
+      if (newLocal != null) {
+        try {
+          return chooseRandom(
+                              newLocal.getNetworkLocation(),
+                              excludedNodes, blocksize, maxNodesPerRack, results);
+        } catch(NotEnoughReplicasException e2) {
+          //otherwise randomly choose one from the network
+          return chooseRandom(NodeBase.ROOT, excludedNodes,
+                              blocksize, maxNodesPerRack, results);
+        }
+      } else {
+        //otherwise randomly choose one from the network
+        return chooseRandom(NodeBase.ROOT, excludedNodes,
+                            blocksize, maxNodesPerRack, results);
+      }
+    }
+  }
+    
+  /* choose <i>numOfReplicas</i> nodes from the racks 
+   * that <i>localMachine</i> is NOT on.
+   * if not enough nodes are available, choose the remaining ones 
+   * from the local rack
+   */
+    
+  private void chooseRemoteRack(int numOfReplicas,
+                                DatanodeDescriptor localMachine,
+                                HashMap<Node, Node> excludedNodes,
+                                long blocksize,
+                                int maxReplicasPerRack,
+                                List<DatanodeDescriptor> results)
+    throws NotEnoughReplicasException {
+    int oldNumOfReplicas = results.size();
+    // randomly choose one node from remote racks
+    try {
+      chooseRandom(numOfReplicas, "~"+localMachine.getNetworkLocation(),
+                   excludedNodes, blocksize, maxReplicasPerRack, results);
+    } catch (NotEnoughReplicasException e) {
+      chooseRandom(numOfReplicas-(results.size()-oldNumOfReplicas),
+                   localMachine.getNetworkLocation(), excludedNodes, blocksize, 
+                   maxReplicasPerRack, results);
+    }
+  }
+
+  /* Randomly choose one target from <i>nodes</i>.
+   * @return the chosen node
+   */
+  private DatanodeDescriptor chooseRandom(
+                                          String nodes,
+                                          HashMap<Node, Node> excludedNodes,
+                                          long blocksize,
+                                          int maxNodesPerRack,
+                                          List<DatanodeDescriptor> results) 
+    throws NotEnoughReplicasException {
+    int numOfAvailableNodes =
+      clusterMap.countNumOfAvailableNodes(nodes, excludedNodes.keySet());
+    while(numOfAvailableNodes > 0) {
+      DatanodeDescriptor choosenNode = 
+        (DatanodeDescriptor)(clusterMap.chooseRandom(nodes));
+
+      Node oldNode = excludedNodes.put(choosenNode, choosenNode);
+      if (oldNode == null) { // choosendNode was not in the excluded list
+        numOfAvailableNodes--;
+        if (isGoodTarget(choosenNode, blocksize, maxNodesPerRack, results)) {
+          results.add(choosenNode);
+          return choosenNode;
+        }
+      }
+    }
+
+    throw new NotEnoughReplicasException(
+        "Not able to place enough replicas");
+  }
+    
+  /* Randomly choose <i>numOfReplicas</i> targets from <i>nodes</i>.
+   */
+  private void chooseRandom(int numOfReplicas,
+                            String nodes,
+                            HashMap<Node, Node> excludedNodes,
+                            long blocksize,
+                            int maxNodesPerRack,
+                            List<DatanodeDescriptor> results)
+    throws NotEnoughReplicasException {
+      
+    int numOfAvailableNodes =
+      clusterMap.countNumOfAvailableNodes(nodes, excludedNodes.keySet());
+    while(numOfReplicas > 0 && numOfAvailableNodes > 0) {
+      DatanodeDescriptor choosenNode = 
+        (DatanodeDescriptor)(clusterMap.chooseRandom(nodes));
+      Node oldNode = excludedNodes.put(choosenNode, choosenNode);
+      if (oldNode == null) {
+        numOfAvailableNodes--;
+
+        if (isGoodTarget(choosenNode, blocksize, maxNodesPerRack, results)) {
+          numOfReplicas--;
+          results.add(choosenNode);
+        }
+      }
+    }
+      
+    if (numOfReplicas>0) {
+      throw new NotEnoughReplicasException(
+                                           "Not able to place enough replicas");
+    }
+  }
+    
+  /* judge if a node is a good target.
+   * return true if <i>node</i> has enough space, 
+   * does not have too much load, and the rack does not have too many nodes
+   */
+  private boolean isGoodTarget(DatanodeDescriptor node,
+                               long blockSize, int maxTargetPerLoc,
+                               List<DatanodeDescriptor> results) {
+    return isGoodTarget(node, blockSize, maxTargetPerLoc,
+                        this.considerLoad, results);
+  }
+    
+  private boolean isGoodTarget(DatanodeDescriptor node,
+                               long blockSize, int maxTargetPerLoc,
+                               boolean considerLoad,
+                               List<DatanodeDescriptor> results) {
+    Log logr = FSNamesystem.LOG;
+    // check if the node is (being) decommissed
+    if (node.isDecommissionInProgress() || node.isDecommissioned()) {
+      logr.debug("Node "+NodeBase.getPath(node)+
+                " is not chosen because the node is (being) decommissioned");
+      return false;
+    }
+
+    long remaining = node.getRemaining() - 
+                     (node.getBlocksScheduled() * blockSize); 
+    // check the remaining capacity of the target machine
+    if (blockSize* FSConstants.MIN_BLOCKS_FOR_WRITE>remaining) {
+      logr.debug("Node "+NodeBase.getPath(node)+
+                " is not chosen because the node does not have enough space");
+      return false;
+    }
+      
+    // check the communication traffic of the target machine
+    if (considerLoad) {
+      double avgLoad = 0;
+      int size = clusterMap.getNumOfLeaves();
+      if (size != 0) {
+        avgLoad = (double)fs.getTotalLoad()/size;
+      }
+      if (node.getXceiverCount() > (2.0 * avgLoad)) {
+        logr.debug("Node "+NodeBase.getPath(node)+
+                  " is not chosen because the node is too busy");
+        return false;
+      }
+    }
+      
+    // check if the target rack has chosen too many nodes
+    String rackname = node.getNetworkLocation();
+    int counter=1;
+    for(Iterator<DatanodeDescriptor> iter = results.iterator();
+        iter.hasNext();) {
+      Node result = iter.next();
+      if (rackname.equals(result.getNetworkLocation())) {
+        counter++;
+      }
+    }
+    if (counter>maxTargetPerLoc) {
+      logr.debug("Node "+NodeBase.getPath(node)+
+                " is not chosen because the rack has too many chosen nodes");
+      return false;
+    }
+    return true;
+  }
+    
+  /* Return a pipeline of nodes.
+   * The pipeline is formed finding a shortest path that 
+   * starts from the writer and traverses all <i>nodes</i>
+   * This is basically a traveling salesman problem.
+   */
+  private DatanodeDescriptor[] getPipeline(
+                                           DatanodeDescriptor writer,
+                                           DatanodeDescriptor[] nodes) {
+    if (nodes.length==0) return nodes;
+      
+    synchronized(clusterMap) {
+      int index=0;
+      if (writer == null || !clusterMap.contains(writer)) {
+        writer = nodes[0];
+      }
+      for(;index<nodes.length; index++) {
+        DatanodeDescriptor shortestNode = nodes[index];
+        int shortestDistance = clusterMap.getDistance(writer, shortestNode);
+        int shortestIndex = index;
+        for(int i=index+1; i<nodes.length; i++) {
+          DatanodeDescriptor currentNode = nodes[i];
+          int currentDistance = clusterMap.getDistance(writer, currentNode);
+          if (shortestDistance>currentDistance) {
+            shortestDistance = currentDistance;
+            shortestNode = currentNode;
+            shortestIndex = i;
+          }
+        }
+        //switch position index & shortestIndex
+        if (index != shortestIndex) {
+          nodes[shortestIndex] = nodes[index];
+          nodes[index] = shortestNode;
+        }
+        writer = shortestNode;
+      }
+    }
+    return nodes;
+  }
+
+  /**
+   * Verify that the block is replicated on at least 2 different racks
+   * if there is more than one rack in the system.
+   * 
+   * @param lBlk block with locations
+   * @param cluster 
+   * @return 1 if the block must be replicated on additional rack,
+   * or 0 if the number of racks is sufficient.
+   */
+  public static int verifyBlockPlacement(LocatedBlock lBlk,
+                                         short replication,
+                                         NetworkTopology cluster) {
+    int numRacks = verifyBlockPlacement(lBlk, Math.min(2,replication), cluster);
+    return numRacks < 0 ? 0 : numRacks;
+  }
+
+  /**
+   * Verify that the block is replicated on at least minRacks different racks
+   * if there is more than minRacks rack in the system.
+   * 
+   * @param lBlk block with locations
+   * @param minRacks number of racks the block should be replicated to
+   * @param cluster 
+   * @return the difference between the required and the actual number of racks
+   * the block is replicated to.
+   */
+  public static int verifyBlockPlacement(LocatedBlock lBlk,
+                                         int minRacks,
+                                         NetworkTopology cluster) {
+    DatanodeInfo[] locs = lBlk.getLocations();
+    if (locs == null)
+      locs = new DatanodeInfo[0];
+    int numRacks = cluster.getNumOfRacks();
+    if(numRacks <= 1) // only one rack
+      return 0;
+    minRacks = Math.min(minRacks, numRacks);
+    // 1. Check that all locations are different.
+    // 2. Count locations on different racks.
+    Set<String> racks = new TreeSet<String>();
+    for (DatanodeInfo dn : locs)
+      racks.add(dn.getNetworkLocation());
+    return minRacks - racks.size();
+  }
+} //end of Replicator
+

+ 35 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java

@@ -0,0 +1,35 @@
+/**
+ * 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 java.io.IOException;
+
+/**
+ * This exception is thrown when the name node is in safe mode.
+ * Client cannot modified namespace until the safe mode is off. 
+ * 
+ */
+public class SafeModeException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public SafeModeException(String text, FSNamesystem.SafeModeInfo mode ) {
+    super(text + ". Name node is in safe mode.\n" + mode.getTurnOffTip());
+  }
+
+}

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

@@ -0,0 +1,585 @@
+/**
+ * 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 java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSUtil.ErrorSimulator;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
+import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
+import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeFile;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.metrics.jvm.JvmMetrics;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.StringUtils;
+
+/**********************************************************
+ * The Secondary NameNode is a helper to the primary NameNode.
+ * The Secondary is responsible for supporting periodic checkpoints 
+ * of the HDFS metadata. The current design allows only one Secondary
+ * NameNode per HDFs cluster.
+ *
+ * The Secondary NameNode is a daemon that periodically wakes
+ * up (determined by the schedule specified in the configuration),
+ * triggers a periodic checkpoint and then goes back to sleep.
+ * The Secondary NameNode uses the ClientProtocol to talk to the
+ * primary NameNode.
+ *
+ **********************************************************/
+@Deprecated // use BackupNode with -checkpoint argument instead.
+public class SecondaryNameNode implements Runnable {
+    
+  public static final Log LOG = 
+    LogFactory.getLog(SecondaryNameNode.class.getName());
+
+  private final long starttime = System.currentTimeMillis();
+  private volatile long lastCheckpointTime = 0;
+
+  private String fsName;
+  private CheckpointStorage checkpointImage;
+
+  private NamenodeProtocol namenode;
+  private Configuration conf;
+  private InetSocketAddress nameNodeAddr;
+  private volatile boolean shouldRun;
+  private HttpServer infoServer;
+  private int infoPort;
+  private String infoBindAddress;
+
+  private Collection<File> checkpointDirs;
+  private Collection<File> checkpointEditsDirs;
+  private long checkpointPeriod;	// in seconds
+  private long checkpointSize;    // size (in MB) of current Edit Log
+
+  /** {@inheritDoc} */
+  public String toString() {
+    return getClass().getSimpleName() + " Status" 
+      + "\nName Node Address    : " + nameNodeAddr   
+      + "\nStart Time           : " + new Date(starttime)
+      + "\nLast Checkpoint Time : " + (lastCheckpointTime == 0? "--": new Date(lastCheckpointTime))
+      + "\nCheckpoint Period    : " + checkpointPeriod + " seconds"
+      + "\nCheckpoint Size      : " + checkpointSize + " MB"
+      + "\nCheckpoint Dirs      : " + checkpointDirs
+      + "\nCheckpoint Edits Dirs: " + checkpointEditsDirs;
+  }
+
+  FSImage getFSImage() {
+    return checkpointImage;
+  }
+
+  /**
+   * Create a connection to the primary namenode.
+   */
+  public SecondaryNameNode(Configuration conf)  throws IOException {
+    try {
+      initialize(conf);
+    } catch(IOException e) {
+      shutdown();
+      throw e;
+    }
+  }
+
+  /**
+   * Initialize SecondaryNameNode.
+   */
+  private void initialize(Configuration conf) throws IOException {
+    // initiate Java VM metrics
+    JvmMetrics.init("SecondaryNameNode", conf.get("session.id"));
+    
+    // Create connection to the namenode.
+    shouldRun = true;
+    nameNodeAddr = NameNode.getAddress(conf);
+
+    this.conf = conf;
+    this.namenode =
+        (NamenodeProtocol) RPC.waitForProxy(NamenodeProtocol.class,
+            NamenodeProtocol.versionID, nameNodeAddr, conf);
+
+    // initialize checkpoint directories
+    fsName = getInfoServer();
+    checkpointDirs = FSImage.getCheckpointDirs(conf,
+                                  "/tmp/hadoop/dfs/namesecondary");
+    checkpointEditsDirs = FSImage.getCheckpointEditsDirs(conf, 
+                                  "/tmp/hadoop/dfs/namesecondary");    
+    checkpointImage = new CheckpointStorage();
+    checkpointImage.recoverCreate(checkpointDirs, checkpointEditsDirs);
+
+    // Initialize other scheduling parameters from the configuration
+    checkpointPeriod = conf.getLong("fs.checkpoint.period", 3600);
+    checkpointSize = conf.getLong("fs.checkpoint.size", 4194304);
+
+    // initialize the webserver for uploading files.
+    InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(
+        conf.get("dfs.secondary.http.address", "0.0.0.0:50090"));
+    infoBindAddress = infoSocAddr.getHostName();
+    int tmpInfoPort = infoSocAddr.getPort();
+    infoServer = new HttpServer("secondary", infoBindAddress, tmpInfoPort,
+        tmpInfoPort == 0, conf);
+    infoServer.setAttribute("secondary.name.node", this);
+    infoServer.setAttribute("name.system.image", checkpointImage);
+    this.infoServer.setAttribute("name.conf", conf);
+    infoServer.addInternalServlet("getimage", "/getimage", GetImageServlet.class);
+    infoServer.start();
+
+    // The web-server port can be ephemeral... ensure we have the correct info
+    infoPort = infoServer.getPort();
+    conf.set("dfs.secondary.http.address", infoBindAddress + ":" +infoPort); 
+    LOG.info("Secondary Web-server up at: " + infoBindAddress + ":" +infoPort);
+    LOG.warn("Checkpoint Period   :" + checkpointPeriod + " secs " +
+             "(" + checkpointPeriod/60 + " min)");
+    LOG.warn("Log Size Trigger    :" + checkpointSize + " bytes " +
+             "(" + checkpointSize/1024 + " KB)");
+  }
+
+  /**
+   * Shut down this instance of the datanode.
+   * Returns only after shutdown is complete.
+   */
+  public void shutdown() {
+    shouldRun = false;
+    try {
+      if (infoServer != null) infoServer.stop();
+    } catch (Exception e) {
+      LOG.warn("Exception shutting down SecondaryNameNode", e);
+    }
+    try {
+      if (checkpointImage != null) checkpointImage.close();
+    } catch(IOException e) {
+      LOG.warn(StringUtils.stringifyException(e));
+    }
+  }
+
+  //
+  // The main work loop
+  //
+  public void run() {
+
+    //
+    // Poll the Namenode (once every 5 minutes) to find the size of the
+    // pending edit log.
+    //
+    long period = 5 * 60;              // 5 minutes
+    if (checkpointPeriod < period) {
+      period = checkpointPeriod;
+    }
+
+    while (shouldRun) {
+      try {
+        Thread.sleep(1000 * period);
+      } catch (InterruptedException ie) {
+        // do nothing
+      }
+      if (!shouldRun) {
+        break;
+      }
+      try {
+        long now = System.currentTimeMillis();
+
+        long size = namenode.getEditLogSize();
+        if (size >= checkpointSize || 
+            now >= lastCheckpointTime + 1000 * checkpointPeriod) {
+          doCheckpoint();
+          lastCheckpointTime = now;
+        }
+      } catch (IOException e) {
+        LOG.error("Exception in doCheckpoint: ");
+        LOG.error(StringUtils.stringifyException(e));
+        e.printStackTrace();
+      } catch (Throwable e) {
+        LOG.error("Throwable Exception in doCheckpoint: ");
+        LOG.error(StringUtils.stringifyException(e));
+        e.printStackTrace();
+        Runtime.getRuntime().exit(-1);
+      }
+    }
+  }
+
+  /**
+   * Download <code>fsimage</code> and <code>edits</code>
+   * files from the name-node.
+   * @throws IOException
+   */
+  private void downloadCheckpointFiles(CheckpointSignature sig
+                                      ) throws IOException {
+    
+    checkpointImage.cTime = sig.cTime;
+    checkpointImage.checkpointTime = sig.checkpointTime;
+
+    // get fsimage
+    String fileid = "getimage=1";
+    Collection<File> list = checkpointImage.getFiles(NameNodeFile.IMAGE,
+        NameNodeDirType.IMAGE);
+    File[] srcNames = list.toArray(new File[list.size()]);
+    assert srcNames.length > 0 : "No checkpoint targets.";
+    TransferFsImage.getFileClient(fsName, fileid, srcNames);
+    LOG.info("Downloaded file " + srcNames[0].getName() + " size " +
+             srcNames[0].length() + " bytes.");
+
+    // get edits file
+    fileid = "getedit=1";
+    list = getFSImage().getFiles(NameNodeFile.EDITS, NameNodeDirType.EDITS);
+    srcNames = list.toArray(new File[list.size()]);;
+    assert srcNames.length > 0 : "No checkpoint targets.";
+    TransferFsImage.getFileClient(fsName, fileid, srcNames);
+    LOG.info("Downloaded file " + srcNames[0].getName() + " size " +
+        srcNames[0].length() + " bytes.");
+
+    checkpointImage.checkpointUploadDone();
+  }
+
+  /**
+   * Copy the new fsimage into the NameNode
+   */
+  private void putFSImage(CheckpointSignature sig) throws IOException {
+    String fileid = "putimage=1&port=" + infoPort +
+      "&machine=" +
+      InetAddress.getLocalHost().getHostAddress() +
+      "&token=" + sig.toString();
+    LOG.info("Posted URL " + fsName + fileid);
+    TransferFsImage.getFileClient(fsName, fileid, (File[])null);
+  }
+
+  /**
+   * Returns the Jetty server that the Namenode is listening on.
+   */
+  private String getInfoServer() throws IOException {
+    URI fsName = FileSystem.getDefaultUri(conf);
+    if (!"hdfs".equals(fsName.getScheme())) {
+      throw new IOException("This is not a DFS");
+    }
+    return conf.get("dfs.http.address", "0.0.0.0:50070");
+  }
+
+  /**
+   * Create a new checkpoint
+   */
+  void doCheckpoint() throws IOException {
+
+    // Do the required initialization of the merge work area.
+    startCheckpoint();
+
+    // Tell the namenode to start logging transactions in a new edit file
+    // Returns a token that would be used to upload the merged image.
+    CheckpointSignature sig = namenode.rollEditLog();
+
+    // error simulation code for junit test
+    if (ErrorSimulator.getErrorSimulation(0)) {
+      throw new IOException("Simulating error0 " +
+                            "after creating edits.new");
+    }
+
+    downloadCheckpointFiles(sig);   // Fetch fsimage and edits
+    doMerge(sig);                   // Do the merge
+  
+    //
+    // Upload the new image into the NameNode. Then tell the Namenode
+    // to make this new uploaded image as the most current image.
+    //
+    putFSImage(sig);
+
+    // error simulation code for junit test
+    if (ErrorSimulator.getErrorSimulation(1)) {
+      throw new IOException("Simulating error1 " +
+                            "after uploading new image to NameNode");
+    }
+
+    namenode.rollFsImage();
+    checkpointImage.endCheckpoint();
+
+    LOG.warn("Checkpoint done. New Image Size: " 
+              + checkpointImage.getFsImageName().length());
+  }
+
+  private void startCheckpoint() throws IOException {
+    checkpointImage.unlockAll();
+    checkpointImage.getEditLog().close();
+    checkpointImage.recoverCreate(checkpointDirs, checkpointEditsDirs);
+    checkpointImage.startCheckpoint();
+  }
+
+  /**
+   * Merge downloaded image and edits and write the new image into
+   * current storage directory.
+   */
+  private void doMerge(CheckpointSignature sig) throws IOException {
+    FSNamesystem namesystem = 
+            new FSNamesystem(checkpointImage, conf);
+    assert namesystem.dir.fsImage == checkpointImage;
+    checkpointImage.doMerge(sig);
+  }
+
+  /**
+   * @param argv The parameters passed to this program.
+   * @exception Exception if the filesystem does not exist.
+   * @return 0 on success, non zero on error.
+   */
+  private int processArgs(String[] argv) throws Exception {
+
+    if (argv.length < 1) {
+      printUsage("");
+      return -1;
+    }
+
+    int exitCode = -1;
+    int i = 0;
+    String cmd = argv[i++];
+
+    //
+    // verify that we have enough command line parameters
+    //
+    if ("-geteditsize".equals(cmd)) {
+      if (argv.length != 1) {
+        printUsage(cmd);
+        return exitCode;
+      }
+    } else if ("-checkpoint".equals(cmd)) {
+      if (argv.length != 1 && argv.length != 2) {
+        printUsage(cmd);
+        return exitCode;
+      }
+      if (argv.length == 2 && !"force".equals(argv[i])) {
+        printUsage(cmd);
+        return exitCode;
+      }
+    }
+
+    exitCode = 0;
+    try {
+      if ("-checkpoint".equals(cmd)) {
+        long size = namenode.getEditLogSize();
+        if (size >= checkpointSize || 
+            argv.length == 2 && "force".equals(argv[i])) {
+          doCheckpoint();
+        } else {
+          System.err.println("EditLog size " + size + " bytes is " +
+                             "smaller than configured checkpoint " +
+                             "size " + checkpointSize + " bytes.");
+          System.err.println("Skipping checkpoint.");
+        }
+      } else if ("-geteditsize".equals(cmd)) {
+        long size = namenode.getEditLogSize();
+        System.out.println("EditLog size is " + size + " bytes");
+      } else {
+        exitCode = -1;
+        LOG.error(cmd.substring(1) + ": Unknown command");
+        printUsage("");
+      }
+    } catch (RemoteException e) {
+      //
+      // This is a error returned by hadoop server. Print
+      // out the first line of the error mesage, ignore the stack trace.
+      exitCode = -1;
+      try {
+        String[] content;
+        content = e.getLocalizedMessage().split("\n");
+        LOG.error(cmd.substring(1) + ": "
+                  + content[0]);
+      } catch (Exception ex) {
+        LOG.error(cmd.substring(1) + ": "
+                  + ex.getLocalizedMessage());
+      }
+    } catch (IOException e) {
+      //
+      // IO exception encountered locally.
+      //
+      exitCode = -1;
+      LOG.error(cmd.substring(1) + ": "
+                + e.getLocalizedMessage());
+    } finally {
+      // Does the RPC connection need to be closed?
+    }
+    return exitCode;
+  }
+
+  /**
+   * Displays format of commands.
+   * @param cmd The command that is being executed.
+   */
+  private void printUsage(String cmd) {
+    if ("-geteditsize".equals(cmd)) {
+      System.err.println("Usage: java SecondaryNameNode"
+                         + " [-geteditsize]");
+    } else if ("-checkpoint".equals(cmd)) {
+      System.err.println("Usage: java SecondaryNameNode"
+                         + " [-checkpoint [force]]");
+    } else {
+      System.err.println("Usage: java SecondaryNameNode " +
+                         "[-checkpoint [force]] " +
+                         "[-geteditsize] ");
+    }
+  }
+
+  /**
+   * main() has some simple utility methods.
+   * @param argv Command line parameters.
+   * @exception Exception if the filesystem does not exist.
+   */
+  public static void main(String[] argv) throws Exception {
+    StringUtils.startupShutdownMessage(SecondaryNameNode.class, argv, LOG);
+    Configuration tconf = new Configuration();
+    if (argv.length >= 1) {
+      SecondaryNameNode secondary = new SecondaryNameNode(tconf);
+      int ret = secondary.processArgs(argv);
+      System.exit(ret);
+    }
+
+    // Create a never ending deamon
+    Daemon checkpointThread = new Daemon(new SecondaryNameNode(tconf)); 
+    checkpointThread.start();
+  }
+
+  static class CheckpointStorage extends FSImage {
+    /**
+     */
+    CheckpointStorage() throws IOException {
+      super();
+    }
+
+    @Override
+    public
+    boolean isConversionNeeded(StorageDirectory sd) {
+      return false;
+    }
+
+    /**
+     * Analyze checkpoint directories.
+     * Create directories if they do not exist.
+     * Recover from an unsuccessful checkpoint is necessary. 
+     * 
+     * @param dataDirs
+     * @param editsDirs
+     * @throws IOException
+     */
+    void recoverCreate(Collection<File> dataDirs,
+                       Collection<File> editsDirs) throws IOException {
+      Collection<File> tempDataDirs = new ArrayList<File>(dataDirs);
+      Collection<File> tempEditsDirs = new ArrayList<File>(editsDirs);
+      this.storageDirs = new ArrayList<StorageDirectory>();
+      setStorageDirectories(tempDataDirs, tempEditsDirs);
+      for (Iterator<StorageDirectory> it = 
+                   dirIterator(); it.hasNext();) {
+        StorageDirectory sd = it.next();
+        boolean isAccessible = true;
+        try { // create directories if don't exist yet
+          if(!sd.getRoot().mkdirs()) {
+            // do nothing, directory is already created
+          }
+        } catch(SecurityException se) {
+          isAccessible = false;
+        }
+        if(!isAccessible)
+          throw new InconsistentFSStateException(sd.getRoot(),
+              "cannot access checkpoint directory.");
+        StorageState curState;
+        try {
+          curState = sd.analyzeStorage(HdfsConstants.StartupOption.REGULAR);
+          // sd is locked but not opened
+          switch(curState) {
+          case NON_EXISTENT:
+            // fail if any of the configured checkpoint dirs are inaccessible 
+            throw new InconsistentFSStateException(sd.getRoot(),
+                  "checkpoint directory does not exist or is not accessible.");
+          case NOT_FORMATTED:
+            break;  // it's ok since initially there is no current and VERSION
+          case NORMAL:
+            break;
+          default:  // recovery is possible
+            sd.doRecover(curState);
+          }
+        } catch (IOException ioe) {
+          sd.unlock();
+          throw ioe;
+        }
+      }
+    }
+
+    /**
+     * Prepare directories for a new checkpoint.
+     * <p>
+     * Rename <code>current</code> to <code>lastcheckpoint.tmp</code>
+     * and recreate <code>current</code>.
+     * @throws IOException
+     */
+    void startCheckpoint() throws IOException {
+      for(StorageDirectory sd : storageDirs) {
+        File curDir = sd.getCurrentDir();
+        File tmpCkptDir = sd.getLastCheckpointTmp();
+        assert !tmpCkptDir.exists() : 
+          tmpCkptDir.getName() + " directory must not exist.";
+        if(curDir.exists()) {
+          // rename current to tmp
+          rename(curDir, tmpCkptDir);
+        }
+        if (!curDir.mkdir())
+          throw new IOException("Cannot create directory " + curDir);
+      }
+    }
+
+    void endCheckpoint() throws IOException {
+      for(StorageDirectory sd : storageDirs) {
+        File tmpCkptDir = sd.getLastCheckpointTmp();
+        File prevCkptDir = sd.getPreviousCheckpoint();
+        // delete previous dir
+        if (prevCkptDir.exists())
+          deleteDir(prevCkptDir);
+        // rename tmp to previous
+        if (tmpCkptDir.exists())
+          rename(tmpCkptDir, prevCkptDir);
+      }
+    }
+
+    /**
+     * Merge image and edits, and verify consistency with the signature.
+     */
+    private void doMerge(CheckpointSignature sig) throws IOException {
+      getEditLog().open();
+      StorageDirectory sdName = null;
+      StorageDirectory sdEdits = null;
+      Iterator<StorageDirectory> it = null;
+      it = dirIterator(NameNodeDirType.IMAGE);
+      if (it.hasNext())
+        sdName = it.next();
+      it = dirIterator(NameNodeDirType.EDITS);
+      if (it.hasNext())
+        sdEdits = it.next();
+      if ((sdName == null) || (sdEdits == null))
+        throw new IOException("Could not locate checkpoint directories");
+      loadFSImage(FSImage.getImageFile(sdName, NameNodeFile.IMAGE));
+      loadFSEdits(sdEdits);
+      sig.validateStorageInfo(this);
+      saveFSImage();
+    }
+  }
+}

+ 72 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/SerialNumberManager.java

@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.util.*;
+
+/** Manage name-to-serial-number maps for users and groups. */
+class SerialNumberManager {
+  /** This is the only instance of {@link SerialNumberManager}.*/
+  static final SerialNumberManager INSTANCE = new SerialNumberManager();
+
+  private SerialNumberMap<String> usermap = new SerialNumberMap<String>();
+  private SerialNumberMap<String> groupmap = new SerialNumberMap<String>();
+
+  private SerialNumberManager() {}
+
+  int getUserSerialNumber(String u) {return usermap.get(u);}
+  int getGroupSerialNumber(String g) {return groupmap.get(g);}
+  String getUser(int n) {return usermap.get(n);}
+  String getGroup(int n) {return groupmap.get(n);}
+
+  {
+    getUserSerialNumber(null);
+    getGroupSerialNumber(null);
+  }
+
+  private static class SerialNumberMap<T> {
+    private int max = 0;
+    private int nextSerialNumber() {return max++;}
+
+    private Map<T, Integer> t2i = new HashMap<T, Integer>();
+    private Map<Integer, T> i2t = new HashMap<Integer, T>();
+
+    synchronized int get(T t) {
+      Integer sn = t2i.get(t);
+      if (sn == null) {
+        sn = nextSerialNumber();
+        t2i.put(t, sn);
+        i2t.put(sn, t);
+      }
+      return sn;
+    }
+
+    synchronized T get(int i) {
+      if (!i2t.containsKey(i)) {
+        throw new IllegalStateException("!i2t.containsKey(" + i
+            + "), this=" + this);
+      }
+      return i2t.get(i);
+    }
+
+    /** {@inheritDoc} */
+    public String toString() {
+      return "max=" + max + ",\n  t2i=" + t2i + ",\n  i2t=" + i2t;
+    }
+  }
+}

+ 85 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java

@@ -0,0 +1,85 @@
+/**
+ * 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 java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.net.InetSocketAddress;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+
+public class StreamFile extends DfsServlet {
+  /** for java.io.Serializable */
+  private static final long serialVersionUID = 1L;
+
+  static InetSocketAddress nameNodeAddr;
+  static DataNode datanode = null;
+  private static final Configuration masterConf = new Configuration();
+  static {
+    if ((datanode = DataNode.getDataNode()) != null) {
+      nameNodeAddr = datanode.getNameNodeAddr();
+    }
+  }
+  
+  /** getting a client for connecting to dfs */
+  protected DFSClient getDFSClient(HttpServletRequest request)
+      throws IOException {
+    Configuration conf = new Configuration(masterConf);
+    UnixUserGroupInformation.saveToConf(conf,
+        UnixUserGroupInformation.UGI_PROPERTY_NAME, getUGI(request));
+    return new DFSClient(nameNodeAddr, conf);
+  }
+  
+  public void doGet(HttpServletRequest request, HttpServletResponse response)
+    throws ServletException, IOException {
+    final String filename = JspHelper.validatePath(
+        request.getParameter("filename"));
+    if (filename == null) {
+      response.setContentType("text/plain");
+      PrintWriter out = response.getWriter();
+      out.print("Invalid input");
+      return;
+    }
+    DFSClient dfs = getDFSClient(request);
+    FSInputStream in = dfs.open(filename);
+    OutputStream os = response.getOutputStream();
+    response.setHeader("Content-Disposition", "attachment; filename=\"" + 
+                       filename + "\"");
+    response.setContentType("application/octet-stream");
+    byte buf[] = new byte[4096];
+    try {
+      int bytesRead;
+      while ((bytesRead = in.read(buf)) != -1) {
+        os.write(buf, 0, bytesRead);
+      }
+    } finally {
+      in.close();
+      os.close();
+      dfs.close();
+    }
+  }
+}

Vissa filer visades inte eftersom för många filer har ändrats