瀏覽代碼

HADOOP-2885. Break the hadoop.dfs package into separate packages under
hadoop.hdfs that reflect whether they are client, server, protocol,
etc. DistributedFileSystem and DFSClient have moved and are now
considered package private. Contributed by Sanjay Radia.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@673857 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 17 年之前
父節點
當前提交
cf9afa130e
共有 100 個文件被更改,包括 1864 次插入1538 次删除
  1. 5 0
      CHANGES.txt
  2. 6 6
      bin/hadoop
  3. 2 2
      build.xml
  4. 3 3
      conf/hadoop-default.xml
  5. 7 4
      docs/changes.html
  6. 3 3
      docs/hadoop-default.html
  7. 1 1
      src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/NewDriverWizardPage.java
  8. 1 1
      src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/dfs/DFSPath.java
  9. 1 1
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdater.java
  10. 1 1
      src/contrib/index/src/test/org/apache/hadoop/contrib/index/mapred/TestDistributionPolicy.java
  11. 1 1
      src/contrib/index/src/test/org/apache/hadoop/contrib/index/mapred/TestIndexUpdater.java
  12. 1 1
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java
  13. 1 1
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleCachefiles.java
  14. 1 1
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamedMerge.java
  15. 1 1
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestSymLink.java
  16. 1 1
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestUlimit.java
  17. 1 1
      src/core/org/apache/hadoop/fs/FileSystem.java
  18. 1 1
      src/core/org/apache/hadoop/fs/FsShell.java
  19. 1 1
      src/core/org/apache/hadoop/fs/Syncable.java
  20. 1 1
      src/core/org/apache/hadoop/fs/permission/AccessControlException.java
  21. 1 1
      src/core/org/apache/hadoop/fs/permission/FsAction.java
  22. 1 1
      src/core/org/apache/hadoop/fs/s3/FileSystemStore.java
  23. 1 1
      src/core/org/apache/hadoop/fs/s3native/Jets3tNativeFileSystemStore.java
  24. 1 1
      src/core/org/apache/hadoop/fs/shell/Command.java
  25. 1 1
      src/core/org/apache/hadoop/fs/shell/CommandFormat.java
  26. 1 1
      src/core/org/apache/hadoop/fs/shell/CommandUtils.java
  27. 1 1
      src/core/org/apache/hadoop/io/MultipleIOException.java
  28. 1 1
      src/core/org/apache/hadoop/log/LogLevel.java
  29. 1 1
      src/core/org/apache/hadoop/metrics/MetricsUtil.java
  30. 4 2
      src/core/org/apache/hadoop/util/DataChecksum.java
  31. 1 1
      src/core/overview.html
  32. 0 1066
      src/hdfs/org/apache/hadoop/dfs/INode.java
  33. 6 3
      src/hdfs/org/apache/hadoop/hdfs/ChecksumDistributedFileSystem.java
  34. 16 11
      src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
  35. 51 0
      src/hdfs/org/apache/hadoop/hdfs/DFSUtil.java
  36. 18 10
      src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java
  37. 4 3
      src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java
  38. 3 3
      src/hdfs/org/apache/hadoop/hdfs/HsftpFileSystem.java
  39. 0 0
      src/hdfs/org/apache/hadoop/hdfs/package.html
  40. 1 1
      src/hdfs/org/apache/hadoop/hdfs/protocol/AlreadyBeingCreatedException.java
  41. 38 28
      src/hdfs/org/apache/hadoop/hdfs/protocol/Block.java
  42. 8 8
      src/hdfs/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
  43. 7 7
      src/hdfs/org/apache/hadoop/hdfs/protocol/BlocksWithLocations.java
  44. 3 3
      src/hdfs/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
  45. 6 4
      src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  46. 4 2
      src/hdfs/org/apache/hadoop/hdfs/protocol/DFSFileInfo.java
  47. 8 8
      src/hdfs/org/apache/hadoop/hdfs/protocol/DatanodeID.java
  48. 16 16
      src/hdfs/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  49. 3 3
      src/hdfs/org/apache/hadoop/hdfs/protocol/FSConstants.java
  50. 6 6
      src/hdfs/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
  51. 8 7
      src/hdfs/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
  52. 1 1
      src/hdfs/org/apache/hadoop/hdfs/protocol/QuotaExceededException.java
  53. 2 2
      src/hdfs/org/apache/hadoop/hdfs/protocol/UnregisteredDatanodeException.java
  54. 13 8
      src/hdfs/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  55. 5 5
      src/hdfs/org/apache/hadoop/hdfs/server/common/GenerationStamp.java
  56. 10 6
      src/hdfs/org/apache/hadoop/hdfs/server/common/GenerationStampStatsUpgradeCommand.java
  57. 2 2
      src/hdfs/org/apache/hadoop/hdfs/server/common/InconsistentFSStateException.java
  58. 4 2
      src/hdfs/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java
  59. 41 73
      src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java
  60. 37 0
      src/hdfs/org/apache/hadoop/hdfs/server/common/StorageInfo.java
  61. 16 13
      src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeManager.java
  62. 3 3
      src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeObject.java
  63. 6 3
      src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeObjectCollection.java
  64. 1 1
      src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeStatusReport.java
  65. 4 1
      src/hdfs/org/apache/hadoop/hdfs/server/common/Upgradeable.java
  66. 28 0
      src/hdfs/org/apache/hadoop/hdfs/server/common/Util.java
  67. 2 1
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  68. 6 3
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
  69. 46 27
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  70. 12 7
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  71. 4 2
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/DatanodeBlockInfo.java
  72. 23 18
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
  73. 3 2
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
  74. 10 4
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/GenerationStampUpgradeDatanode.java
  75. 7 3
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java
  76. 8 3
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java
  77. 1 1
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
  78. 1 1
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeStatistics.java
  79. 2 2
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeStatisticsMBean.java
  80. 2 2
      src/hdfs/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java
  81. 6 4
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java
  82. 3 2
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java
  83. 4 3
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/CorruptReplicasMap.java
  84. 12 6
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java
  85. 4 2
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
  86. 6 2
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  87. 14 9
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  88. 20 12
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  89. 40 53
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  90. 7 2
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
  91. 2 2
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java
  92. 14 5
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/GenerationStampUpgradeNamenode.java
  93. 1 1
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
  94. 1 1
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/Host2NodesMap.java
  95. 381 0
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/INode.java
  96. 444 0
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  97. 165 0
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  98. 168 0
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  99. 11 4
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java
  100. 1 1
      src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseExpiredException.java

+ 5 - 0
CHANGES.txt

@@ -22,6 +22,11 @@ Trunk (unreleased changes)
     HADOOP-3652. Remove deprecated class OutputFormatBase.
     (Amareshwari Sriramadasu via cdouglas)
 
+    HADOOP-2885. Break the hadoop.dfs package into separate packages under
+    hadoop.hdfs that reflect whether they are client, server, protocol, 
+    etc. DistributedFileSystem and DFSClient have moved and are now 
+    considered package private. (Sanjay Radia via omalley)
+
   NEW FEATURES
 
     HADOOP-3341. Allow streaming jobs to specify the field separator for map

+ 6 - 6
bin/hadoop

@@ -197,13 +197,13 @@ unset IFS
 
 # figure out which class to run
 if [ "$COMMAND" = "namenode" ] ; then
-  CLASS='org.apache.hadoop.dfs.NameNode'
+  CLASS='org.apache.hadoop.hdfs.server.namenode.NameNode'
   HADOOP_OPTS="$HADOOP_OPTS $HADOOP_NAMENODE_OPTS"
 elif [ "$COMMAND" = "secondarynamenode" ] ; then
-  CLASS='org.apache.hadoop.dfs.SecondaryNameNode'
+  CLASS='org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode'
   HADOOP_OPTS="$HADOOP_OPTS $HADOOP_SECONDARYNAMENODE_OPTS"
 elif [ "$COMMAND" = "datanode" ] ; then
-  CLASS='org.apache.hadoop.dfs.DataNode'
+  CLASS='org.apache.hadoop.hdfs.server.datanode.DataNode'
   HADOOP_OPTS="$HADOOP_OPTS $HADOOP_DATANODE_OPTS"
 elif [ "$COMMAND" = "fs" ] ; then
   CLASS=org.apache.hadoop.fs.FsShell
@@ -212,13 +212,13 @@ elif [ "$COMMAND" = "dfs" ] ; then
   CLASS=org.apache.hadoop.fs.FsShell
   HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
 elif [ "$COMMAND" = "dfsadmin" ] ; then
-  CLASS=org.apache.hadoop.dfs.DFSAdmin
+  CLASS=org.apache.hadoop.hdfs.tools.DFSAdmin
   HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
 elif [ "$COMMAND" = "fsck" ] ; then
-  CLASS=org.apache.hadoop.dfs.DFSck
+  CLASS=org.apache.hadoop.hdfs.tools.DFSck
   HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
 elif [ "$COMMAND" = "balancer" ] ; then
-  CLASS=org.apache.hadoop.dfs.Balancer
+  CLASS=org.apache.hadoop.hdfs.server.balancer.Balancer
   HADOOP_OPTS="$HADOOP_OPTS $HADOOP_BALANCER_OPTS"
 elif [ "$COMMAND" = "jobtracker" ] ; then
   CLASS=org.apache.hadoop.mapred.JobTracker

+ 2 - 2
build.xml

@@ -584,8 +584,8 @@
     <copy file="${test.src.dir}/org/apache/hadoop/mapred/test.tar" todir="${test.cache.data}"/>
     <copy file="${test.src.dir}/org/apache/hadoop/mapred/test.tgz" todir="${test.cache.data}"/>
     <copy file="${test.src.dir}/org/apache/hadoop/mapred/test.tar.gz" todir="${test.cache.data}"/>
-    <copy file="${test.src.dir}/org/apache/hadoop/dfs/hadoop-14-dfs-dir.tgz" todir="${test.cache.data}"/>
-    <copy file="${test.src.dir}/org/apache/hadoop/dfs/hadoop-dfs-dir.txt" todir="${test.cache.data}"/>
+    <copy file="${test.src.dir}/org/apache/hadoop/hdfs/hadoop-14-dfs-dir.tgz" todir="${test.cache.data}"/>
+    <copy file="${test.src.dir}/org/apache/hadoop/hdfs/hadoop-dfs-dir.txt" todir="${test.cache.data}"/>
     <copy file="${test.src.dir}/org/apache/hadoop/cli/testConf.xml" todir="${test.cache.data}"/>
     <copy file="${test.src.dir}/org/apache/hadoop/cli/clitest_data/data15bytes" todir="${test.cache.data}"/>
     <copy file="${test.src.dir}/org/apache/hadoop/cli/clitest_data/data30bytes" todir="${test.cache.data}"/>

+ 3 - 3
conf/hadoop-default.xml

@@ -171,7 +171,7 @@ creations/deletions), or "all".</description>
 
 <property>
   <name>fs.hdfs.impl</name>
-  <value>org.apache.hadoop.dfs.DistributedFileSystem</value>
+  <value>org.apache.hadoop.hdfs.DistributedFileSystem</value>
   <description>The FileSystem for hdfs: uris.</description>
 </property>
 
@@ -195,12 +195,12 @@ creations/deletions), or "all".</description>
 
 <property>
   <name>fs.hftp.impl</name>
-  <value>org.apache.hadoop.dfs.HftpFileSystem</value>
+  <value>org.apache.hadoop.hdfs.HftpFileSystem</value>
 </property>
 
 <property>
   <name>fs.hsftp.impl</name>
-  <value>org.apache.hadoop.dfs.HsftpFileSystem</value>
+  <value>org.apache.hadoop.hdfs.HsftpFileSystem</value>
 </property>
 
 <property>

+ 7 - 4
docs/changes.html

@@ -86,7 +86,7 @@ All of them default to "\t".<br />(Zheng Shao via omalley)</li>
     </ol>
   </li>
   <li><a href="javascript:toggleList('trunk_(unreleased_changes)_._improvements_')">  IMPROVEMENTS
-</a>&nbsp;&nbsp;&nbsp;(4)
+</a>&nbsp;&nbsp;&nbsp;(6)
     <ol id="trunk_(unreleased_changes)_._improvements_">
       <li><a href="http://issues.apache.org/jira/browse/HADOOP-3577">HADOOP-3577</a>. Tools to inject blocks into name node and simulated
 data nodes for testing.<br />(Sanjay Radia via hairong)</li>
@@ -95,6 +95,8 @@ may be processed by map/reduce.<br />(cdouglas via omalley)</li>
       <li><a href="http://issues.apache.org/jira/browse/HADOOP-3655">HADOOP-3655</a>. Add additional ant properties to control junit.<br />(Steve
 Loughran via omalley)</li>
       <li><a href="http://issues.apache.org/jira/browse/HADOOP-3543">HADOOP-3543</a>. Update the copyright year to 2008.<br />(cdouglas via omalley)</li>
+      <li><a href="http://issues.apache.org/jira/browse/HADOOP-3587">HADOOP-3587</a>. Add a unit test for the contrib/data_join framework.<br />(cdouglas)</li>
+      <li><a href="http://issues.apache.org/jira/browse/HADOOP-3402">HADOOP-3402</a>. Add terasort example program<br />(omalley)</li>
     </ol>
   </li>
   <li><a href="javascript:toggleList('trunk_(unreleased_changes)_._optimizations_')">  OPTIMIZATIONS
@@ -201,8 +203,7 @@ Bialecki via omalley)</li>
 via omalley)</li>
       <li><a href="http://issues.apache.org/jira/browse/HADOOP-3317">HADOOP-3317</a>. Add default port for HDFS namenode.  The port in
 "hdfs:" URIs now defaults to 8020, so that one may simply use URIs
-of the form "hdfs://example.com/dir/file".
-</li>
+of the form "hdfs://example.com/dir/file".<br />(cutting)</li>
       <li><a href="http://issues.apache.org/jira/browse/HADOOP-2019">HADOOP-2019</a>. Adds support for .tar, .tgz and .tar.gz files in
 DistributedCache<br />(Amareshwari Sriramadasu via ddas)</li>
       <li><a href="http://issues.apache.org/jira/browse/HADOOP-3058">HADOOP-3058</a>. Add FSNamesystem status metrics.<br />(Lohit Vjayarenu via rangadi)</li>
@@ -362,7 +363,7 @@ InputFormat.validateInput.<br />(tomwhite via omalley)</li>
     </ol>
   </li>
   <li><a href="javascript:toggleList('release_0.18.0_-_unreleased_._bug_fixes_')">  BUG FIXES
-</a>&nbsp;&nbsp;&nbsp;(108)
+</a>&nbsp;&nbsp;&nbsp;(109)
     <ol id="release_0.18.0_-_unreleased_._bug_fixes_">
       <li><a href="http://issues.apache.org/jira/browse/HADOOP-2905">HADOOP-2905</a>. 'fsck -move' triggers NPE in NameNode.<br />(Lohit Vjayarenu via rangadi)</li>
       <li>Increment ClientProtocol.versionID missed by <a href="http://issues.apache.org/jira/browse/HADOOP-2585">HADOOP-2585</a>.<br />(shv)</li>
@@ -577,6 +578,8 @@ open.<br />(Benjamin Gufler via hairong)</li>
       <li><a href="http://issues.apache.org/jira/browse/HADOOP-3604">HADOOP-3604</a>. Work around a JVM synchronization problem observed while
 retrieving the address of direct buffers from compression code by obtaining
 a lock during this call.<br />(Arun C Murthy via cdouglas)</li>
+      <li><a href="http://issues.apache.org/jira/browse/HADOOP-3678">HADOOP-3678</a>. Avoid spurious exceptions logged at DataNode when clients
+read from DFS.<br />(rangadi)</li>
     </ol>
   </li>
 </ul>

+ 3 - 3
docs/hadoop-default.html

@@ -99,7 +99,7 @@ creations/deletions), or "all".</td>
 <td><a name="fs.file.impl">fs.file.impl</a></td><td>org.apache.hadoop.fs.LocalFileSystem</td><td>The FileSystem for file: uris.</td>
 </tr>
 <tr>
-<td><a name="fs.hdfs.impl">fs.hdfs.impl</a></td><td>org.apache.hadoop.dfs.DistributedFileSystem</td><td>The FileSystem for hdfs: uris.</td>
+<td><a name="fs.hdfs.impl">fs.hdfs.impl</a></td><td>org.apache.hadoop.hdfs.DistributedFileSystem</td><td>The FileSystem for hdfs: uris.</td>
 </tr>
 <tr>
 <td><a name="fs.s3.impl">fs.s3.impl</a></td><td>org.apache.hadoop.fs.s3.S3FileSystem</td><td>The FileSystem for s3: uris.</td>
@@ -111,10 +111,10 @@ creations/deletions), or "all".</td>
 <td><a name="fs.kfs.impl">fs.kfs.impl</a></td><td>org.apache.hadoop.fs.kfs.KosmosFileSystem</td><td>The FileSystem for kfs: uris.</td>
 </tr>
 <tr>
-<td><a name="fs.hftp.impl">fs.hftp.impl</a></td><td>org.apache.hadoop.dfs.HftpFileSystem</td><td></td>
+<td><a name="fs.hftp.impl">fs.hftp.impl</a></td><td>org.apache.hadoop.hdfs.HftpFileSystem</td><td></td>
 </tr>
 <tr>
-<td><a name="fs.hsftp.impl">fs.hsftp.impl</a></td><td>org.apache.hadoop.dfs.HsftpFileSystem</td><td></td>
+<td><a name="fs.hsftp.impl">fs.hsftp.impl</a></td><td>org.apache.hadoop.hdfs.HsftpFileSystem</td><td></td>
 </tr>
 <tr>
 <td><a name="fs.ftp.impl">fs.ftp.impl</a></td><td>org.apache.hadoop.fs.ftp.FTPFileSystem</td><td>The FileSystem for ftp: uris.</td>

+ 1 - 1
src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/NewDriverWizardPage.java

@@ -260,4 +260,4 @@ public class NewDriverWizardPage extends NewTypeWizardPage {
 
     return text;
   }
-}
+}

+ 1 - 1
src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/dfs/DFSPath.java

@@ -22,7 +22,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.logging.Logger;
 
-import org.apache.hadoop.dfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.eclipse.ErrorMessageDialog;
 import org.apache.hadoop.eclipse.server.ConfProp;
 import org.apache.hadoop.eclipse.server.HadoopServer;

+ 1 - 1
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdater.java

@@ -147,4 +147,4 @@ public class IndexUpdater implements IIndexUpdater {
       }
     }
   }
-}
+}

+ 1 - 1
src/contrib/index/src/test/org/apache/hadoop/contrib/index/mapred/TestDistributionPolicy.java

@@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.contrib.index.example.HashingDistributionPolicy;
 import org.apache.hadoop.contrib.index.example.RoundRobinDistributionPolicy;
 import org.apache.hadoop.contrib.index.lucene.FileSystemDirectory;
-import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.MiniMRCluster;

+ 1 - 1
src/contrib/index/src/test/org/apache/hadoop/contrib/index/mapred/TestIndexUpdater.java

@@ -24,7 +24,7 @@ import java.text.NumberFormat;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.contrib.index.lucene.FileSystemDirectory;
-import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;

+ 1 - 1
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java

@@ -32,7 +32,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 
 /**
  * This class tests cacheArchive option of streaming 

+ 1 - 1
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleCachefiles.java

@@ -26,7 +26,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 /**
  * This test case tests the symlink creation
  * utility provided by distributed caching 

+ 1 - 1
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamedMerge.java

@@ -31,7 +31,7 @@ import java.util.Arrays;
 import junit.framework.TestCase;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;

+ 1 - 1
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestSymLink.java

@@ -26,7 +26,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 /**
  * This test case tests the symlink creation
  * utility provided by distributed caching 

+ 1 - 1
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestUlimit.java

@@ -23,7 +23,7 @@ import java.io.*;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;

+ 1 - 1
src/core/org/apache/hadoop/fs/FileSystem.java

@@ -27,11 +27,11 @@ import javax.security.auth.login.LoginException;
 
 import org.apache.commons.logging.*;
 
-import org.apache.hadoop.dfs.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.*;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.security.UserGroupInformation;
 

+ 1 - 1
src/core/org/apache/hadoop/fs/FsShell.java

@@ -43,7 +43,7 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.dfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 
 /** Provide command line access to a FileSystem. */
 public class FsShell extends Configured implements Tool {

+ 1 - 1
src/core/org/apache/hadoop/fs/Syncable.java

@@ -27,4 +27,4 @@ public interface Syncable {
    * @throws IOException
    */
   public void sync() throws IOException;
-}
+}

+ 1 - 1
src/core/org/apache/hadoop/fs/permission/AccessControlException.java

@@ -38,4 +38,4 @@ public class AccessControlException extends java.io.IOException {
    * @param s the detail message.
    */
   public AccessControlException(String s) {super(s);}
-}
+}

+ 1 - 1
src/core/org/apache/hadoop/fs/permission/FsAction.java

@@ -65,4 +65,4 @@ public enum FsAction {
   public FsAction not() {
     return values()[7 - INDEX];
   }
-}
+}

+ 1 - 1
src/core/org/apache/hadoop/fs/s3/FileSystemStore.java

@@ -60,4 +60,4 @@ public interface FileSystemStore {
    * @throws IOException
    */
   void dump() throws IOException;
-}
+}

+ 1 - 1
src/core/org/apache/hadoop/fs/s3native/Jets3tNativeFileSystemStore.java

@@ -242,4 +242,4 @@ class Jets3tNativeFileSystemStore implements NativeFileSystemStore {
     System.out.println(sb);
   }
   
-}
+}

+ 1 - 1
src/core/org/apache/hadoop/fs/shell/Command.java

@@ -81,4 +81,4 @@ abstract public class Command {
     }
     return exitCode;
   }
-}
+}

+ 1 - 1
src/core/org/apache/hadoop/fs/shell/CommandFormat.java

@@ -72,4 +72,4 @@ public class CommandFormat {
   public boolean getOpt(String option) {
     return options.get(option);
   }
-}
+}

+ 1 - 1
src/core/org/apache/hadoop/fs/shell/CommandUtils.java

@@ -25,4 +25,4 @@ final class CommandUtils {
     }
     return b.toString();
   }
-}
+}

+ 1 - 1
src/core/org/apache/hadoop/io/MultipleIOException.java

@@ -46,4 +46,4 @@ public class MultipleIOException extends IOException {
     }
     return new MultipleIOException(exceptions);
   }
-}
+}

+ 1 - 1
src/core/org/apache/hadoop/log/LogLevel.java

@@ -148,4 +148,4 @@ public class LogLevel {
       out.println(MARKER + "Effective level: <b>" + lev + "</b><br />");
     }
   }
-}
+}

+ 1 - 1
src/core/org/apache/hadoop/metrics/MetricsUtil.java

@@ -26,7 +26,7 @@ import org.apache.commons.logging.LogFactory;
 /**
  * Utility class to simplify creation and reporting of hadoop metrics.
  *
- * For examples of usage, see {@link org.apache.hadoop.dfs.DataNode}.
+ * For examples of usage, see {@link org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics}.
  * @see org.apache.hadoop.metrics.MetricsRecord
  * @see org.apache.hadoop.metrics.MetricsContext
  * @see org.apache.hadoop.metrics.ContextFactory

+ 4 - 2
src/hdfs/org/apache/hadoop/dfs/DataChecksum.java → src/core/org/apache/hadoop/util/DataChecksum.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.util;
 
 import java.util.zip.Checksum;
 import java.util.zip.CRC32;
@@ -205,8 +205,10 @@ public class DataChecksum implements Checksum {
   public int getNumBytesInSum() {
     return inSum;
   }
+  
+  public static final int SIZE_OF_INTEGER = Integer.SIZE / Byte.SIZE;
   static public int getChecksumHeaderSize() {
-    return 1 + FSConstants.SIZE_OF_INTEGER; // type byte, bytesPerChecksum int
+    return 1 + SIZE_OF_INTEGER; // type byte, bytesPerChecksum int
   }
   //Checksum Interface. Just a wrapper around member summer.
   public long getValue() {

+ 1 - 1
src/core/overview.html

@@ -132,7 +132,7 @@ following:
 
 <ol>
 
-<li>The {@link org.apache.hadoop.dfs.NameNode} (Distributed Filesystem
+<li>The {@link org.apache.hadoop.hdfs.server.namenode.NameNode} (Distributed Filesystem
 master) host.  This is specified with the configuration
 property <tt><a
 href="../hadoop-default.html#fs.default.name">fs.default.name</a></tt>.

+ 0 - 1066
src/hdfs/org/apache/hadoop/dfs/INode.java

@@ -1,1066 +0,0 @@
-/**
- * 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.dfs;
-
-import java.io.FileNotFoundException;
-import java.io.UnsupportedEncodingException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Arrays;
-import java.util.List;
-import java.io.IOException;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.permission.*;
-import org.apache.hadoop.dfs.BlocksMap.BlockInfo;
-
-/**
- * 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;
-
-  //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;
-  }
-
-  INode(PermissionStatus permissions, long mTime) {
-    this.name = null;
-    this.parent = null;
-    this.modificationTime = mTime;
-    setPermissionStatus(permissions);
-  }
-
-  protected INode(String name, PermissionStatus permissions) {
-    this(permissions, 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());
-  }
-
-  /**
-   * 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 */
-  protected 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 */
-  protected 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} */
-  protected 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
-   */
-  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}. */
-  final ContentSummary computeContentSummary() {
-    long[] a = computeContentSummary(new long[]{0,0,0});
-    return new ContentSummary(a[0], a[1], a[2], getQuota());
-  }
-  /**
-   * @return an array of three longs. 
-   * 0: length, 1: file count, 2: directory count
-   */
-  abstract long[] computeContentSummary(long[] summary);
-  
-  /**
-   * Get the quota set for this inode
-   * @return the quota if it is set; -1 otherwise
-   */
-  long getQuota() {
-    return -1;
-  }
-
-  /**
-   * Get the total number of names in the tree
-   * rooted at this inode including the root
-   * @return The total number of names in this tree
-   */
-  long numItemsInTree() {
-    return 1;
-  }
-    
-  /**
-   * 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
-   */
-  long getModificationTime() {
-    return this.modificationTime;
-  }
-
-  /**
-   * Set last modification time of inode.
-   */
-  void setModificationTime(long modtime) {
-    assert isDirectory();
-    if (this.modificationTime <= modtime) {
-      this.modificationTime = modtime;
-    }
-  }
-
-  /**
-   * 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;
-  }
-}
-
-/**
- * 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;
-  }
-
-  INodeDirectory(PermissionStatus permissions, long mTime) {
-    super(permissions, mTime);
-    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
-   */
-  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
-   */
-  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; 
-   *          inserted INode, 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;
-  }
-
-  /**
-   */
-  long numItemsInTree() {
-    long total = 1L;
-    if (children == null) {
-      return total;
-    }
-    for (INode child : children) {
-      total += child.numItemsInTree();
-    }
-    return total;
-  }
-
-  /** {@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;
-  }
-}
-
-/**
- * Directory INode class that has a quota restriction
- */
-class INodeDirectoryWithQuota extends INodeDirectory {
-  private long quota;
-  private long count;
-  
-  /** Convert an existing directory inode to one with the given quota
-   * 
-   * @param quota Quota to be assigned to this inode
-   * @param other The other inode from which all other properties are copied
-   */
-  INodeDirectoryWithQuota(long quota, INodeDirectory other)
-  throws QuotaExceededException {
-    super(other);
-    this.count = other.numItemsInTree();
-    setQuota(quota);
-  }
-  
-  /** constructor with no quota verification */
-  INodeDirectoryWithQuota(
-      PermissionStatus permissions, long modificationTime, long quota)
-  {
-    super(permissions, modificationTime);
-    this.quota = quota;
-  }
-  
-  /** constructor with no quota verification */
-  INodeDirectoryWithQuota(String name, PermissionStatus permissions, long quota)
-  {
-    super(name, permissions);
-    this.quota = quota;
-  }
-  
-  /** Get this directory's quota
-   * @return this directory's quota
-   */
-  long getQuota() {
-    return quota;
-  }
-  
-  /** Set this directory's quota
-   * 
-   * @param quota Quota to be set
-   * @throws QuotaExceededException if the given quota is less than 
-   *                                the size of the tree
-   */
-  void setQuota(long quota) throws QuotaExceededException {
-    verifyQuota(quota, this.count);
-    this.quota = quota;
-  }
-  
-  /** 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 count;
-  }
-  
-  /** Update the size of the tree
-   * 
-   * @param delta the change of the tree size
-   * @throws QuotaExceededException if the changed size is greater 
-   *                                than the quota
-   */
-  void updateNumItemsInTree(long delta) throws QuotaExceededException {
-    long newCount = this.count + delta;
-    if (delta>0) {
-      verifyQuota(this.quota, newCount);
-    }
-    this.count = newCount;
-  }
-  
-  /** Set the size of the tree rooted at this directory
-   * 
-   * @param count size of the directory to be set
-   * @throws QuotaExceededException if the given count is greater than quota
-   */
-  void setCount(long count) throws QuotaExceededException {
-    verifyQuota(this.quota, count);
-    this.count = count;
-  }
-  
-  /** Verify if the count satisfies the quota restriction 
-   * @throws QuotaExceededException if the given quota is less than the count
-   */
-  private static void verifyQuota(long quota, long count)
-  throws QuotaExceededException {
-    if (quota < count) {
-      throw new QuotaExceededException(quota, count);
-    }
-  }
-}
-
-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 preferredBlockSize) {
-    this(permissions, new BlockInfo[nrBlocks], replication,
-        modificationTime, preferredBlockSize);
-  }
-
-  protected INodeFile() {
-    blocks = null;
-    blockReplication = 0;
-    preferredBlockSize = 0;
-  }
-
-  protected INodeFile(PermissionStatus permissions, BlockInfo[] blklist,
-                      short replication, long modificationTime,
-                      long preferredBlockSize) {
-    super(permissions, modificationTime);
-    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));
-  }
-
-  boolean isDirectory() {
-    return false;
-  }
-
-  /**
-   * Get block replication for the file 
-   * @return block replication
-   */
-  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];
-      for (int i = 0; i < size; i++) {
-        newlist[i] = this.blocks[i];
-      }
-      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]++;
-    return summary;
-  }
-
-  /**
-   * Get the preferred block size of the file.
-   * @return the number of bytes
-   */
-  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);
-  }
-}
-
-class INodeFileUnderConstruction extends INodeFile {
-  StringBytesWritable clientName = null;         // lease holder
-  StringBytesWritable clientMachine = null;
-  DatanodeDescriptor clientNode = null; // 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
-  
-  INodeFileUnderConstruction() {}
-
-  INodeFileUnderConstruction(PermissionStatus permissions,
-                             short replication,
-                             long preferredBlockSize,
-                             long modTime,
-                             String clientName,
-                             String clientMachine,
-                             DatanodeDescriptor clientNode) 
-                             throws IOException {
-    super(permissions.applyUMask(UMASK), 0, replication, modTime,
-        preferredBlockSize);
-    this.clientName = new StringBytesWritable(clientName);
-    this.clientMachine = new StringBytesWritable(clientMachine);
-    this.clientNode = clientNode;
-  }
-
-  INodeFileUnderConstruction(byte[] name,
-                             short blockReplication,
-                             long modificationTime,
-                             long preferredBlockSize,
-                             BlockInfo[] blocks,
-                             PermissionStatus perm,
-                             String clientName,
-                             String clientMachine,
-                             DatanodeDescriptor clientNode)
-                             throws IOException {
-    super(perm, blocks, blockReplication, modificationTime, 
-          preferredBlockSize);
-    setLocalName(name);
-    this.clientName = new StringBytesWritable(clientName);
-    this.clientMachine = new StringBytesWritable(clientMachine);
-    this.clientNode = clientNode;
-  }
-
-  String getClientName() throws IOException {
-    return clientName.getString();
-  }
-
-  String getClientMachine() throws IOException {
-    return clientMachine.getString();
-  }
-
-  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
-  //
-  INodeFile convertToInodeFile() {
-    INodeFile obj = new INodeFile(getPermissionStatus(),
-                                  getBlocks(),
-                                  getReplication(),
-                                  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;
-  }
-
-  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);
-  }
-
-  /**
-   * 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.");
-      }
-    }
-  }
-}

+ 6 - 3
src/hdfs/org/apache/hadoop/dfs/ChecksumDistributedFileSystem.java → src/hdfs/org/apache/hadoop/hdfs/ChecksumDistributedFileSystem.java

@@ -16,13 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs;
 
 import java.io.*;
 import java.net.*;
 
 import org.apache.hadoop.fs.*;
-import org.apache.hadoop.dfs.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.conf.Configuration;
 
 /**
@@ -69,7 +72,7 @@ public class ChecksumDistributedFileSystem extends ChecksumFileSystem {
   /**
    * Enter, leave or get safe mode.
    *  
-   * @see org.apache.hadoop.dfs.ClientProtocol#setSafeMode(FSConstants.SafeModeAction)
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(FSConstants.SafeModeAction)
    */
   public boolean setSafeMode(FSConstants.SafeModeAction action) 
     throws IOException {

+ 16 - 11
src/hdfs/org/apache/hadoop/dfs/DFSClient.java → src/hdfs/org/apache/hadoop/hdfs/DFSClient.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.io.retry.RetryPolicies;
@@ -27,7 +27,12 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.dfs.DistributedFileSystem.DiskStatus;
+import org.apache.hadoop.hdfs.DistributedFileSystem.DiskStatus;
+import org.apache.hadoop.hdfs.protocol.*;
+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.UnixUserGroupInformation;
 import org.apache.hadoop.util.*;
 
@@ -56,11 +61,11 @@ import javax.security.auth.login.LoginException;
  * filesystem tasks.
  *
  ********************************************************/
-class DFSClient implements FSConstants {
+public class DFSClient implements FSConstants {
   public static final Log LOG = LogFactory.getLog(DFSClient.class);
-  static final int MAX_BLOCK_ACQUIRE_FAILURES = 3;
+  public static final int MAX_BLOCK_ACQUIRE_FAILURES = 3;
   private static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
-  final ClientProtocol namenode;
+  public final ClientProtocol namenode;
   final private ClientProtocol rpcNamenode;
   final UnixUserGroupInformation ugi;
   volatile boolean clientRunning = true;
@@ -83,11 +88,11 @@ class DFSClient implements FSConstants {
   private TreeMap<String, OutputStream> pendingCreates =
     new TreeMap<String, OutputStream>();
  
-  static ClientProtocol createNamenode(Configuration conf) throws IOException {
+  public static ClientProtocol createNamenode(Configuration conf) throws IOException {
     return createNamenode(NameNode.getAddress(conf), conf);
   }
 
-  static ClientProtocol createNamenode( InetSocketAddress nameNodeAddr,
+  public static ClientProtocol createNamenode( InetSocketAddress nameNodeAddr,
       Configuration conf) throws IOException {
     try {
       return createNamenode(createRPCNamenode(nameNodeAddr, conf,
@@ -809,7 +814,7 @@ class DFSClient implements FSConstants {
   /** This is a wrapper around connection to datadone
    * and understands checksum, offset etc
    */
-  static class BlockReader extends FSInputChecker {
+  public static class BlockReader extends FSInputChecker {
 
     private Socket dnSock; //for now just sending checksumOk.
     private DataInputStream in;
@@ -1034,14 +1039,14 @@ class DFSClient implements FSConstants {
       checksumSize = this.checksum.getChecksumSize();
     }
 
-    static BlockReader newBlockReader(Socket sock, String file, long blockId, 
+    public static BlockReader newBlockReader(Socket sock, String file, long blockId, 
         long genStamp, long startOffset, long len, int bufferSize) throws IOException {
       return newBlockReader(sock, file, blockId, genStamp, startOffset, len, bufferSize,
           true);
     }
 
     /** Java Doc required */
-    static BlockReader newBlockReader( Socket sock, String file, long blockId, 
+    public static BlockReader newBlockReader( Socket sock, String file, long blockId, 
                                        long genStamp,
                                        long startOffset, long len,
                                        int bufferSize, boolean verifyChecksum)
@@ -1099,7 +1104,7 @@ class DFSClient implements FSConstants {
     /** kind of like readFully(). Only reads as much as possible.
      * And allows use of protected readFully().
      */
-    int readAll(byte[] buf, int offset, int len) throws IOException {
+    public int readAll(byte[] buf, int offset, int len) throws IOException {
       return readFully(this, buf, offset, len);
     }
     

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

@@ -0,0 +1,51 @@
+/**
+ * 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;
+  }
+
+}
+

+ 18 - 10
src/hdfs/org/apache/hadoop/dfs/DistributedFileSystem.java → src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs;
 
 import java.io.*;
 import java.net.*;
@@ -24,10 +24,18 @@ 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.dfs.FSConstants.DatanodeReportType;
-import org.apache.hadoop.dfs.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
+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.util.*;
 
+
 /****************************************************************
  * Implementation of the abstract FileSystem for the DFS system.
  * This object is the way end-user code interacts with a Hadoop
@@ -91,7 +99,7 @@ public class DistributedFileSystem extends FileSystem {
 
   public void setWorkingDirectory(Path dir) {
     String result = makeAbsolute(dir).toUri().getPath();
-    if (!FSNamesystem.isValidName(result)) {
+    if (!DFSUtil.isValidName(result)) {
       throw new IllegalArgumentException("Invalid DFS directory name " + 
                                          result);
     }
@@ -106,7 +114,7 @@ public class DistributedFileSystem extends FileSystem {
   private String getPathName(Path file) {
     checkPath(file);
     String result = makeAbsolute(file).toUri().getPath();
-    if (!FSNamesystem.isValidName(result)) {
+    if (!DFSUtil.isValidName(result)) {
       throw new IllegalArgumentException("Pathname " + result + " from " +
                                          file+" is not a valid DFS filename.");
     }
@@ -183,16 +191,16 @@ public class DistributedFileSystem extends FileSystem {
   }
 
   /** Clear a directory's quota
-   * @see ClientProtocol#clearQuota(String)
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#clearQuota(String)
    */
-  void clearQuota(Path src) throws IOException {
+  public void clearQuota(Path src) throws IOException {
     dfs.clearQuota(getPathName(src));
   }
   
   /** Set a directory's quota
-   * @see ClientProtocol#setQuota(String, long) 
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long) 
    */
-  void setQuota(Path src, long quota) throws IOException {
+  public void setQuota(Path src, long quota) throws IOException {
     if (quota <= 0) {
       throw new IllegalArgumentException("Quota should be a positive number: "
           + quota);
@@ -287,7 +295,7 @@ public class DistributedFileSystem extends FileSystem {
   /**
    * Enter, leave or get safe mode.
    *  
-   * @see org.apache.hadoop.dfs.ClientProtocol#setSafeMode(
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(
    *    FSConstants.SafeModeAction)
    */
   public boolean setSafeMode(FSConstants.SafeModeAction action) 

+ 4 - 3
src/hdfs/org/apache/hadoop/dfs/HftpFileSystem.java → src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs;
 
 import java.io.FileNotFoundException;
 import java.io.InputStream;
@@ -50,6 +50,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 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.security.*;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.StringUtils;
@@ -57,8 +58,8 @@ import org.apache.hadoop.util.StringUtils;
 /** 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.dfs.ListPathsServlet
- * @see org.apache.hadoop.dfs.FileDataServlet
+ * @see org.apache.hadoop.hdfs.server.namenode.ListPathsServlet
+ * @see org.apache.hadoop.hdfs.server.namenode.FileDataServlet
  */
 public class HftpFileSystem extends FileSystem {
   static {

+ 3 - 3
src/hdfs/org/apache/hadoop/dfs/HsftpFileSystem.java → src/hdfs/org/apache/hadoop/hdfs/HsftpFileSystem.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
 
@@ -30,8 +30,8 @@ import org.apache.hadoop.fs.Path;
 /** 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.dfs.ListPathsServlet
- * @see org.apache.hadoop.dfs.FileDataServlet
+ * @see org.apache.hadoop.hdfs.server.namenode.ListPathsServlet
+ * @see org.apache.hadoop.hdfs.server.namenode.FileDataServlet
  */
 public class HsftpFileSystem extends HftpFileSystem {
 

+ 0 - 0
src/hdfs/org/apache/hadoop/dfs/package.html → src/hdfs/org/apache/hadoop/hdfs/package.html


+ 1 - 1
src/hdfs/org/apache/hadoop/dfs/AlreadyBeingCreatedException.java → src/hdfs/org/apache/hadoop/hdfs/protocol/AlreadyBeingCreatedException.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
 import java.io.IOException;
 

+ 38 - 28
src/hdfs/org/apache/hadoop/dfs/Block.java → src/hdfs/org/apache/hadoop/hdfs/protocol/Block.java

@@ -15,9 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
 import java.io.*;
+
+import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.io.*;
 
 /**************************************************
@@ -25,7 +27,7 @@ import org.apache.hadoop.io.*;
  * long.
  *
  **************************************************/
-class Block implements Writable, Comparable<Block> {
+public class Block implements Writable, Comparable<Block> {
 
   static {                                      // register a ctor
     WritableFactories.setFactory
@@ -37,11 +39,11 @@ class Block implements Writable, Comparable<Block> {
 
   // generation stamp of blocks that pre-date the introduction of
   // a generation stamp.
-  static final long GRANDFATHER_GENERATION_STAMP = 0;
+  public static final long GRANDFATHER_GENERATION_STAMP = 0;
 
   /**
    */
-  static boolean isBlockFilename(File f) {
+  public static boolean isBlockFilename(File f) {
     String name = f.getName();
     if ( name.startsWith( "blk_" ) && 
         name.indexOf( '.' ) < 0 ) {
@@ -55,19 +57,19 @@ class Block implements Writable, Comparable<Block> {
     return Long.parseLong(name.substring("blk_".length()));
   }
 
-  long blkid;
-  long len;
-  long generationStamp;
+  private long blockId;
+  private long numBytes;
+  private long generationStamp;
 
-  Block() {this(0, 0, 0);}
+  public Block() {this(0, 0, 0);}
 
-  Block(final long blkid, final long len, final long generationStamp) {
+  public Block(final long blkid, final long len, final long generationStamp) {
     set(blkid, len, generationStamp);
   }
 
-  Block(final long blkid) {this(blkid, 0, GenerationStamp.WILDCARD_STAMP);}
+  public Block(final long blkid) {this(blkid, 0, GenerationStamp.WILDCARD_STAMP);}
 
-  Block(Block blk) {this(blk.blkid, blk.len, blk.generationStamp);}
+  public Block(Block blk) {this(blk.blockId, blk.numBytes, blk.generationStamp);}
 
   /**
    * Find the blockid from the given filename
@@ -77,34 +79,42 @@ class Block implements Writable, Comparable<Block> {
   }
 
   public void set(long blkid, long len, long genStamp) {
-    this.blkid = blkid;
-    this.len = len;
+    this.blockId = blkid;
+    this.numBytes = len;
     this.generationStamp = genStamp;
   }
   /**
    */
   public long getBlockId() {
-    return blkid;
+    return blockId;
+  }
+  
+  public void setBlockId(long bid) {
+    blockId = bid;
   }
 
   /**
    */
   public String getBlockName() {
-    return "blk_" + String.valueOf(blkid);
+    return "blk_" + String.valueOf(blockId);
   }
 
   /**
    */
   public long getNumBytes() {
-    return len;
+    return numBytes;
   }
   public void setNumBytes(long len) {
-    this.len = len;
+    this.numBytes = len;
   }
 
-  long getGenerationStamp() {
+  public long getGenerationStamp() {
     return generationStamp;
   }
+  
+  public void setGenerationStamp(long stamp) {
+    generationStamp = stamp;
+  }
 
   /**
    */
@@ -116,17 +126,17 @@ class Block implements Writable, Comparable<Block> {
   // Writable
   /////////////////////////////////////
   public void write(DataOutput out) throws IOException {
-    out.writeLong(blkid);
-    out.writeLong(len);
+    out.writeLong(blockId);
+    out.writeLong(numBytes);
     out.writeLong(generationStamp);
   }
 
   public void readFields(DataInput in) throws IOException {
-    this.blkid = in.readLong();
-    this.len = in.readLong();
+    this.blockId = in.readLong();
+    this.numBytes = in.readLong();
     this.generationStamp = in.readLong();
-    if (len < 0) {
-      throw new IOException("Unexpected block size: " + len);
+    if (numBytes < 0) {
+      throw new IOException("Unexpected block size: " + numBytes);
     }
   }
 
@@ -146,9 +156,9 @@ class Block implements Writable, Comparable<Block> {
     validateGenerationStamp(this.generationStamp);
     validateGenerationStamp(b.generationStamp);
 
-    if (blkid < b.blkid) {
+    if (blockId < b.blockId) {
       return -1;
-    } else if (blkid == b.blkid) {
+    } else if (blockId == b.blockId) {
       return GenerationStamp.compare(generationStamp, b.generationStamp);
     } else {
       return 1;
@@ -162,7 +172,7 @@ class Block implements Writable, Comparable<Block> {
     }
     final Block that = (Block)o;
     //Wildcard generationStamp is ALLOWED here
-    return this.blkid == that.blkid
+    return this.blockId == that.blockId
       && GenerationStamp.equalsWithWildcard(
           this.generationStamp, that.generationStamp);
   }
@@ -170,6 +180,6 @@ class Block implements Writable, Comparable<Block> {
   /** {@inheritDoc} */
   public int hashCode() {
     //GenerationStamp is IRRELEVANT and should not be used here
-    return 37 * 17 + (int) (blkid^(blkid>>>32));
+    return 37 * 17 + (int) (blockId^(blockId>>>32));
   }
 }

+ 8 - 8
src/hdfs/org/apache/hadoop/dfs/BlockListAsLongs.java → src/hdfs/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
 /**
  * This class provides an interface for accessing list of blocks that
@@ -24,7 +24,7 @@ package org.apache.hadoop.dfs;
  * as a Block[] we can send it as a long[].
  *
  */
-class BlockListAsLongs {
+public class BlockListAsLongs {
   /**
    * A block as 3 longs
    *   block-id and block length and generation stamp
@@ -49,7 +49,7 @@ class BlockListAsLongs {
    * @return the output array of long[]
    */
   
-  static long[] convertToArrayLongs(final Block[] blockArray) {
+  public static long[] convertToArrayLongs(final Block[] blockArray) {
     long[] blocksAsLongs = new long[blockArray.length * LONGS_PER_BLOCK];
 
     BlockListAsLongs bl = new BlockListAsLongs(blocksAsLongs);
@@ -65,7 +65,7 @@ class BlockListAsLongs {
    * Constructor
    * @param iBlockList - BlockListALongs create from this long[] parameter
    */
-  BlockListAsLongs(final long[] iBlockList) {
+  public BlockListAsLongs(final long[] iBlockList) {
     if (iBlockList == null) {
       blockList = new long[0];
     } else {
@@ -82,7 +82,7 @@ class BlockListAsLongs {
    * The number of blocks
    * @return - the number of blocks
    */
-  int getNumberOfBlocks() {
+  public int getNumberOfBlocks() {
     return blockList.length/LONGS_PER_BLOCK;
   }
   
@@ -92,7 +92,7 @@ class BlockListAsLongs {
    * @param index - the block whose block-id is desired
    * @return the block-id
    */
-  long getBlockId(final int index)  {
+  public long getBlockId(final int index)  {
     return blockList[index2BlockId(index)];
   }
   
@@ -101,7 +101,7 @@ class BlockListAsLongs {
    * @param index - the block whose block-len is desired
    * @return - the block-len
    */
-  long getBlockLen(final int index)  {
+  public long getBlockLen(final int index)  {
     return blockList[index2BlockLen(index)];
   }
 
@@ -110,7 +110,7 @@ class BlockListAsLongs {
    * @param index - the block whose block-len is desired
    * @return - the generation stamp
    */
-  long getBlockGenStamp(final int index)  {
+  public long getBlockGenStamp(final int index)  {
     return blockList[index2BlockGenStamp(index)];
   }
   

+ 7 - 7
src/hdfs/org/apache/hadoop/dfs/BlocksWithLocations.java → src/hdfs/org/apache/hadoop/hdfs/protocol/BlocksWithLocations.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -29,12 +29,12 @@ import org.apache.hadoop.io.WritableUtils;
  *  It provide efficient customized serialization/deserialization methods
  *  in stead of using the default array (de)serialization provided by RPC
  */
-class BlocksWithLocations implements Writable {
+public class BlocksWithLocations implements Writable {
 
   /**
    * A class to keep track of a block and its locations
    */
-  static class BlockWithLocations  implements Writable {
+  public static class BlockWithLocations  implements Writable {
     Block block;
     String datanodeIDs[];
     
@@ -51,12 +51,12 @@ class BlocksWithLocations implements Writable {
     }
     
     /** get the block */
-    Block getBlock() {
+    public Block getBlock() {
       return block;
     }
     
     /** get the block's locations */
-    String[] getDatanodes() {
+    public String[] getDatanodes() {
       return datanodeIDs;
     }
     
@@ -87,12 +87,12 @@ class BlocksWithLocations implements Writable {
   }
 
   /** Constructor with one parameter */
-  BlocksWithLocations( BlockWithLocations[] blocks ) {
+  public BlocksWithLocations( BlockWithLocations[] blocks ) {
     this.blocks = blocks;
   }
 
   /** getter */
-  BlockWithLocations[] getBlocks() {
+  public BlockWithLocations[] getBlocks() {
     return blocks;
   }
 

+ 3 - 3
src/hdfs/org/apache/hadoop/dfs/ClientDatanodeProtocol.java → src/hdfs/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
 import java.io.IOException;
 
@@ -25,7 +25,7 @@ import org.apache.hadoop.ipc.VersionedProtocol;
 
 /** An client-datanode protocol for block recovery
  */
-interface ClientDatanodeProtocol extends VersionedProtocol {
+public interface ClientDatanodeProtocol extends VersionedProtocol {
   public static final Log LOG = LogFactory.getLog(ClientDatanodeProtocol.class);
 
   /**
@@ -35,7 +35,7 @@ interface ClientDatanodeProtocol extends VersionedProtocol {
 
   /** Start generation-stamp recovery for specified block
    * @param block the specified block
-   * @param DatanodeInfo the list of possible locations of specified block
+   * @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.

+ 6 - 4
src/hdfs/org/apache/hadoop/dfs/ClientProtocol.java → src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -15,23 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
 import java.io.*;
 
 import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.hadoop.dfs.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.fs.ContentSummary;
 
 /**********************************************************************
  * ClientProtocol is used by user code via 
- * {@link DistributedFileSystem} class to communicate 
+ * {@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.
  *
  **********************************************************************/
-interface ClientProtocol extends VersionedProtocol {
+public interface ClientProtocol extends VersionedProtocol {
 
   /**
    * Compared to the previous version the following changes have been introduced:

+ 4 - 2
src/hdfs/org/apache/hadoop/dfs/DFSFileInfo.java → src/hdfs/org/apache/hadoop/hdfs/protocol/DFSFileInfo.java

@@ -15,8 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileStatus;
@@ -29,7 +31,7 @@ import org.apache.hadoop.fs.FileStatus;
  * Block locations are sorted by the distance to the current client.
  * 
  ******************************************************/
-class DFSFileInfo extends FileStatus {
+public class DFSFileInfo extends FileStatus {
   static {                                      // register a ctor
     WritableFactories.setFactory
       (DFSFileInfo.class,

+ 8 - 8
src/hdfs/org/apache/hadoop/dfs/DatanodeID.java → src/hdfs/org/apache/hadoop/hdfs/protocol/DatanodeID.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -32,12 +32,12 @@ import org.apache.hadoop.io.WritableComparable;
  * 
  */
 public class DatanodeID implements WritableComparable {
-  static final DatanodeID[] EMPTY_ARRAY = {}; 
+  public static final DatanodeID[] EMPTY_ARRAY = {}; 
 
-  protected String name;      /// hostname:portNumber
-  protected String storageID; /// unique per cluster storageID
+  public String name;      /// hostname:portNumber
+  public String storageID; /// unique per cluster storageID
   protected int infoPort;     /// the port where the infoserver is running
-  protected int ipcPort;     /// the port where the ipc server is running
+  public int ipcPort;     /// the port where the ipc server is running
 
   /** Equivalent to DatanodeID(""). */
   public DatanodeID() {this("");}
@@ -101,9 +101,9 @@ public class DatanodeID implements WritableComparable {
   }
 
   /**
-   * @sets data storage ID.
+   * sets the data storage ID.
    */
-  void setStorageID(String storageID) {
+  public void setStorageID(String storageID) {
     this.storageID = storageID;
   }
 
@@ -150,7 +150,7 @@ public class DatanodeID implements WritableComparable {
    * Update fields when a new registration request comes in.
    * Note that this does not update storageID.
    */
-  void updateRegInfo(DatanodeID nodeReg) {
+  public void updateRegInfo(DatanodeID nodeReg) {
     name = nodeReg.getName();
     infoPort = nodeReg.getInfoPort();
     // update any more fields added in future.

+ 16 - 16
src/hdfs/org/apache/hadoop/dfs/DatanodeInfo.java → src/hdfs/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -55,12 +55,12 @@ public class DatanodeInfo extends DatanodeID implements Node {
   protected AdminStates adminState;
 
 
-  DatanodeInfo() {
+  public DatanodeInfo() {
     super();
     adminState = null;
   }
   
-  DatanodeInfo(DatanodeInfo from) {
+  public DatanodeInfo(DatanodeInfo from) {
     super(from);
     this.capacity = from.getCapacity();
     this.dfsUsed = from.getDfsUsed();
@@ -72,7 +72,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.hostName = from.hostName;
   }
 
-  DatanodeInfo(DatanodeID nodeID) {
+  public DatanodeInfo(DatanodeID nodeID) {
     super(nodeID);
     this.capacity = 0L;
     this.dfsUsed = 0L;
@@ -82,7 +82,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.adminState = null;    
   }
   
-  DatanodeInfo(DatanodeID nodeID, String location, String hostName) {
+  protected DatanodeInfo(DatanodeID nodeID, String location, String hostName) {
     this(nodeID);
     this.location = location;
     this.hostName = hostName;
@@ -104,22 +104,22 @@ public class DatanodeInfo extends DatanodeID implements Node {
   public int getXceiverCount() { return xceiverCount; }
 
   /** Sets raw capacity. */
-  void setCapacity(long capacity) { 
+  public void setCapacity(long capacity) { 
     this.capacity = capacity; 
   }
 
   /** Sets raw free space. */
-  void setRemaining(long remaining) { 
+  public void setRemaining(long remaining) { 
     this.remaining = remaining; 
   }
 
   /** Sets time when this information was accurate. */
-  void setLastUpdate(long lastUpdate) { 
+  public void setLastUpdate(long lastUpdate) { 
     this.lastUpdate = lastUpdate; 
   }
 
   /** Sets number of active connections */
-  void setXceiverCount(int xceiverCount) { 
+  public void setXceiverCount(int xceiverCount) { 
     this.xceiverCount = xceiverCount; 
   }
 
@@ -166,7 +166,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   }
 
   /** A formatted string for printing the status of the DataNode. */
-  String dumpDatanode() {
+  public String dumpDatanode() {
     StringBuffer buffer = new StringBuffer();
     long c = getCapacity();
     long r = getRemaining();
@@ -195,7 +195,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
    * Start decommissioning a node.
    * old state.
    */
-  void startDecommission() {
+  public void startDecommission() {
     adminState = AdminStates.DECOMMISSION_INPROGRESS;
   }
 
@@ -203,14 +203,14 @@ public class DatanodeInfo extends DatanodeID implements Node {
    * Stop decommissioning a node.
    * old state.
    */
-  void stopDecommission() {
+  public void stopDecommission() {
     adminState = null;
   }
 
   /**
    * Returns true if the node is in the process of being decommissioned
    */
-  boolean isDecommissionInProgress() {
+  public boolean isDecommissionInProgress() {
     if (adminState == AdminStates.DECOMMISSION_INPROGRESS) {
       return true;
     }
@@ -220,7 +220,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   /**
    * Returns true if the node has been decommissioned.
    */
-  boolean isDecommissioned() {
+  public boolean isDecommissioned() {
     if (adminState == AdminStates.DECOMMISSIONED) {
       return true;
     }
@@ -230,7 +230,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   /**
    * Sets the admin state to indicate that decommision is complete.
    */
-  void setDecommissioned() {
+  public void setDecommissioned() {
     adminState = AdminStates.DECOMMISSIONED;
   }
 
@@ -247,7 +247,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   /**
    * Sets the admin state of this node.
    */
-  void setAdminState(AdminStates newState) {
+  protected void setAdminState(AdminStates newState) {
     if (newState == AdminStates.NORMAL) {
       adminState = null;
     }

+ 3 - 3
src/hdfs/org/apache/hadoop/dfs/FSConstants.java → src/hdfs/org/apache/hadoop/hdfs/protocol/FSConstants.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
 import org.apache.hadoop.conf.Configuration;
 
@@ -134,7 +134,7 @@ public interface FSConstants {
     
   public static final int BUFFER_SIZE = new Configuration().getInt("io.file.buffer.size", 4096);
   //Used for writing header etc.
-  static final int SMALL_BUFFER_SIZE = Math.min(BUFFER_SIZE/2, 512);
+  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;
@@ -155,7 +155,7 @@ public interface FSConstants {
     
     private String name = null;
     private StartupOption(String arg) {this.name = arg;}
-    String getName() {return name;}
+    public String getName() {return name;}
   }
 
   // type of the datanode report

+ 6 - 6
src/hdfs/org/apache/hadoop/dfs/LocatedBlock.java → src/hdfs/org/apache/hadoop/hdfs/protocol/LocatedBlock.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
 import org.apache.hadoop.io.*;
 
@@ -26,7 +26,7 @@ import java.io.*;
  * objects.  It tells where to find a Block.
  * 
  ****************************************************/
-class LocatedBlock implements Writable {
+public class LocatedBlock implements Writable {
 
   static {                                      // register a ctor
     WritableFactories.setFactory
@@ -84,15 +84,15 @@ class LocatedBlock implements Writable {
 
   /**
    */
-  DatanodeInfo[] getLocations() {
+  public DatanodeInfo[] getLocations() {
     return locs;
   }
   
-  long getStartOffset() {
+  public long getStartOffset() {
     return offset;
   }
   
-  long getBlockSize() {
+  public long getBlockSize() {
     return b.getNumBytes();
   }
 
@@ -104,7 +104,7 @@ class LocatedBlock implements Writable {
     this.corrupt = corrupt;
   }
   
-  boolean isCorrupt() {
+  public boolean isCorrupt() {
     return this.corrupt;
   }
 

+ 8 - 7
src/hdfs/org/apache/hadoop/dfs/LocatedBlocks.java → src/hdfs/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -43,10 +43,11 @@ public class LocatedBlocks implements Writable {
     underConstruction = false;
   }
   
-  LocatedBlocks(INodeFile inode, List<LocatedBlock> blks) {
-    fileLength = inode.computeContentSummary().getLength();
+  public LocatedBlocks(long flength, List<LocatedBlock> blks, boolean isUnderConstuction) {
+
+    fileLength = flength;
     blocks = blks;
-    underConstruction = inode.isUnderConstruction();
+    underConstruction = isUnderConstuction;
   }
   
   /**
@@ -90,7 +91,7 @@ public class LocatedBlocks implements Writable {
    * 
    * @return block if found, or null otherwise.
    */
-  int findBlock(long offset) {
+  public int findBlock(long offset) {
     // create fake block of size 1 as a key
     LocatedBlock key = new LocatedBlock();
     key.setStartOffset(offset);
@@ -114,7 +115,7 @@ public class LocatedBlocks implements Writable {
     return Collections.binarySearch(blocks, key, comp);
   }
   
-  void insertRange(int blockIdx, List<LocatedBlock> newBlocks) {
+  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(); 
@@ -142,7 +143,7 @@ public class LocatedBlocks implements Writable {
     }
   }
   
-  static int getInsertIndex(int binSearchResult) {
+  public static int getInsertIndex(int binSearchResult) {
     return binSearchResult >= 0 ? binSearchResult : -(binSearchResult+1);
   }
 

+ 1 - 1
src/hdfs/org/apache/hadoop/dfs/QuotaExceededException.java → src/hdfs/org/apache/hadoop/hdfs/protocol/QuotaExceededException.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
 import java.io.IOException;
 

+ 2 - 2
src/hdfs/org/apache/hadoop/dfs/UnregisteredDatanodeException.java → src/hdfs/org/apache/hadoop/hdfs/protocol/UnregisteredDatanodeException.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.protocol;
 
 import java.io.IOException;
 
@@ -26,7 +26,7 @@ import java.io.IOException;
  * registered is trying to access the name node.
  * 
  */
-class UnregisteredDatanodeException extends IOException {
+public class UnregisteredDatanodeException extends IOException {
 
   public UnregisteredDatanodeException(DatanodeID nodeID) {
     super("Unregistered data node: " + nodeID.getName());

+ 13 - 8
src/hdfs/org/apache/hadoop/dfs/Balancer.java → src/hdfs/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.balancer;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
@@ -51,8 +51,13 @@ 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.dfs.BlocksWithLocations.BlockWithLocations;
-import org.apache.hadoop.dfs.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.common.Util;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
@@ -171,7 +176,7 @@ import org.apache.hadoop.util.ToolRunner;
 
 public class Balancer implements Tool {
   private static final Log LOG = 
-    LogFactory.getLog("org.apache.hadoop.dfs.Balancer");
+    LogFactory.getLog(Balancer.class.getName());
   final private static long MAX_BLOCKS_SIZE_TO_FETCH = 2*1024*1024*1024L; //2GB
 
   private Configuration conf;
@@ -724,7 +729,7 @@ public class Balancer implements Tool {
      */ 
     private static final long MAX_ITERATION_TIME = 20*60*1000L; //20 mins
     private void dispatchBlocks() {
-      long startTime = FSNamesystem.now();
+      long startTime = Util.now();
       this.blocksToReceive = 2*scheduledSize;
       boolean isTimeUp = false;
       while(!isTimeUp && scheduledSize>0 &&
@@ -753,7 +758,7 @@ public class Balancer implements Tool {
         } 
         
         // check if time is up or not
-        if (FSNamesystem.now()-startTime > MAX_ITERATION_TIME) {
+        if (Util.now()-startTime > MAX_ITERATION_TIME) {
           isTimeUp = true;
           continue;
         }
@@ -1365,7 +1370,7 @@ public class Balancer implements Tool {
    * @exception any exception occurs during datanode balancing
    */
   public int run(String[] args) throws Exception {
-    long startTime = FSNamesystem.now();
+    long startTime = Util.now();
     OutputStream out = null;
     try {
       // initialize a balancer
@@ -1463,7 +1468,7 @@ public class Balancer implements Tool {
       } catch(IOException ignored) {
       }
       System.out.println("Balancing took " + 
-          time2Str(FSNamesystem.now()-startTime));
+          time2Str(Util.now()-startTime));
     }
   }
 

+ 5 - 5
src/hdfs/org/apache/hadoop/dfs/GenerationStamp.java → src/hdfs/org/apache/hadoop/hdfs/server/common/GenerationStamp.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.common;
 
 import java.io.*;
 import org.apache.hadoop.io.*;
@@ -23,7 +23,7 @@ import org.apache.hadoop.io.*;
 /****************************************************************
  * A GenerationStamp is a Hadoop FS primitive, identified by a long.
  ****************************************************************/
-class GenerationStamp implements WritableComparable<GenerationStamp> {
+public class GenerationStamp implements WritableComparable<GenerationStamp> {
   public static final long WILDCARD_STAMP = 1;
   public static final long FIRST_VALID_STAMP = 1000L;
 
@@ -40,7 +40,7 @@ class GenerationStamp implements WritableComparable<GenerationStamp> {
   /**
    * Create a new instance, initialized to FIRST_VALID_STAMP.
    */
-  GenerationStamp() {this(GenerationStamp.FIRST_VALID_STAMP);}
+  public GenerationStamp() {this(GenerationStamp.FIRST_VALID_STAMP);}
 
   /**
    * Create a new instance, initialized to the specified value.
@@ -86,7 +86,7 @@ class GenerationStamp implements WritableComparable<GenerationStamp> {
   /////////////////////////////////////
   // Comparable
   /////////////////////////////////////
-  static int compare(long x, long y) {
+  public static int compare(long x, long y) {
     return x < y? -1: x == y? 0: 1;
   }
 
@@ -103,7 +103,7 @@ class GenerationStamp implements WritableComparable<GenerationStamp> {
     return genstamp == ((GenerationStamp)o).genstamp;
   }
 
-  static boolean equalsWithWildcard(long x, long y) {
+  public static boolean equalsWithWildcard(long x, long y) {
     return x == y || x == WILDCARD_STAMP || y == WILDCARD_STAMP;  
   }
 

+ 10 - 6
src/hdfs/org/apache/hadoop/dfs/GenerationStampStatsUpgradeCommand.java → src/hdfs/org/apache/hadoop/hdfs/server/common/GenerationStampStatsUpgradeCommand.java

@@ -15,19 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.common;
 
 import java.io.*;
 
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.namenode.GenerationStampUpgradeNamenode;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+
 /**
  * The Datanode sends this statistics object to the Namenode periodically
  * during a Generation Stamp Upgrade.
  */
-class GenerationStampStatsUpgradeCommand extends UpgradeCommand {
-  DatanodeID datanodeId;
-  int blocksUpgraded;
-  int blocksRemaining;
-  int errors;
+public class GenerationStampStatsUpgradeCommand extends UpgradeCommand {
+  public DatanodeID datanodeId;
+  public int blocksUpgraded;
+  public int blocksRemaining;
+  public int errors;
 
   GenerationStampStatsUpgradeCommand() {
     super(GenerationStampUpgradeNamenode.DN_CMD_STATS, 0, (short)0);

+ 2 - 2
src/hdfs/org/apache/hadoop/dfs/InconsistentFSStateException.java → src/hdfs/org/apache/hadoop/hdfs/server/common/InconsistentFSStateException.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.common;
 
 import java.io.File;
 import java.io.IOException;
@@ -26,7 +26,7 @@ import org.apache.hadoop.util.StringUtils;
  * and is not recoverable. 
  * 
  */
-class InconsistentFSStateException extends IOException {
+public class InconsistentFSStateException extends IOException {
 
   public InconsistentFSStateException(File dir, String descr) {
     super("Directory " + getFilePath(dir)

+ 4 - 2
src/hdfs/org/apache/hadoop/dfs/IncorrectVersionException.java → src/hdfs/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java

@@ -15,16 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+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 appication.
  * 
  */
-class IncorrectVersionException extends IOException {
+public class IncorrectVersionException extends IOException {
 
   public IncorrectVersionException(int versionReported, String ofWhat) {
     this(versionReported, ofWhat, FSConstants.LAYOUT_VERSION);

+ 41 - 73
src/hdfs/org/apache/hadoop/dfs/Storage.java → src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.common;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -31,46 +31,14 @@ import java.util.Properties;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.dfs.FSConstants.StartupOption;
-import org.apache.hadoop.dfs.FSConstants.NodeType;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+import org.apache.hadoop.hdfs.protocol.FSConstants.NodeType;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
 
-/**
- * Common class for storage information.
- * 
- * TODO namespaceID should be long and computed as hash(address + port)
- */
-class StorageInfo {
-  int   layoutVersion;  // Version read from the stored file.
-  int   namespaceID;    // namespace id of the storage
-  long  cTime;          // creation timestamp
-  
-  StorageInfo () {
-    this(0, 0, 0L);
-  }
-  
-  StorageInfo(int layoutV, int nsID, long cT) {
-    layoutVersion = layoutV;
-    namespaceID = nsID;
-    cTime = cT;
-  }
-  
-  StorageInfo(StorageInfo from) {
-    setStorageInfo(from);
-  }
 
-  public int    getLayoutVersion(){ return layoutVersion; }
-  public int    getNamespaceID()  { return namespaceID; }
-  public long   getCTime()        { return cTime; }
-
-  public void   setStorageInfo(StorageInfo from) {
-    layoutVersion = from.layoutVersion;
-    namespaceID = from.namespaceID;
-    cTime = from.cTime;
-  }
-}
 
 /**
  * Storage information file.
@@ -90,8 +58,8 @@ class StorageInfo {
  * The locks are released when the servers stop (normally or abnormally).
  * 
  */
-abstract class Storage extends StorageInfo {
-  public static final Log LOG = LogFactory.getLog("org.apache.hadoop.dfs.Storage");
+public abstract class Storage extends StorageInfo {
+  public static final Log LOG = LogFactory.getLog(Storage.class.getName());
 
   // Constants
   
@@ -99,12 +67,12 @@ abstract class Storage extends StorageInfo {
   protected static final int LAST_PRE_UPGRADE_LAYOUT_VERSION = -3;
   
   // this corresponds to Hadoop-0.14.
-  protected static final int LAST_UPGRADABLE_LAYOUT_VERSION = -7;
+  public static final int LAST_UPGRADABLE_LAYOUT_VERSION = -7;
   protected static final String LAST_UPGRADABLE_HADOOP_VERSION = "Hadoop-0.14";
   
   private   static final String STORAGE_FILE_LOCK     = "in_use.lock";
   protected static final String STORAGE_FILE_VERSION  = "VERSION";
-  protected   static final String STORAGE_DIR_CURRENT   = "current";
+  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";
@@ -112,7 +80,7 @@ abstract class Storage extends StorageInfo {
   private   static final String STORAGE_TMP_LAST_CKPT = "lastcheckpoint.tmp";
   private   static final String STORAGE_PREVIOUS_CKPT = "previous.checkpoint";
   
-  protected enum StorageState {
+  public enum StorageState {
     NON_EXISTENT,
     NOT_FORMATTED,
     COMPLETE_UPGRADE,
@@ -131,11 +99,11 @@ abstract class Storage extends StorageInfo {
   /**
    * One of the storage directories.
    */
-  class StorageDirectory {
-    File              root; // root directory
+  public class StorageDirectory {
+    public File              root; // root directory
     FileLock          lock; // storage lock
     
-    StorageDirectory(File dir) {
+    public StorageDirectory(File dir) {
       this.root = dir;
       this.lock = null;
     }
@@ -145,11 +113,11 @@ abstract class Storage extends StorageInfo {
      * 
      * @throws IOException if file cannot be read or contains inconsistent data
      */
-    void read() throws IOException {
+    public void read() throws IOException {
       read(getVersionFile());
     }
     
-    void read(File from) throws IOException {
+    public void read(File from) throws IOException {
       RandomAccessFile file = new RandomAccessFile(from, "rws");
       FileInputStream in = null;
       try {
@@ -171,12 +139,12 @@ abstract class Storage extends StorageInfo {
      * 
      * @throws IOException
      */
-    void write() throws IOException {
+    public void write() throws IOException {
       corruptPreUpgradeStorage(root);
       write(getVersionFile());
     }
 
-    void write(File to) throws IOException {
+    public void write(File to) throws IOException {
       Properties props = new Properties();
       setFields(props, this);
       RandomAccessFile file = new RandomAccessFile(to, "rws");
@@ -219,7 +187,7 @@ abstract class Storage extends StorageInfo {
      * 
      * @throws IOException
      */
-    void clearDirectory() throws IOException {
+    public void clearDirectory() throws IOException {
       File curDir = this.getCurrentDir();
       if (curDir.exists())
         if (!(FileUtil.fullyDelete(curDir)))
@@ -228,31 +196,31 @@ abstract class Storage extends StorageInfo {
         throw new IOException("Cannot create directory " + curDir);
     }
 
-    File getCurrentDir() {
+    public File getCurrentDir() {
       return new File(root, STORAGE_DIR_CURRENT);
     }
-    File getVersionFile() {
+    public File getVersionFile() {
       return new File(new File(root, STORAGE_DIR_CURRENT), STORAGE_FILE_VERSION);
     }
-    File getPreviousVersionFile() {
+    public File getPreviousVersionFile() {
       return new File(new File(root, STORAGE_DIR_PREVIOUS), STORAGE_FILE_VERSION);
     }
-    File getPreviousDir() {
+    public File getPreviousDir() {
       return new File(root, STORAGE_DIR_PREVIOUS);
     }
-    File getPreviousTmp() {
+    public File getPreviousTmp() {
       return new File(root, STORAGE_TMP_PREVIOUS);
     }
-    File getRemovedTmp() {
+    public File getRemovedTmp() {
       return new File(root, STORAGE_TMP_REMOVED);
     }
-    File getFinalizedTmp() {
+    public File getFinalizedTmp() {
       return new File(root, STORAGE_TMP_FINALIZED);
     }
-    File getLastCheckpointTmp() {
+    public File getLastCheckpointTmp() {
       return new File(root, STORAGE_TMP_LAST_CKPT);
     }
-    File getPreviousCheckpoint() {
+    public File getPreviousCheckpoint() {
       return new File(root, STORAGE_PREVIOUS_CKPT);
     }
 
@@ -265,7 +233,7 @@ abstract class Storage extends StorageInfo {
      * @throws {@link InconsistentFSStateException} if directory state is not 
      * consistent and cannot be recovered 
      */
-    StorageState analyzeStorage(StartupOption startOpt) throws IOException {
+    public StorageState analyzeStorage(StartupOption startOpt) throws IOException {
       assert root != null : "root is null";
       String rootPath = root.getCanonicalPath();
       try { // check that storage exists
@@ -372,7 +340,7 @@ abstract class Storage extends StorageInfo {
      * @param curState specifies what/how the state should be recovered
      * @throws IOException
      */
-    void doRecover(StorageState curState) throws IOException {
+    public void doRecover(StorageState curState) throws IOException {
       File curDir = getCurrentDir();
       String rootPath = root.getCanonicalPath();
       switch(curState) {
@@ -434,7 +402,7 @@ abstract class Storage extends StorageInfo {
      * 
      * @throws IOException if locking fails
      */
-    void lock() throws IOException {
+    public void lock() throws IOException {
       this.lock = tryLock();
       if (lock == null) {
         String msg = "Cannot lock storage " + this.root 
@@ -474,7 +442,7 @@ abstract class Storage extends StorageInfo {
      * 
      * @throws IOException
      */
-    void unlock() throws IOException {
+    public void unlock() throws IOException {
       if (this.lock == null)
         return;
       this.lock.release();
@@ -486,26 +454,26 @@ abstract class Storage extends StorageInfo {
   /**
    * Create empty storage info of the specified type
    */
-  Storage(NodeType type) {
+  protected Storage(NodeType type) {
     super();
     this.storageType = type;
   }
   
-  Storage(NodeType type, int nsID, long cT) {
+  protected Storage(NodeType type, int nsID, long cT) {
     super(FSConstants.LAYOUT_VERSION, nsID, cT);
     this.storageType = type;
   }
   
-  Storage(NodeType type, StorageInfo storageInfo) {
+  protected Storage(NodeType type, StorageInfo storageInfo) {
     super(storageInfo);
     this.storageType = type;
   }
   
-  int getNumStorageDirs() {
+  public int getNumStorageDirs() {
     return storageDirs.size();
   }
   
-  StorageDirectory getStorageDir(int idx) {
+  public StorageDirectory getStorageDir(int idx) {
     return storageDirs.get(idx);
   }
   
@@ -513,7 +481,7 @@ abstract class Storage extends StorageInfo {
     storageDirs.add(sd);
   }
   
-  abstract boolean isConversionNeeded(StorageDirectory sd) throws IOException;
+  public abstract boolean isConversionNeeded(StorageDirectory sd) throws IOException;
 
   /*
    * Coversion is no longer supported. So this should throw exception if
@@ -532,7 +500,7 @@ abstract class Storage extends StorageInfo {
    * 
    * @param oldVersion
    */
-  static void checkVersionUpgradable(int oldVersion) 
+  protected static void checkVersionUpgradable(int oldVersion) 
                                      throws IOException {
     if (oldVersion > LAST_UPGRADABLE_LAYOUT_VERSION) {
       String msg = "*********** Upgrade is not supported from this older" +
@@ -601,13 +569,13 @@ abstract class Storage extends StorageInfo {
     props.setProperty("cTime", String.valueOf(cTime));
   }
 
-  static void rename(File from, File to) throws IOException {
+  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());
   }
 
-  static void deleteDir(File dir) throws IOException {
+  protected static void deleteDir(File dir) throws IOException {
     if (!FileUtil.fullyDelete(dir))
       throw new IOException("Failed to delete " + dir.getCanonicalPath());
   }
@@ -641,7 +609,7 @@ abstract class Storage extends StorageInfo {
    * @throws IOException
    * @see StorageDirectory#lock()
    */
-  boolean isLockSupported(int idx) throws IOException {
+  public boolean isLockSupported(int idx) throws IOException {
     StorageDirectory sd = storageDirs.get(idx);
     FileLock firstLock = null;
     FileLock secondLock = null;
@@ -672,7 +640,7 @@ abstract class Storage extends StorageInfo {
     return VersionInfo.getRevision();
   }
 
-  static String getRegistrationID(StorageInfo storage) {
+  public static String getRegistrationID(StorageInfo storage) {
     return "NS-" + Integer.toString(storage.getNamespaceID())
       + "-" + Integer.toString(storage.getLayoutVersion())
       + "-" + Long.toString(storage.getCTime());

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

@@ -0,0 +1,37 @@
+package org.apache.hadoop.hdfs.server.common;
+
+
+/**
+ * Common class for storage information.
+ * 
+ * TODO namespaceID should be long and computed as hash(address + port)
+ */
+public class StorageInfo {
+  public int   layoutVersion;  // Version read from the stored file.
+  public int   namespaceID;    // namespace id of the storage
+  public long  cTime;          // creation timestamp
+  
+  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);
+  }
+
+  public int    getLayoutVersion(){ return layoutVersion; }
+  public int    getNamespaceID()  { return namespaceID; }
+  public long   getCTime()        { return cTime; }
+
+  public void   setStorageInfo(StorageInfo from) {
+    layoutVersion = from.layoutVersion;
+    namespaceID = from.namespaceID;
+    cTime = from.cTime;
+  }
+}

+ 16 - 13
src/hdfs/org/apache/hadoop/dfs/UpgradeManager.java → src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeManager.java

@@ -15,52 +15,55 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+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 
  *
  */
-abstract class UpgradeManager {
+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;
 
-  synchronized UpgradeCommand getBroadcastCommand() {
+  public synchronized UpgradeCommand getBroadcastCommand() {
     return this.broadcastCommand;
   }
 
-  boolean getUpgradeState() {
+  public boolean getUpgradeState() {
     return this.upgradeState;
   }
 
-  int getUpgradeVersion(){
+  public int getUpgradeVersion(){
     return this.upgradeVersion;
   }
 
-  void setUpgradeState(boolean uState, int uVersion) {
+  public void setUpgradeState(boolean uState, int uVersion) {
     this.upgradeState = uState;
     this.upgradeVersion = uVersion;
   }
 
-  SortedSet<Upgradeable> getDistributedUpgrades() throws IOException {
+  public SortedSet<Upgradeable> getDistributedUpgrades() throws IOException {
     return UpgradeObjectCollection.getDistributedUpgrades(
                                             getUpgradeVersion(), getType());
   }
 
-  short getUpgradeStatus() {
+  public short getUpgradeStatus() {
     if(currentUpgrades == null)
       return 100;
     return currentUpgrades.first().getUpgradeStatus();
   }
 
-  boolean initializeUpgrade() throws IOException {
+  public boolean initializeUpgrade() throws IOException {
     currentUpgrades = getDistributedUpgrades();
     if(currentUpgrades == null) {
       // set new upgrade state
@@ -73,14 +76,14 @@ abstract class UpgradeManager {
     return true;
   }
 
-  boolean isUpgradeCompleted() {
+  public boolean isUpgradeCompleted() {
     if (currentUpgrades == null) {
       return true;
     }
     return false;
   }
 
-  abstract FSConstants.NodeType getType();
-  abstract boolean startUpgrade() throws IOException;
-  abstract void completeUpgrade() throws IOException;
+  public abstract FSConstants.NodeType getType();
+  public abstract boolean startUpgrade() throws IOException;
+  public abstract void completeUpgrade() throws IOException;
 }

+ 3 - 3
src/hdfs/org/apache/hadoop/dfs/UpgradeObject.java → src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeObject.java

@@ -15,11 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.common;
 
 import java.io.IOException;
 
-import org.apache.hadoop.dfs.UpgradeObjectCollection.UOSignature;
+import org.apache.hadoop.hdfs.server.common.UpgradeObjectCollection.UOSignature;
 
 /**
  * Abstract upgrade object.
@@ -27,7 +27,7 @@ import org.apache.hadoop.dfs.UpgradeObjectCollection.UOSignature;
  * Contains default implementation of common methods of {@link Upgradeable}
  * interface.
  */
-abstract class UpgradeObject implements Upgradeable {
+public abstract class UpgradeObject implements Upgradeable {
   protected short status;
   
   public short getUpgradeStatus() {

+ 6 - 3
src/hdfs/org/apache/hadoop/dfs/UpgradeObjectCollection.java → src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeObjectCollection.java

@@ -15,12 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+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.hdfs.server.datanode.GenerationStampUpgradeDatanode;
+import org.apache.hadoop.hdfs.server.namenode.GenerationStampUpgradeNamenode;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -28,7 +31,7 @@ import org.apache.hadoop.util.StringUtils;
  *
  * Upgrade objects should be registered here before they can be used. 
  */
-class UpgradeObjectCollection {
+public class UpgradeObjectCollection {
   static {
     initialize();
     // Registered distributed upgrade objects here
@@ -109,7 +112,7 @@ class UpgradeObjectCollection {
     upgradeTable.add(new UOSignature(uo));
   }
 
-  static SortedSet<Upgradeable> getDistributedUpgrades(int versionFrom, 
+  public static SortedSet<Upgradeable> getDistributedUpgrades(int versionFrom, 
                                                        FSConstants.NodeType type
                                                        ) throws IOException {
     assert FSConstants.LAYOUT_VERSION <= versionFrom : "Incorrect version " 

+ 1 - 1
src/hdfs/org/apache/hadoop/dfs/UpgradeStatusReport.java → src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeStatusReport.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.common;
 
 import java.io.DataInput;
 import java.io.DataOutput;

+ 4 - 1
src/hdfs/org/apache/hadoop/dfs/Upgradeable.java → src/hdfs/org/apache/hadoop/hdfs/server/common/Upgradeable.java

@@ -15,10 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.common;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+
 /**
  * Common interface for distributed upgrade objects.
  * 

+ 28 - 0
src/hdfs/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();
+  }
+}

+ 2 - 1
src/hdfs/org/apache/hadoop/dfs/BlockMetadataHeader.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.BufferedInputStream;
 import java.io.DataInputStream;
@@ -25,6 +25,7 @@ import java.io.FileInputStream;
 import java.io.IOException;
 
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.DataChecksum;
 
 
 /**

+ 6 - 3
src/hdfs/org/apache/hadoop/dfs/DataBlockScanner.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.BufferedReader;
 import java.io.Closeable;
@@ -46,7 +46,10 @@ 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.dfs.DataNode.BlockSender;
+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.hdfs.server.datanode.DataNode.BlockSender;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.StringUtils;
 
@@ -268,7 +271,7 @@ public class DataBlockScanner implements Runnable {
   }
 
   /** @return the last scan time */
-  synchronized long getLastScanTime(Block block) {
+  public synchronized long getLastScanTime(Block block) {
     BlockScanInfo info = blockMap.get(block);
     return info == null? 0: info.lastScanTime;
   }

+ 46 - 27
src/hdfs/org/apache/hadoop/dfs/DataNode.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.datanode;
 
 import org.apache.commons.logging.*;
 
@@ -31,11 +31,30 @@ import org.apache.hadoop.util.*;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.mapred.StatusHttpServer;
-import org.apache.hadoop.dfs.BlockCommand;
-import org.apache.hadoop.dfs.DatanodeProtocol;
-import org.apache.hadoop.dfs.FSDatasetInterface.MetaDataInputStream;
-import org.apache.hadoop.dfs.datanode.metrics.DataNodeMetrics;
-import org.apache.hadoop.dfs.BlockMetadataHeader;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.UnregisteredDatanodeException;
+import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
+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.InterDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.MetaDataInputStream;
+import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 
 import java.io.*;
 import java.net.*;
@@ -81,7 +100,7 @@ import java.security.SecureRandom;
  **********************************************************/
 public class DataNode extends Configured 
     implements InterDatanodeProtocol, ClientDatanodeProtocol, FSConstants, Runnable {
-  public static final Log LOG = LogFactory.getLog("org.apache.hadoop.dfs.DataNode");
+  public static final Log LOG = LogFactory.getLog(DataNode.class.getName());
 
   /**
    * Use {@link NetUtils#createSocketAddr(String)} instead.
@@ -99,14 +118,14 @@ public class DataNode extends Configured
    */
   private static final int MIN_BUFFER_WITH_TRANSFERTO = 64*1024;
   
-  DatanodeProtocol namenode = null;
-  FSDatasetInterface data = null;
-  DatanodeRegistration dnRegistration = null;
+  public DatanodeProtocol namenode = null;
+  public FSDatasetInterface data = null;
+  public DatanodeRegistration dnRegistration = null;
 
   volatile boolean shouldRun = true;
   private LinkedList<Block> receivedBlockList = new LinkedList<Block>();
   private LinkedList<String> delHints = new LinkedList<String>();
-  final static String EMPTY_DEL_HINT = "";
+  public final static String EMPTY_DEL_HINT = "";
   int xmitsInProgress = 0;
   Daemon dataXceiveServer = null;
   ThreadGroup threadGroup = null;
@@ -132,8 +151,8 @@ public class DataNode extends Configured
   private boolean transferToAllowed = true;
   private int writePacketSize = 0;
   
-  DataBlockScanner blockScanner = null;
-  Daemon blockScannerThread = null;
+  public DataBlockScanner blockScanner = null;
+  public Daemon blockScannerThread = null;
   
   private static final Random R = new Random();
 
@@ -148,13 +167,13 @@ public class DataNode extends Configured
   private long estimateBlockSize;
   
   // The following three fields are to support balancing
-  final static short MAX_BALANCING_THREADS = 5;
+  public final static short MAX_BALANCING_THREADS = 5;
   private Semaphore balancingSem = new Semaphore(MAX_BALANCING_THREADS);
   long balanceBandwidth;
   private Throttler balancingThrottler;
 
   // For InterDataNodeProtocol
-  Server ipcServer;
+  public Server ipcServer;
   
   // Record all sockets opend for data transfer
   Map<Socket, Socket> childSockets = Collections.synchronizedMap(
@@ -259,7 +278,7 @@ public class DataNode extends Configured
           //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.dfs.SimulatedFSDataset"), conf);
+              Class.forName("org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset"), conf);
         } catch (ClassNotFoundException e) {
           throw new IOException(StringUtils.stringifyException(e));
         }
@@ -409,7 +428,7 @@ public class DataNode extends Configured
     return datanodeObject;
   } 
 
-  static InterDatanodeProtocol createInterDataNodeProtocolProxy(
+  public static InterDatanodeProtocol createInterDataNodeProtocolProxy(
       DatanodeID datanodeid, Configuration conf) throws IOException {
     InetSocketAddress addr = NetUtils.createSocketAddr(
         datanodeid.getHost() + ":" + datanodeid.getIpcPort());
@@ -440,7 +459,7 @@ public class DataNode extends Configured
     return "<namenode>";
   }
 
-  static void setNewStorageID(DatanodeRegistration dnReg) {
+  public static void setNewStorageID(DatanodeRegistration dnReg) {
     /* Return 
      * "DS-randInt-ipaddr-currentTimeMillis"
      * It is considered extermely rare for all these numbers to match
@@ -1468,7 +1487,7 @@ public class DataNode extends Configured
    * This class is thread safe. It can be shared by multiple threads.
    * The parameter bandwidthPerSec specifies the total bandwidth shared by threads.
    */
-  static class Throttler {
+  public static class Throttler {
     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
@@ -1479,7 +1498,7 @@ public class DataNode extends Configured
     /** Constructor 
      * @param bandwidthPerSec bandwidth allowed in bytes per second. 
      */
-    Throttler(long bandwidthPerSec) {
+    public Throttler(long bandwidthPerSec) {
       this(500, bandwidthPerSec);  // by default throttling period is 500ms 
     }
 
@@ -1636,7 +1655,7 @@ public class DataNode extends Configured
    ************************************************************************ */
   
   /** Header size for a packet */
-  static final int PKT_HEADER_LEN = ( 4 + /* Packet payload length */
+  public static final int PKT_HEADER_LEN = ( 4 + /* Packet payload length */
                                       8 + /* offset in block */
                                       8 + /* seqno */
                                       1   /* isLastPacketInBlock */);
@@ -2903,7 +2922,7 @@ public class DataNode extends Configured
   /** Start a single datanode daemon and wait for it to finish.
    *  If this thread is specifically interrupted, it will stop waiting.
    */
-  static void runDatanodeDaemon(DataNode dn) throws IOException {
+  public static void runDatanodeDaemon(DataNode dn) throws IOException {
     if (dn != null) {
       //register datanode
       dn.register();
@@ -2916,7 +2935,7 @@ public class DataNode extends Configured
   /** Instantiate a single datanode object. This must be run by invoking
    *  {@link DataNode#runDatanodeDaemon(DataNode)} subsequently. 
    */
-  static DataNode instantiateDataNode(String args[],
+  public static DataNode instantiateDataNode(String args[],
                                       Configuration conf) throws IOException {
     if (conf == null)
       conf = new Configuration();
@@ -2938,7 +2957,7 @@ public class DataNode extends Configured
   /** Instantiate & Start a single datanode daemon and wait for it to finish.
    *  If this thread is specifically interrupted, it will stop waiting.
    */
-  static DataNode createDataNode(String args[],
+  public static DataNode createDataNode(String args[],
                                  Configuration conf) throws IOException {
     DataNode dn = instantiateDataNode(args, conf);
     runDatanodeDaemon(dn);
@@ -2964,7 +2983,7 @@ public class DataNode extends Configured
    * no directory from this directory list can be created.
    * @throws IOException
    */
-  static DataNode makeInstance(String[] dataDirs, Configuration conf)
+  public static DataNode makeInstance(String[] dataDirs, Configuration conf)
     throws IOException {
     ArrayList<File> dirs = new ArrayList<File>();
     for (int i = 0; i < dataDirs.length; i++) {
@@ -3078,12 +3097,12 @@ public class DataNode extends Configured
     if (LOG.isDebugEnabled()) {
       LOG.debug("block=" + block);
     }
-    Block stored = data.getStoredBlock(block.blkid);
+    Block stored = data.getStoredBlock(block.getBlockId());
     return stored == null?
         null: new BlockMetaDataInfo(stored, blockScanner.getLastScanTime(stored));
   }
 
-  Daemon recoverBlocks(final Block[] blocks, final DatanodeInfo[][] targets) {
+  public Daemon recoverBlocks(final Block[] blocks, final DatanodeInfo[][] targets) {
     Daemon d = new Daemon(threadGroup, new Runnable() {
       public void run() {
         LeaseManager.recoverBlocks(blocks, targets, namenode, getConf());

+ 12 - 7
src/hdfs/org/apache/hadoop/dfs/DataStorage.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -29,8 +29,13 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Properties;
 
-import org.apache.hadoop.dfs.FSConstants.StartupOption;
-import org.apache.hadoop.dfs.FSConstants.NodeType;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.protocol.FSConstants.NodeType;
+import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+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.protocol.NamespaceInfo;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.fs.FileUtil.HardLink;
 import org.apache.hadoop.io.IOUtils;
@@ -40,7 +45,7 @@ import org.apache.hadoop.io.IOUtils;
  * <p>
  * @see Storage
  */
-class DataStorage extends Storage {
+public class DataStorage extends Storage {
   // Constants
   final static String BLOCK_SUBDIR_PREFIX = "subdir";
   final static String BLOCK_FILE_PREFIX = "blk_";
@@ -58,12 +63,12 @@ class DataStorage extends Storage {
     this.storageID = strgID;
   }
   
-  DataStorage(StorageInfo storageInfo, String strgID) {
+  public DataStorage(StorageInfo storageInfo, String strgID) {
     super(NodeType.DATA_NODE, storageInfo);
     this.storageID = strgID;
   }
 
-  String getStorageID() {
+  public String getStorageID() {
     return storageID;
   }
   
@@ -177,7 +182,7 @@ class DataStorage extends Storage {
       storageID = ssid;
   }
 
-  boolean isConversionNeeded(StorageDirectory sd) throws IOException {
+  public boolean isConversionNeeded(StorageDirectory sd) throws IOException {
     File oldF = new File(sd.root, "storage");
     if (!oldF.exists())
       return false;

+ 4 - 2
src/hdfs/org/apache/hadoop/dfs/DatanodeBlockInfo.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/DatanodeBlockInfo.java

@@ -15,13 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+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.dfs.FSDataset.FSVolume;
+
+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;

+ 23 - 18
src/hdfs/org/apache/hadoop/dfs/FSDataset.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.*;
 import java.util.*;
@@ -25,19 +25,24 @@ 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.FSConstants;
 import org.apache.hadoop.metrics.util.MBeanUtil;
+import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.dfs.datanode.metrics.FSDatasetMBean;
+import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+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.
  *
  ***************************************************/
-class FSDataset implements FSConstants, FSDatasetInterface {
+public class FSDataset implements FSConstants, FSDatasetInterface {
 
 
   /**
@@ -602,8 +607,8 @@ class FSDataset implements FSConstants, FSDatasetInterface {
     }
   }
 
-  File findBlockFile(Block b) {
-    assert b.generationStamp == GenerationStamp.WILDCARD_STAMP;
+  public File findBlockFile(Block b) {
+    assert b.getGenerationStamp() == GenerationStamp.WILDCARD_STAMP;
 
     File blockfile = null;
     ActiveFile activefile = ongoingCreates.get(b);
@@ -630,8 +635,8 @@ class FSDataset implements FSConstants, FSDatasetInterface {
       return null;
     }
     File metafile = findMetaFile(blockfile);
-    b.generationStamp = parseGenerationStamp(blockfile, metafile);
-    b.len = blockfile.length();
+    b.setGenerationStamp(parseGenerationStamp(blockfile, metafile));
+    b.setNumBytes(blockfile.length());
     return b;
   }
 
@@ -703,7 +708,7 @@ class FSDataset implements FSConstants, FSDatasetInterface {
   /**
    * Get File name for a given block.
    */
-  protected synchronized File getBlockFile(Block b) throws IOException {
+  public synchronized File getBlockFile(Block b) throws IOException {
     File f = validateBlockFile(b);
     if(f == null) {
       if (InterDatanodeProtocol.LOG.isDebugEnabled()) {
@@ -738,12 +743,12 @@ class FSDataset implements FSConstants, FSDatasetInterface {
    * 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 b Block
+   * @param block Block
    * @param numLinks Detach if the number of links exceed this value
    * @throws IOException
    * @return - true if the specified block was detached
    */
-  boolean detachBlock(Block block, int numLinks) throws IOException {
+  public boolean detachBlock(Block block, int numLinks) throws IOException {
     DatanodeBlockInfo info = null;
 
     synchronized (this) {
@@ -801,19 +806,19 @@ class FSDataset implements FSConstants, FSDatasetInterface {
     }
 
     //update generation stamp
-    if (oldgs > newblock.generationStamp) {
-      throw new IOException("Cannot update block (id=" + newblock.blkid
+    if (oldgs > newblock.getGenerationStamp()) {
+      throw new IOException("Cannot update block (id=" + newblock.getBlockId()
           + ") generation stamp from " + oldgs
-          + " to " + newblock.generationStamp);
+          + " to " + newblock.getGenerationStamp());
     }
     
     //update length
-    if (newblock.len > oldblock.len) {
+    if (newblock.getNumBytes() > oldblock.getNumBytes()) {
       throw new IOException("Cannot update block file (=" + blockFile
-          + ") length from " + oldblock.len + " to " + newblock.len);
+          + ") length from " + oldblock.getNumBytes() + " to " + newblock.getNumBytes());
     }
-    if (newblock.len < oldblock.len) {
-      truncateBlock(blockFile, tmpMetaFile, oldblock.len, newblock.len);
+    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)
@@ -1146,7 +1151,7 @@ class FSDataset implements FSConstants, FSDatasetInterface {
   /**
    * Turn the block identifier into a filename.
    */
-  synchronized File getFile(Block b) {
+  public synchronized File getFile(Block b) {
     DatanodeBlockInfo info = volumeMap.get(b);
     if (info != null) {
       return info.getFile();

+ 3 - 2
src/hdfs/org/apache/hadoop/dfs/FSDatasetInterface.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.datanode;
 
 
 import java.io.FilterInputStream;
@@ -26,7 +26,8 @@ import java.io.OutputStream;
 
 
 
-import org.apache.hadoop.dfs.datanode.metrics.FSDatasetMBean;
+import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 
 /**

+ 10 - 4
src/hdfs/org/apache/hadoop/dfs/GenerationStampUpgradeDatanode.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/GenerationStampUpgradeDatanode.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.*;
 import java.util.*;
@@ -29,6 +29,12 @@ import java.net.InetSocketAddress;
 import java.net.SocketTimeoutException;
 
 import org.apache.commons.logging.*;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.GenerationStampStatsUpgradeCommand;
+import org.apache.hadoop.hdfs.server.namenode.GenerationStampUpgradeNamenode;
+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.io.Text;
 import org.apache.hadoop.io.retry.*;
 import org.apache.hadoop.util.StringUtils;
@@ -41,10 +47,10 @@ import org.apache.hadoop.ipc.RPC;
  * generation stamp is written to each metadata file. Please see
  * HADOOP-1700 for details.
  */
-class GenerationStampUpgradeDatanode extends UpgradeObjectDatanode {
+public class GenerationStampUpgradeDatanode extends UpgradeObjectDatanode {
 
   public static final Log LOG = 
-    LogFactory.getLog("org.apache.hadoop.dfs.GenerationStampUpgrade");
+    LogFactory.getLog(GenerationStampUpgradeDatanode.class.getName());
 
   DatanodeProtocol namenode;
   InetSocketAddress namenodeAddr;
@@ -223,7 +229,7 @@ class GenerationStampUpgradeDatanode extends UpgradeObjectDatanode {
   // This method iterates through all the blocks on a datanode and
   // do the upgrade.
   //
-  void doUpgrade() throws IOException {
+  public void doUpgrade() throws IOException {
     
     if (upgradeCompleted.get()) {
       assert offlineUpgrade.get() : 

+ 7 - 3
src/hdfs/org/apache/hadoop/dfs/UpgradeManagerDatanode.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java

@@ -15,10 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+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.UpgradeManager;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.util.Daemon;
 
 /**
@@ -63,7 +67,7 @@ class UpgradeManagerDatanode extends UpgradeManager {
    * @return true if distributed upgrade is required or false otherwise
    * @throws IOException
    */
-  synchronized boolean startUpgrade() throws IOException {
+  public synchronized boolean startUpgrade() throws IOException {
     if(upgradeState) {  // upgrade is already in progress
       assert currentUpgrades != null : 
         "UpgradeManagerDatanode.currentUpgrades is null.";
@@ -125,7 +129,7 @@ class UpgradeManagerDatanode extends UpgradeManager {
         + "The upgrade object is not defined.");
   }
 
-  synchronized void completeUpgrade() throws IOException {
+  public synchronized void completeUpgrade() throws IOException {
     assert currentUpgrades != null : 
       "UpgradeManagerDatanode.currentUpgrades is null.";
     UpgradeObjectDatanode curUO = (UpgradeObjectDatanode)currentUpgrades.first();

+ 8 - 3
src/hdfs/org/apache/hadoop/dfs/UpgradeObjectDatanode.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java

@@ -15,8 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.datanode;
 
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+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;
@@ -25,7 +30,7 @@ import java.net.SocketTimeoutException;
  * Base class for data-node upgrade objects.
  * Data-node upgrades are run in separate threads.
  */
-abstract class UpgradeObjectDatanode extends UpgradeObject implements Runnable {
+public abstract class UpgradeObjectDatanode extends UpgradeObject implements Runnable {
   private DataNode dataNode = null;
 
   public FSConstants.NodeType getType() {
@@ -44,7 +49,7 @@ abstract class UpgradeObjectDatanode extends UpgradeObject implements Runnable {
    * Specifies how the upgrade is performed. 
    * @throws IOException
    */
-  abstract void doUpgrade() throws IOException;
+  public abstract void doUpgrade() throws IOException;
 
   /**
    * Specifies what to do before the upgrade is started.

+ 1 - 1
src/hdfs/org/apache/hadoop/dfs/datanode/metrics/DataNodeMetrics.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.datanode.metrics;
+package org.apache.hadoop.hdfs.server.datanode.metrics;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics.MetricsContext;

+ 1 - 1
src/hdfs/org/apache/hadoop/dfs/datanode/metrics/DataNodeStatistics.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeStatistics.java

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

+ 2 - 2
src/hdfs/org/apache/hadoop/dfs/datanode/metrics/DataNodeStatisticsMBean.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeStatisticsMBean.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs.datanode.metrics;
+package org.apache.hadoop.hdfs.server.datanode.metrics;
 
 /**
  * 
@@ -39,7 +39,7 @@ package org.apache.hadoop.dfs.datanode.metrics;
  * The context with the update thread is used to average the data periodically.
  * <p>
  * Name Node Status info is reported in another MBean
- * @see org.apache.hadoop.dfs.datanode.metrics.FSDatasetMBean
+ * @see org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean
  *
  */
 public interface DataNodeStatisticsMBean {

+ 2 - 2
src/hdfs/org/apache/hadoop/dfs/datanode/metrics/FSDatasetMBean.java → src/hdfs/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.datanode.metrics;
+package org.apache.hadoop.hdfs.server.datanode.metrics;
 
 import java.io.IOException;
 
@@ -27,7 +27,7 @@ import java.io.IOException;
  * convention.) 
  * <p>
  * Data Node runtime statistic  info is report in another MBean
- * @see org.apache.hadoop.dfs.datanode.metrics.DataNodeStatisticsMBean
+ * @see org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeStatisticsMBean
  *
  */
 public interface FSDatasetMBean {

+ 6 - 4
src/hdfs/org/apache/hadoop/dfs/BlocksMap.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java

@@ -15,21 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+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 {
+public class BlocksMap {
         
   /**
    * Internal class for block metadata.
    */
-  static class BlockInfo extends Block {
+  public static class BlockInfo extends Block {
     private INodeFile          inode;
 
     /**
@@ -42,7 +44,7 @@ class BlocksMap {
      */
     private Object[] triplets;
 
-    BlockInfo(Block blk, int replication) {
+    public BlockInfo(Block blk, int replication) {
       super(blk);
       this.triplets = new Object[3*replication];
       this.inode = null;

+ 3 - 2
src/hdfs/org/apache/hadoop/dfs/CheckpointSignature.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java

@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+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.io.WritableComparable;
 
 /**
  * A unique signature intended to identify checkpoint transactions.
  */
-class CheckpointSignature extends StorageInfo 
+public class CheckpointSignature extends StorageInfo 
                       implements WritableComparable<CheckpointSignature> {
   private static final String FIELD_SEPARATOR = ":";
   long editsTime = -1L;

+ 4 - 3
src/hdfs/org/apache/hadoop/dfs/CorruptReplicasMap.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/CorruptReplicasMap.java

@@ -15,8 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.namenode;
 
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.ipc.Server;
 
 import java.util.*;
@@ -30,7 +31,7 @@ import java.util.*;
  * Mapping: Block -> TreeSet<DatanodeDescriptor> 
  */
 
-class CorruptReplicasMap{
+public class CorruptReplicasMap{
 
   private Map<Block, Collection<DatanodeDescriptor>> corruptReplicasMap =
     new TreeMap<Block, Collection<DatanodeDescriptor>>();
@@ -99,7 +100,7 @@ class CorruptReplicasMap{
     return ((nodes != null) && (nodes.contains(node)));
   }
 
-  int numCorruptReplicas(Block blk) {
+  public int numCorruptReplicas(Block blk) {
     Collection<DatanodeDescriptor> nodes = getNodes(blk);
     return (nodes == null) ? 0 : nodes.size();
   }

+ 12 - 6
src/hdfs/org/apache/hadoop/dfs/DatanodeDescriptor.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java

@@ -15,13 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.DataInput;
 import java.io.IOException;
 import java.util.*;
 
-import org.apache.hadoop.dfs.BlocksMap.BlockInfo;
+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.UTF8;
 import org.apache.hadoop.io.WritableUtils;
@@ -39,9 +45,9 @@ import org.apache.hadoop.io.WritableUtils;
  **************************************************/
 public class DatanodeDescriptor extends DatanodeInfo {
   /** Block and targets pair */
-  static class BlockTargetPair {
-    final Block block;
-    final DatanodeDescriptor[] targets;    
+  public static class BlockTargetPair {
+    public final Block block;
+    public final DatanodeDescriptor[] targets;    
 
     BlockTargetPair(Block block, DatanodeDescriptor[] targets) {
       this.block = block;
@@ -198,7 +204,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this.blockList = null;
   }
 
-  int numBlocks() {
+  public int numBlocks() {
     return blockList == null ? 0 : blockList.listCount(this);
   }
 

+ 4 - 2
src/hdfs/org/apache/hadoop/dfs/DfsServlet.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 
@@ -25,6 +25,8 @@ import javax.servlet.http.*;
 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.ipc.RemoteException;
 import org.apache.hadoop.security.*;
 import org.znerd.xmlenc.XMLOutputter;
@@ -74,4 +76,4 @@ abstract class DfsServlet extends HttpServlet {
     doc.attribute("message", msg.substring(msg.indexOf(":") + 1).trim());
     doc.endTag();
   }
-}
+}

+ 6 - 2
src/hdfs/org/apache/hadoop/dfs/FSDirectory.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.*;
 import java.util.*;
@@ -27,7 +27,11 @@ 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.dfs.BlocksMap.BlockInfo;
+import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
+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.namenode.BlocksMap.BlockInfo;
 
 /*************************************************
  * FSDirectory stores the filesystem directory state.

+ 14 - 9
src/hdfs/org/apache/hadoop/dfs/FSEditLog.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.BufferedInputStream;
 import java.io.DataInput;
@@ -32,6 +32,11 @@ import java.util.ArrayList;
 import java.lang.Math;
 import java.nio.channels.FileChannel;
 
+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.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.fs.permission.*;
 
@@ -39,7 +44,7 @@ import org.apache.hadoop.fs.permission.*;
  * FSEditLog maintains a log of the namespace modifications.
  * 
  */
-class FSEditLog {
+public class FSEditLog {
   private static final byte OP_ADD = 0;
   private static final byte OP_RENAME = 1;  // rename
   private static final byte OP_DELETE = 2;  // delete
@@ -244,7 +249,7 @@ class FSEditLog {
    * 
    * @throws IOException
    */
-  synchronized void open() throws IOException {
+  public synchronized void open() throws IOException {
     numTransactions = totalTimeTransactions = 0;
     int size = getNumStorageDirs();
     if (editStreams == null)
@@ -262,7 +267,7 @@ class FSEditLog {
     }
   }
 
-  synchronized void createEditLogFile(File name) throws IOException {
+  public synchronized void createEditLogFile(File name) throws IOException {
     EditLogOutputStream eStream = new EditLogOutputStream(name);
     eStream.create();
     eStream.close();
@@ -282,7 +287,7 @@ class FSEditLog {
   /**
    * Shutdown the filestore
    */
-  synchronized void close() throws IOException {
+  public synchronized void close() throws IOException {
     while (isSyncRunning) {
       try {
         wait(1000);
@@ -740,7 +745,7 @@ class FSEditLog {
   //
   // Sync all modifications done by this thread.
   //
-  void logSync() {
+  public void logSync() {
     ArrayList<EditLogOutputStream> errorStreams = null;
     long syncStart = 0;
 
@@ -840,7 +845,7 @@ class FSEditLog {
    * Add open lease record to edit log. 
    * Records the block locations of the last block.
    */
-  void logOpenFile(String path, INodeFileUnderConstruction newNode) 
+  public void logOpenFile(String path, INodeFileUnderConstruction newNode) 
                    throws IOException {
 
     UTF8 nameReplicationPair[] = new UTF8[] { 
@@ -859,7 +864,7 @@ class FSEditLog {
   /** 
    * Add close lease record to edit log.
    */
-  void logCloseFile(String path, INodeFile newNode) {
+  public void logCloseFile(String path, INodeFile newNode) {
     UTF8 nameReplicationPair[] = new UTF8[] {
       new UTF8(path),
       FSEditLog.toLogReplication(newNode.getReplication()),
@@ -874,7 +879,7 @@ class FSEditLog {
   /** 
    * Add create directory record to edit log
    */
-  void logMkDir(String path, INode newNode) {
+  public void logMkDir(String path, INode newNode) {
     UTF8 info[] = new UTF8[] {
       new UTF8(path),
       FSEditLog.toLogLong(newNode.getModificationTime())

+ 20 - 12
src/hdfs/org/apache/hadoop/dfs/FSImage.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
@@ -43,18 +43,26 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.dfs.FSConstants.CheckpointStates;
-import org.apache.hadoop.dfs.FSConstants.StartupOption;
-import org.apache.hadoop.dfs.FSConstants.NodeType;
+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.protocol.FSConstants.CheckpointStates;
+import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+import org.apache.hadoop.hdfs.protocol.FSConstants.NodeType;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.dfs.BlocksMap.BlockInfo;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
+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;
 
 /**
  * FSImage handles checkpointing and logging of the namespace edits.
  * 
  */
-class FSImage extends Storage {
+public class FSImage extends Storage {
 
   private static final SimpleDateFormat DATE_FORM =
     new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
@@ -107,14 +115,14 @@ class FSImage extends Storage {
     setStorageDirectories(fsDirs);
   }
 
-  FSImage(StorageInfo storageInfo) {
+  public FSImage(StorageInfo storageInfo) {
     super(NodeType.NAME_NODE, storageInfo);
   }
 
   /**
    * Represents an Image (image and edit file).
    */
-  FSImage(File imageDir) throws IOException {
+  public FSImage(File imageDir) throws IOException {
     this();
     ArrayList<File> dirs = new ArrayList<File>(1);
     dirs.add(imageDir);
@@ -556,11 +564,11 @@ class FSImage extends Storage {
     storageDirs.remove(index);
   }
 
-  FSEditLog getEditLog() {
+  public FSEditLog getEditLog() {
     return editLog;
   }
 
-  boolean isConversionNeeded(StorageDirectory sd) throws IOException {
+  public boolean isConversionNeeded(StorageDirectory sd) throws IOException {
     File oldImageDir = new File(sd.root, "image");
     if (!oldImageDir.exists()) {
       if(sd.getVersionFile().exists())
@@ -888,7 +896,7 @@ class FSImage extends Storage {
    * Save the contents of the FS image
    * and create empty edits.
    */
-  void saveFSImage() throws IOException {
+  public void saveFSImage() throws IOException {
     editLog.createNewIfMissing();
     for (int idx = 0; idx < getNumStorageDirs(); idx++) {
       StorageDirectory sd = getStorageDir(idx);
@@ -1226,7 +1234,7 @@ class FSImage extends Storage {
     return getImageFile(0, NameNodeFile.IMAGE);
   }
 
-  File getFsEditName() throws IOException {
+  public File getFsEditName() throws IOException {
     return getEditLog().getFsEditName();
   }
 

+ 40 - 53
src/hdfs/org/apache/hadoop/dfs/FSNamesystem.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -15,14 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.commons.logging.*;
 
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.dfs.BlocksMap.BlockInfo;
-import org.apache.hadoop.dfs.BlocksWithLocations.BlockWithLocations;
-import org.apache.hadoop.dfs.namenode.metrics.FSNamesystemMBean;
+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.Storage;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
+import org.apache.hadoop.hdfs.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
+import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMetrics;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.*;
@@ -32,7 +38,12 @@ import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.ScriptBasedMapping;
-import org.apache.hadoop.dfs.LeaseManager.Lease;
+import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
+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.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
@@ -68,7 +79,7 @@ import javax.security.auth.login.LoginException;
  * 4)  machine --> blocklist (inverted #2)
  * 5)  LRU cache of updated-heartbeat machines
  ***************************************************/
-class FSNamesystem implements FSConstants, FSNamesystemMBean {
+public class FSNamesystem implements FSConstants, FSNamesystemMBean {
   public static final Log LOG = LogFactory.getLog("org.apache.hadoop.fs.FSNamesystem");
   public static final String AUDIT_FORMAT =
     "ugi=%s\t" +  // ugi
@@ -94,7 +105,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
   //
   // Stores the correct file name hierarchy
   //
-  FSDirectory dir;
+  public FSDirectory dir;
 
   //
   // Mapping: Block -> { INode, datanodes, self ref } 
@@ -105,7 +116,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
   //
   // Store blocks-->datanodedescriptor(s) map of corrupt replicas
   //
-  CorruptReplicasMap corruptReplicas = new CorruptReplicasMap();
+  public CorruptReplicasMap corruptReplicas = new CorruptReplicasMap();
     
   /**
    * Stores the datanode -> block map.  
@@ -176,16 +187,16 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
   private UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
   private PendingReplicationBlocks pendingReplications;
 
-  LeaseManager leaseManager = new LeaseManager(this); 
+  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
-  Daemon lmthread = null;   // LeaseMonitor thread
+  public Daemon lmthread = null;   // LeaseMonitor thread
   Daemon smmthread = null;  // SafeModeMonitor thread
-  Daemon replthread = null;  // Replication thread
+  public Daemon replthread = null;  // Replication thread
   Daemon resthread = null; //ResolutionMonitor thread
   
   volatile boolean fsRunning = true;
@@ -336,7 +347,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
     LOG.info("Web-server up at: " + infoHost + ":" + infoPort);
   }
 
-  static Collection<File> getNamespaceDirs(Configuration conf) {
+  public static Collection<File> getNamespaceDirs(Configuration conf) {
     Collection<String> dirNames = conf.getStringCollection("dfs.name.dir");
     if (dirNames.isEmpty())
       dirNames.add("/tmp/hadoop/dfs/name");
@@ -698,7 +709,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
    * Get block locations within the specified range.
    * @see ClientProtocol#getBlockLocations(String, long, long)
    */
-  LocatedBlocks getBlockLocations(String src, long offset, long length
+  public LocatedBlocks getBlockLocations(String src, long offset, long length
       ) throws IOException {
     if (offset < 0) {
       throw new IOException("Negative offset is not supported. File: " + src );
@@ -729,7 +740,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
       return null;
     }
     if (blocks.length == 0) {
-      return new LocatedBlocks(inode, new ArrayList<LocatedBlock>(blocks.length));
+      return inode.createLocatedBlocks(new ArrayList<LocatedBlock>(blocks.length));
     }
     List<LocatedBlock> results;
     results = new ArrayList<LocatedBlock>(blocks.length);
@@ -780,7 +791,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
           && curBlk < blocks.length 
           && results.size() < nrBlocksToReturn);
     
-    return new LocatedBlocks(inode, results);
+    return inode.createLocatedBlocks(results);
   }
 
   /**
@@ -909,7 +920,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
                                   +src+" for "+holder+" at "+clientMachine);
     if (isInSafeMode())
       throw new SafeModeException("Cannot create file" + src, safeMode);
-    if (!isValidName(src)) {
+    if (!DFSUtil.isValidName(src)) {
       throw new IOException("Invalid file name: " + src);
     }
     if (isPermissionEnabled) {
@@ -1019,7 +1030,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
     }
     if (isInSafeMode())
       throw new SafeModeException("Cannot append file" + src, safeMode);
-    if (!isValidName(src)) {
+    if (!DFSUtil.isValidName(src)) {
       throw new IOException("Invalid file name: " + src);
     }
     if (isPermissionEnabled) {
@@ -1326,7 +1337,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
   /**
    * Mark the block belonging to datanode as corrupt
    * @param blk Block to be marked as corrupt
-   * @param datanode Datanode which holds the corrupt replica
+   * @param dn Datanode which holds the corrupt replica
    */
   public synchronized void markBlockAsCorrupt(Block blk, DatanodeInfo dn)
     throws IOException {
@@ -1410,7 +1421,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
     NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: " + src + " to " + dst);
     if (isInSafeMode())
       throw new SafeModeException("Cannot rename " + src, safeMode);
-    if (!isValidName(dst)) {
+    if (!DFSUtil.isValidName(dst)) {
       throw new IOException("Invalid name: " + dst);
     }
 
@@ -1497,30 +1508,6 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
     return dir.getFileInfo(src);
   }
 
-  /**
-   * Whether the pathname is valid.  Currently prohibits relative paths, 
-   * and names which contain a ":" or "/" 
-   */
-  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;
-  }
   /**
    * Create all the necessary directories
    */
@@ -1555,7 +1542,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
     }
     if (isInSafeMode())
       throw new SafeModeException("Cannot create directory " + src, safeMode);
-    if (!isValidName(src)) {
+    if (!DFSUtil.isValidName(src)) {
       throw new IOException("Invalid directory name: " + src);
     }
     if (isPermissionEnabled) {
@@ -1728,7 +1715,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
     }
     else {
       // update last block, construct newblockinfo and add it to the blocks map
-      lastblock.set(lastblock.blkid, newlength, newgenerationstamp);
+      lastblock.set(lastblock.getBlockId(), newlength, newgenerationstamp);
       final BlockInfo newblockinfo = blocksMap.addINode(lastblock, pendingFile);
     
       //update block info
@@ -1897,7 +1884,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
    * namespaceID and will continue serving the datanodes that has previously
    * registered with the namenode without restarting the whole cluster.
    * 
-   * @see DataNode#register()
+   * @see org.apache.hadoop.hdfs.server.datanode.DataNode#register()
    */
   public synchronized void registerDatanode(DatanodeRegistration nodeReg
                                             ) throws IOException {
@@ -2194,7 +2181,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
    * 
    * @return number of blocks scheduled for replication or removal.
    */
-  int computeDatanodeWork() throws IOException {
+  public int computeDatanodeWork() throws IOException {
     int workFound = 0;
     int blocksToProcess = 0;
     int nodesToProcess = 0;
@@ -2463,7 +2450,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
     return blocksToInvalidate.size();
   }
 
-  void setNodeReplicationLimit(int limit) {
+  public void setNodeReplicationLimit(int limit) {
     this.maxReplicationStreams = limit;
   }
 
@@ -3306,7 +3293,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
     throws IOException {
 
     if (!node.isDecommissionInProgress() && !node.isDecommissioned()) {
-      LOG.info("Start Decommissioning node " + node.name);
+      LOG.info("Start Decommissioning node " + node.getName());
       node.startDecommission();
       //
       // all the blocks that reside on this node have to be 
@@ -3324,7 +3311,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
    */
   public void stopDecommission (DatanodeDescriptor node) 
     throws IOException {
-    LOG.info("Stop Decommissioning node " + node.name);
+    LOG.info("Stop Decommissioning node " + node.getName());
     node.stopDecommission();
   }
 
@@ -3467,7 +3454,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
     if (node.isDecommissionInProgress()) {
       if (!isReplicationInProgress(node)) {
         node.setDecommissioned();
-        LOG.info("Decommission complete for node " + node.name);
+        LOG.info("Decommission complete for node " + node.getName());
       }
     }
     if (node.isDecommissioned()) {
@@ -4331,14 +4318,14 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
   /**
    * Sets the generation stamp for this filesystem
    */
-  void setGenerationStamp(long stamp) {
+  public void setGenerationStamp(long stamp) {
     generationStamp.setStamp(stamp);
   }
 
   /**
    * Gets the generation stamp for this filesystem
    */
-  long getGenerationStamp() {
+  public long getGenerationStamp() {
     return generationStamp.getStamp();
   }
 

+ 7 - 2
src/hdfs/org/apache/hadoop/dfs/FileDataServlet.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 import java.net.URI;
@@ -28,10 +28,15 @@ import java.util.Random;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
+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.dfs.HftpFileSystem
+ * @see org.apache.hadoop.hdfs.HftpFileSystem
  */
 public class FileDataServlet extends DfsServlet {
 

+ 2 - 2
src/hdfs/org/apache/hadoop/dfs/FsckServlet.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/FsckServlet.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.util.*;
 import java.io.*;
@@ -33,7 +33,7 @@ import javax.servlet.http.HttpServletResponse;
  */
 public class FsckServlet extends HttpServlet {
 
-  private static final Log LOG = LogFactory.getLog("org.apache.hadoop.dfs.FSNamesystem");
+  private static final Log LOG = LogFactory.getLog(FSNamesystem.class.getName());
 
   @SuppressWarnings("unchecked")
   public void doGet(HttpServletRequest request,

+ 14 - 5
src/hdfs/org/apache/hadoop/dfs/GenerationStampUpgradeNamenode.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/GenerationStampUpgradeNamenode.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.*;
 import java.util.*;
@@ -28,6 +28,14 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.net.InetSocketAddress;
 
 import org.apache.commons.logging.*;
+import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+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.GenerationStampStatsUpgradeCommand;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.retry.*;
 import org.apache.hadoop.util.StringUtils;
@@ -44,18 +52,18 @@ import org.apache.hadoop.ipc.RPC;
  * Once an upgrade starts at the namenode , this class manages the upgrade 
  * process.
  */
-class GenerationStampUpgradeNamenode extends UpgradeObjectNamenode {
+public class GenerationStampUpgradeNamenode extends UpgradeObjectNamenode {
   
   public static final Log LOG = 
-    LogFactory.getLog("org.apache.hadoop.dfs.GenerationStampUpgradeNamenode");
+    LogFactory.getLog(GenerationStampUpgradeNamenode.class.getName());
   
   static final long inactivityExtension = 10*1000; // 10 seconds
   AtomicLong lastNodeCompletionTime = new AtomicLong(0);
 
   // The layout version before the generation stamp upgrade.
-  static final int PRE_GENERATIONSTAMP_LAYOUT_VERSION = -13;
+  public static final int PRE_GENERATIONSTAMP_LAYOUT_VERSION = -13;
 
-  static final int DN_CMD_STATS = 300;
+  public static final int DN_CMD_STATS = 300;
   
   enum UpgradeStatus {
     INITIALIZED,
@@ -150,6 +158,7 @@ class GenerationStampUpgradeNamenode extends UpgradeObjectNamenode {
   }
 
   @Override
+  public
   UpgradeCommand processUpgradeCommand(UpgradeCommand command) 
                                            throws IOException {
     switch (command.getAction()) {

+ 1 - 1
src/hdfs/org/apache/hadoop/dfs/GetImageServlet.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java

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

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

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

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

@@ -0,0 +1,381 @@
+/**
+ * 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.io.UnsupportedEncodingException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Arrays;
+import java.util.List;
+import java.io.IOException;
+
+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;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
+
+/**
+ * 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.
+ */
+public abstract class INode implements Comparable<byte[]> {
+  protected byte[] name;
+  protected INodeDirectory parent;
+  protected long modificationTime;
+
+  //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;
+  }
+
+  INode(PermissionStatus permissions, long mTime) {
+    this.name = null;
+    this.parent = null;
+    this.modificationTime = mTime;
+    setPermissionStatus(permissions);
+  }
+
+  protected INode(String name, PermissionStatus permissions) {
+    this(permissions, 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());
+  }
+
+  /**
+   * 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});
+    return new ContentSummary(a[0], a[1], a[2], getQuota());
+  }
+  /**
+   * @return an array of three longs. 
+   * 0: length, 1: file count, 2: directory count
+   */
+  abstract long[] computeContentSummary(long[] summary);
+  
+  /**
+   * Get the quota set for this inode
+   * @return the quota if it is set; -1 otherwise
+   */
+  long getQuota() {
+    return -1;
+  }
+
+  /**
+   * Get the total number of names in the tree
+   * rooted at this inode including the root
+   * @return The total number of names in this tree
+   */
+  long numItemsInTree() {
+    return 1;
+  }
+    
+  /**
+   * 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;
+    }
+  }
+
+  /**
+   * 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());
+  }
+}

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

@@ -0,0 +1,444 @@
+/**
+ * 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;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+
+
+/**
+ * Directory INode class.
+ */
+public 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);
+    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
+   */
+  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; 
+   *          inserted INode, 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;
+  }
+
+  /**
+   */
+  long numItemsInTree() {
+    long total = 1L;
+    if (children == null) {
+      return total;
+    }
+    for (INode child : children) {
+      total += child.numItemsInTree();
+    }
+    return total;
+  }
+
+  /** {@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;
+  }
+}
+
+/**
+ * Directory INode class that has a quota restriction
+ */
+class INodeDirectoryWithQuota extends INodeDirectory {
+  private long quota;
+  private long count;
+  
+  /** Convert an existing directory inode to one with the given quota
+   * 
+   * @param quota Quota to be assigned to this inode
+   * @param other The other inode from which all other properties are copied
+   */
+  INodeDirectoryWithQuota(long quota, INodeDirectory other)
+  throws QuotaExceededException {
+    super(other);
+    this.count = other.numItemsInTree();
+    setQuota(quota);
+  }
+  
+  /** constructor with no quota verification */
+  INodeDirectoryWithQuota(
+      PermissionStatus permissions, long modificationTime, long quota)
+  {
+    super(permissions, modificationTime);
+    this.quota = quota;
+  }
+  
+  /** constructor with no quota verification */
+  INodeDirectoryWithQuota(String name, PermissionStatus permissions, long quota)
+  {
+    super(name, permissions);
+    this.quota = quota;
+  }
+  
+  /** Get this directory's quota
+   * @return this directory's quota
+   */
+  long getQuota() {
+    return quota;
+  }
+  
+  /** Set this directory's quota
+   * 
+   * @param quota Quota to be set
+   * @throws QuotaExceededException if the given quota is less than 
+   *                                the size of the tree
+   */
+  void setQuota(long quota) throws QuotaExceededException {
+    verifyQuota(quota, this.count);
+    this.quota = quota;
+  }
+  
+  /** 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 count;
+  }
+  
+  /** Update the size of the tree
+   * 
+   * @param delta the change of the tree size
+   * @throws QuotaExceededException if the changed size is greater 
+   *                                than the quota
+   */
+  void updateNumItemsInTree(long delta) throws QuotaExceededException {
+    long newCount = this.count + delta;
+    if (delta>0) {
+      verifyQuota(this.quota, newCount);
+    }
+    this.count = newCount;
+  }
+  
+  /** Set the size of the tree rooted at this directory
+   * 
+   * @param count size of the directory to be set
+   * @throws QuotaExceededException if the given count is greater than quota
+   */
+  void setCount(long count) throws QuotaExceededException {
+    verifyQuota(this.quota, count);
+    this.count = count;
+  }
+  
+  /** Verify if the count satisfies the quota restriction 
+   * @throws QuotaExceededException if the given quota is less than the count
+   */
+  private static void verifyQuota(long quota, long count)
+  throws QuotaExceededException {
+    if (quota < count) {
+      throw new QuotaExceededException(quota, count);
+    }
+  }
+}

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

@@ -0,0 +1,165 @@
+/**
+ * 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;
+
+public 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 preferredBlockSize) {
+    this(permissions, new BlockInfo[nrBlocks], replication,
+        modificationTime, preferredBlockSize);
+  }
+
+  protected INodeFile() {
+    blocks = null;
+    blockReplication = 0;
+    preferredBlockSize = 0;
+  }
+
+  protected INodeFile(PermissionStatus permissions, BlockInfo[] blklist,
+                      short replication, long modificationTime,
+                      long preferredBlockSize) {
+    super(permissions, modificationTime);
+    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];
+      for (int i = 0; i < size; i++) {
+        newlist[i] = this.blocks[i];
+      }
+      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]++;
+    return summary;
+  }
+
+  /**
+   * 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);
+  }
+}

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

@@ -0,0 +1,168 @@
+/**
+ * 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;
+
+
+public class INodeFileUnderConstruction extends INodeFile {
+  StringBytesWritable clientName = null;         // lease holder
+  StringBytesWritable clientMachine = null;
+  DatanodeDescriptor clientNode = null; // 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
+  
+  INodeFileUnderConstruction() {}
+
+  INodeFileUnderConstruction(PermissionStatus permissions,
+                             short replication,
+                             long preferredBlockSize,
+                             long modTime,
+                             String clientName,
+                             String clientMachine,
+                             DatanodeDescriptor clientNode) 
+                             throws IOException {
+    super(permissions.applyUMask(UMASK), 0, replication, modTime,
+        preferredBlockSize);
+    this.clientName = new StringBytesWritable(clientName);
+    this.clientMachine = new StringBytesWritable(clientMachine);
+    this.clientNode = clientNode;
+  }
+
+  public INodeFileUnderConstruction(byte[] name,
+                             short blockReplication,
+                             long modificationTime,
+                             long preferredBlockSize,
+                             BlockInfo[] blocks,
+                             PermissionStatus perm,
+                             String clientName,
+                             String clientMachine,
+                             DatanodeDescriptor clientNode)
+                             throws IOException {
+    super(perm, blocks, blockReplication, modificationTime, 
+          preferredBlockSize);
+    setLocalName(name);
+    this.clientName = new StringBytesWritable(clientName);
+    this.clientMachine = new StringBytesWritable(clientMachine);
+    this.clientNode = clientNode;
+  }
+
+  String getClientName() throws IOException {
+    return clientName.getString();
+  }
+
+  String getClientMachine() throws IOException {
+    return clientMachine.getString();
+  }
+
+  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
+  //
+  INodeFile convertToInodeFile() {
+    INodeFile obj = new INodeFile(getPermissionStatus(),
+                                  getBlocks(),
+                                  getReplication(),
+                                  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;
+  }
+
+  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);
+  }
+
+  /**
+   * 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.");
+      }
+    }
+  }
+}

+ 11 - 4
src/hdfs/org/apache/hadoop/dfs/JspHelper.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
@@ -32,7 +32,14 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.jsp.JspWriter;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.dfs.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.DFSClient;
+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.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.net.NetUtils;
@@ -42,13 +49,13 @@ public class JspHelper {
   final static public String WEB_UGI_PROPERTY_NAME = "dfs.web.ugi";
 
   static FSNamesystem fsn = null;
-  static InetSocketAddress nameNodeAddr;
+  public static InetSocketAddress nameNodeAddr;
   public static Configuration conf = new Configuration();
   public static final UnixUserGroupInformation webUGI
   = UnixUserGroupInformation.createImmutable(
       conf.getStrings(WEB_UGI_PROPERTY_NAME));
 
-  static int defaultChunkSizeToView = 
+  public static int defaultChunkSizeToView = 
     conf.getInt("dfs.default.chunk.view.size", 32 * 1024);
   static Random rand = new Random();
 

+ 1 - 1
src/hdfs/org/apache/hadoop/dfs/LeaseExpiredException.java → src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseExpiredException.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 

部分文件因文件數量過多而無法顯示