Bladeren bron

Merge -r 796828:800617 from trunk to branch HDFS-265

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/branches/HDFS-265@800624 13f79535-47bb-0310-9956-ffa450edef68
Hairong Kuang 16 jaren geleden
bovenliggende
commit
7e2bf2e6a2
45 gewijzigde bestanden met toevoegingen van 1261 en 749 verwijderingen
  1. 40 0
      CHANGES.txt
  2. 68 15
      build.xml
  3. 2 2
      ivy.xml
  4. 1 1
      ivy/libraries.properties
  5. 159 69
      src/java/org/apache/hadoop/hdfs/DFSClient.java
  6. 126 26
      src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java
  7. 5 3
      src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  8. 17 13
      src/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  9. 41 40
      src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  10. 6 6
      src/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java
  11. 22 22
      src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
  12. 3 3
      src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
  13. 276 0
      src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfo.java
  14. 74 80
      src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java
  15. 12 281
      src/java/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java
  16. 0 1
      src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java
  17. 3 3
      src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  18. 90 83
      src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  19. 0 1
      src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  20. 2 3
      src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  21. 0 1
      src/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  22. 0 1
      src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  23. 5 0
      src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  24. 10 2
      src/test/findbugsExcludeFile.xml
  25. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestAbandonBlock.java
  26. 141 3
      src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  27. 4 3
      src/test/hdfs/org/apache/hadoop/hdfs/TestDFSPermission.java
  28. 25 20
      src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
  29. 8 8
      src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
  30. 3 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestFSInputChecker.java
  31. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend.java
  32. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend3.java
  33. 8 8
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java
  34. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestGetBlocks.java
  35. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
  36. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery.java
  37. 20 16
      src/test/hdfs/org/apache/hadoop/hdfs/TestReplication.java
  38. 58 0
      src/test/hdfs/org/apache/hadoop/hdfs/TestSetTimes.java
  39. 7 9
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
  40. 5 5
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
  41. 5 3
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
  42. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
  43. 0 3
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
  44. 3 3
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
  45. 2 1
      src/test/hdfs/org/apache/hadoop/security/TestPermission.java

+ 40 - 0
CHANGES.txt

@@ -45,6 +45,31 @@ Trunk (unreleased changes)
     HDFS-458. Create a new ant target, run-commit-test.  (Jakob Homan
     HDFS-458. Create a new ant target, run-commit-test.  (Jakob Homan
     via szetszwo)
     via szetszwo)
 
 
+    HDFS-493. Change build.xml so that the fault-injected tests are executed
+    only by the run-test-*-faul-inject targets.  (Konstantin Boudnik via
+    szetszwo)
+
+    HADOOP-6160. Fix releaseaudit target to run on specific directories.
+    (gkesavan)
+
+    HDFS-508. Factor out BlockInfo from BlocksMap. (shv)
+
+    HDFS-510. Rename DatanodeBlockInfo to be ReplicaInfo.
+    (Jakob Homan & Hairong Kuang via shv)
+
+    HDFS-500. Deprecate NameNode methods deprecated in NameNodeProtocol.
+    (Jakob Homan via shv)
+
+    HDFS-514. Change DFSClient.namenode from public to private.  (Bill Zeller
+    via szetszwo)
+
+    HDFS-496. Use PureJavaCrc32 in HDFS.  (Todd Lipcon via szetszwo)
+
+    HDFS-511. Remove redundant block searches in BlockManager. (shv)
+
+    HDFS-504. Update the modification time of a file when the file 
+    is closed. (Chun Zhang via dhruba)
+
   BUG FIXES
   BUG FIXES
     HDFS-76. Better error message to users when commands fail because of 
     HDFS-76. Better error message to users when commands fail because of 
     lack of quota. Allow quota to be set even if the limit is lower than
     lack of quota. Allow quota to be set even if the limit is lower than
@@ -89,6 +114,21 @@ Trunk (unreleased changes)
     HDFS-463. CreateEditLog utility broken after HDFS-396 (URI for
     HDFS-463. CreateEditLog utility broken after HDFS-396 (URI for
     FSImage). (Suresh Srinivas via rangadi)
     FSImage). (Suresh Srinivas via rangadi)
 
 
+    HDFS-484. Fix bin-package and package target to package jar files.
+    (gkesavan)
+
+    HDFS-501. Use enum to define the constants in DataTransferProtocol.
+    (szetszwo)
+
+    HDFS-490. Eliminate the deprecated warnings introduced by H-5438.
+    (He Yongqiang via szetszwo)
+
+    HDFS-119. Fix a bug in logSync(), which causes NameNode block forever.
+    (Suresh Srinivas via shv)
+
+    HDFS-167. Fix a bug in DFSClient that caused infinite retries on write.
+    (Bill Zeller via szetszwo)
+
 Release 0.20.1 - Unreleased
 Release 0.20.1 - Unreleased
 
 
   IMPROVEMENTS
   IMPROVEMENTS

+ 68 - 15
build.xml

@@ -362,6 +362,8 @@
           description="Run Fault Injection related hdfs tests">
           description="Run Fault Injection related hdfs tests">
     <subant buildpath="build.xml" target="run-test-hdfs">
     <subant buildpath="build.xml" target="run-test-hdfs">
       <property name="build.dir" value="${build-fi.dir}"/>
       <property name="build.dir" value="${build-fi.dir}"/>
+      <property name="test.fault.inject" value="yes"/>
+      <property name="test.include" value="TestFi*"/>
     </subant>
     </subant>
   </target>
   </target>
 
 
@@ -369,6 +371,8 @@
           description="Run hdfs Fault Injection related unit tests that require mapred">
           description="Run hdfs Fault Injection related unit tests that require mapred">
     <subant buildpath="build.xml" target="run-test-hdfs-with-mr">
     <subant buildpath="build.xml" target="run-test-hdfs-with-mr">
       <property name="build.dir" value="${build-fi.dir}"/>
       <property name="build.dir" value="${build-fi.dir}"/>
+      <property name="test.fault.inject" value="yes"/>
+      <property name="test.include" value="TestFi*"/>
     </subant>
     </subant>
   </target>
   </target>
 
 
@@ -551,6 +555,35 @@
     </jar>
     </jar>
   </target>
   </target>
 
 
+  <condition property="tests.notestcase">
+    <and>
+      <isfalse value="${test.fault.inject}"/>
+      <not>
+        <isset property="testcase"/>
+      </not>
+    </and>
+  </condition>
+  <condition property="tests.notestcase.fi">
+    <and>
+      <not>
+        <isset property="testcase" />
+      </not>
+      <istrue value="${test.fault.inject}" />
+    </and>
+  </condition>
+  <condition property="tests.testcase">
+    <and>
+      <isfalse value="${test.fault.inject}" />
+      <isset property="testcase" />
+    </and>
+  </condition>
+  <condition property="tests.testcase.fi">
+    <and>
+      <istrue value="${test.fault.inject}" />
+      <isset property="testcase" />
+    </and>
+  </condition>
+
   <!-- ================================================================== -->
   <!-- ================================================================== -->
   <!-- Run unit tests                                                     --> 
   <!-- Run unit tests                                                     --> 
   <!-- ================================================================== -->
   <!-- ================================================================== -->
@@ -586,15 +619,22 @@
           <propertyref regex="fi.*"/>
           <propertyref regex="fi.*"/>
         </syspropertyset>
         </syspropertyset>
         <formatter type="${test.junit.output.format}" />
         <formatter type="${test.junit.output.format}" />
-        <batchtest todir="${test.build.dir}" unless="testcase">
+        <batchtest todir="${test.build.dir}" if="tests.notestcase">
           <fileset dir="${test.src.dir}/hdfs" excludes="**/${test.exclude}.java">
           <fileset dir="${test.src.dir}/hdfs" excludes="**/${test.exclude}.java">
              <patternset>
              <patternset>
                <includesfile name="@{test.file}"/>
                <includesfile name="@{test.file}"/>
              </patternset>
              </patternset>
          </fileset>
          </fileset>
         </batchtest>
         </batchtest>
-        <batchtest todir="${test.build.dir}" if="testcase">
+        <batchtest todir="${test.build.dir}" if="tests.notestcase.fi">
+          <fileset dir="${test.src.dir}/aop"
+            includes="**/${test.include}.java"
+            excludes="**/${test.exclude}.java" />
+        </batchtest>
+        <batchtest todir="${test.build.dir}" if="tests.testcase">
           <fileset dir="${test.src.dir}/hdfs" includes="**/${testcase}.java"/>
           <fileset dir="${test.src.dir}/hdfs" includes="**/${testcase}.java"/>
+        </batchtest>
+        <batchtest todir="${test.build.dir}" if="tests.testcase.fi">
           <fileset dir="${test.src.dir}/aop" includes="**/${testcase}.java"/>
           <fileset dir="${test.src.dir}/aop" includes="**/${testcase}.java"/>
         </batchtest>
         </batchtest>
       </junit>
       </junit>
@@ -640,14 +680,22 @@
         <propertyref regex="fi.*"/>
         <propertyref regex="fi.*"/>
       </syspropertyset>
       </syspropertyset>
       <formatter type="${test.junit.output.format}" />
       <formatter type="${test.junit.output.format}" />
-      <batchtest todir="${test.build.dir}" unless="testcase">
+      <batchtest todir="${test.build.dir}" if="tests.notestcase">
         <fileset dir="${test.src.dir}/hdfs-with-mr"
         <fileset dir="${test.src.dir}/hdfs-with-mr"
            includes="**/${test.include}.java"
            includes="**/${test.include}.java"
-     excludes="**/${test.exclude}.java" />
+           excludes="**/${test.exclude}.java" />
       </batchtest>
       </batchtest>
-      <batchtest todir="${test.build.dir}" if="testcase">
+      <batchtest todir="${test.build.dir}" if="tests.notestcase.fi">
+        <fileset dir="${test.src.dir}/aop"
+          includes="**/${test.include}.java"
+          excludes="**/${test.exclude}.java" />
+      </batchtest>
+      <batchtest todir="${test.build.dir}" if="tests.testcase">
         <fileset dir="${test.src.dir}/hdfs-with-mr" includes="**/${testcase}.java"/>
         <fileset dir="${test.src.dir}/hdfs-with-mr" includes="**/${testcase}.java"/>
       </batchtest>
       </batchtest>
+      <batchtest todir="${test.build.dir}" if="tests.testcase.fi">
+        <fileset dir="${test.src.dir}/aop" includes="**/${testcase}.java"/>
+      </batchtest>
     </junit>
     </junit>
     <antcall target="checkfailure"/>
     <antcall target="checkfailure"/>
   </target>  
   </target>  
@@ -956,7 +1004,7 @@
     </copy>
     </copy>
 
 
     <copy todir="${dist.dir}"> 
     <copy todir="${dist.dir}"> 
-      <fileset file="${build.dir}/${final.name}-*.jar"/>
+      <fileset file="${build.dir}/${name}-*.jar"/>
     </copy>
     </copy>
 
 
     <copy todir="${dist.dir}/conf">
     <copy todir="${dist.dir}/conf">
@@ -1039,7 +1087,7 @@
     </copy>
     </copy>
 
 
     <copy todir="${dist.dir}"> 
     <copy todir="${dist.dir}"> 
-      <fileset file="${build.dir}/${final.name}-*.jar"/>
+      <fileset file="${build.dir}/${name}-*.jar"/>
     </copy>
     </copy>
     
     
     <copy todir="${dist.dir}/conf">
     <copy todir="${dist.dir}/conf">
@@ -1081,12 +1129,19 @@
   <!-- ================================================================== -->
   <!-- ================================================================== -->
   <!-- Perform audit activities for the release                           -->
   <!-- Perform audit activities for the release                           -->
   <!-- ================================================================== -->
   <!-- ================================================================== -->
-  <target name="releaseaudit" depends="package,ivy-retrieve-releaseaudit" description="Release Audit activities">
-    <fail unless="rat.present" message="Failed to load class [${rat.reporting.classname}]."/>
-    <java classname="${rat.reporting.classname}" fork="true">
-      <classpath refid="releaseaudit-classpath"/>
-      <arg value="${build.dir}/${final.name}"/>
-    </java>
+  <target name="rats-taskdef" depends="ivy-retrieve-releaseaudit">
+     <typedef format="xml" resource="org/apache/rat/anttasks/antlib.xml" uri="antlib:org.apache.rat.anttasks"
+      classpathref="releaseaudit-classpath"/>
+  </target>
+
+  <target name="releaseaudit" depends="package, rats-taskdef" description="Release Audit activities">
+   <rat:report xmlns:rat="antlib:org.apache.rat.anttasks">
+      <fileset dir="${dist.dir}">
+        <exclude name="CHANGES.txt"/>
+        <exclude name="docs/"/>
+        <exclude name="lib/jdiff/"/>
+      </fileset>
+    </rat:report>
   </target>
   </target>
 
 
   <!-- ================================================================== -->
   <!-- ================================================================== -->
@@ -1360,8 +1415,6 @@
     <ivy:retrieve settingsRef="${ant.project.name}.ivy.settings"
     <ivy:retrieve settingsRef="${ant.project.name}.ivy.settings"
       pattern="${build.ivy.lib.dir}/${ivy.artifact.retrieve.pattern}" />
       pattern="${build.ivy.lib.dir}/${ivy.artifact.retrieve.pattern}" />
     <ivy:cachepath pathid="releaseaudit-classpath" conf="releaseaudit"/>
     <ivy:cachepath pathid="releaseaudit-classpath" conf="releaseaudit"/>
-    <available classname="${rat.reporting.classname}" 
-      classpathref="releaseaudit-classpath" property="rat.present" value="true"/>
   </target>
   </target>
 
 
   <target name="ivy-report" depends="ivy-resolve-releaseaudit"
   <target name="ivy-report" depends="ivy-resolve-releaseaudit"

+ 2 - 2
ivy.xml

@@ -224,8 +224,8 @@
       name="junit"
       name="junit"
       rev="${junit.version}"
       rev="${junit.version}"
       conf="common->default"/>
       conf="common->default"/>
-    <dependency org="com.google.code.p.arat"
-      name="rat-lib"
+    <dependency org="org.apache.rat"
+      name="apache-rat-tasks"
       rev="${rats-lib.version}"
       rev="${rats-lib.version}"
       conf="releaseaudit->default"/>
       conf="releaseaudit->default"/>
     <dependency org="commons-lang"
     <dependency org="commons-lang"

+ 1 - 1
ivy/libraries.properties

@@ -60,7 +60,7 @@ mina-core.version=2.0.0-M6
 
 
 oro.version=2.0.8
 oro.version=2.0.8
 
 
-rats-lib.version=0.5.1
+rats-lib.version=0.6
 
 
 servlet.version=4.0.6
 servlet.version=4.0.6
 servlet-api-2.5.version=6.1.14
 servlet-api-2.5.version=6.1.14

+ 159 - 69
src/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -17,42 +17,100 @@
  */
  */
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.io.retry.RetryPolicies;
-import org.apache.hadoop.io.retry.RetryPolicy;
-import org.apache.hadoop.io.retry.RetryProxy;
-import org.apache.hadoop.fs.*;
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Op.BLOCK_CHECKSUM;
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.CHECKSUM_OK;
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR_ACCESS_TOKEN;
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.SUCCESS;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import javax.net.SocketFactory;
+import javax.security.auth.login.LoginException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSInputChecker;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FSOutputSummer;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.ipc.*;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.net.NodeBase;
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.security.InvalidAccessTokenException;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.ipc.Client;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessToken;
 import org.apache.hadoop.security.AccessToken;
+import org.apache.hadoop.security.InvalidAccessTokenException;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.security.UnixUserGroupInformation;
-import org.apache.hadoop.util.*;
-
-import org.apache.commons.logging.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-import java.util.zip.CRC32;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.ConcurrentHashMap;
-import java.nio.BufferOverflowException;
-import java.nio.ByteBuffer;
-
-import javax.net.SocketFactory;
-import javax.security.auth.login.LoginException;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.PureJavaCrc32;
+import org.apache.hadoop.util.StringUtils;
 
 
 /********************************************************
 /********************************************************
  * DFSClient can connect to a Hadoop Filesystem and 
  * DFSClient can connect to a Hadoop Filesystem and 
@@ -69,8 +127,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   public static final Log LOG = LogFactory.getLog(DFSClient.class);
   public static final Log LOG = LogFactory.getLog(DFSClient.class);
   public 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
   private static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
-  public final ClientProtocol namenode;
-  final private ClientProtocol rpcNamenode;
+  private ClientProtocol namenode;
+  private ClientProtocol rpcNamenode;
   final UnixUserGroupInformation ugi;
   final UnixUserGroupInformation ugi;
   volatile boolean clientRunning = true;
   volatile boolean clientRunning = true;
   Random r = new Random();
   Random r = new Random();
@@ -161,6 +219,29 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   public DFSClient(InetSocketAddress nameNodeAddr, Configuration conf,
   public DFSClient(InetSocketAddress nameNodeAddr, Configuration conf,
                    FileSystem.Statistics stats)
                    FileSystem.Statistics stats)
     throws IOException {
     throws IOException {
+    this(conf, stats);
+    this.rpcNamenode = createRPCNamenode(nameNodeAddr, conf, ugi);
+    this.namenode = createNamenode(this.rpcNamenode);
+  }
+
+  /** 
+   * Create a new DFSClient connected to the given namenode
+   * and rpcNamenode objects.
+   * 
+   * This constructor was written to allow easy testing of the DFSClient class.
+   * End users will most likely want to use one of the other constructors.
+   */
+  public DFSClient(ClientProtocol namenode, ClientProtocol rpcNamenode,
+                   Configuration conf, FileSystem.Statistics stats)
+    throws IOException {
+      this(conf, stats);
+      this.namenode = namenode;
+      this.rpcNamenode = rpcNamenode;
+  }
+
+  
+  private DFSClient(Configuration conf, FileSystem.Statistics stats)
+    throws IOException {
     this.conf = conf;
     this.conf = conf;
     this.stats = stats;
     this.stats = stats;
     this.socketTimeout = conf.getInt("dfs.socket.timeout", 
     this.socketTimeout = conf.getInt("dfs.socket.timeout", 
@@ -182,9 +263,6 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       throw (IOException)(new IOException().initCause(e));
       throw (IOException)(new IOException().initCause(e));
     }
     }
 
 
-    this.rpcNamenode = createRPCNamenode(nameNodeAddr, conf, ugi);
-    this.namenode = createNamenode(rpcNamenode);
-
     String taskId = conf.get("mapred.task.id");
     String taskId = conf.get("mapred.task.id");
     if (taskId != null) {
     if (taskId != null) {
       this.clientName = "DFSClient_" + taskId; 
       this.clientName = "DFSClient_" + taskId; 
@@ -372,6 +450,14 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     return create(src, overwrite, replication, blockSize, null);
     return create(src, overwrite, replication, blockSize, null);
   }
   }
 
 
+  /**
+   * Get the namenode associated with this DFSClient object
+   * @return the namenode associated with this DFSClient object
+   */
+  public ClientProtocol getNamenode() {
+    return namenode;
+  }
+  
   
   
   /**
   /**
    * Create a new dfs file with the specified block replication 
    * Create a new dfs file with the specified block replication 
@@ -619,15 +705,14 @@ public class DFSClient implements FSConstants, java.io.Closeable {
         try {
         try {
           if (LOG.isDebugEnabled()) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("write to " + datanodes[j].getName() + ": "
             LOG.debug("write to " + datanodes[j].getName() + ": "
-                + DataTransferProtocol.OP_BLOCK_CHECKSUM +
-                ", block=" + block);
+                + BLOCK_CHECKSUM + ", block=" + block);
           }
           }
           DataTransferProtocol.Sender.opBlockChecksum(out, block.getBlockId(),
           DataTransferProtocol.Sender.opBlockChecksum(out, block.getBlockId(),
               block.getGenerationStamp(), lb.getAccessToken());
               block.getGenerationStamp(), lb.getAccessToken());
 
 
-          final short reply = in.readShort();
-          if (reply != DataTransferProtocol.OP_STATUS_SUCCESS) {
-            if (reply == DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN
+          final DataTransferProtocol.Status reply = DataTransferProtocol.Status.read(in);
+          if (reply != SUCCESS) {
+            if (reply == ERROR_ACCESS_TOKEN
                 && i > lastRetriedIndex) {
                 && i > lastRetriedIndex) {
               if (LOG.isDebugEnabled()) {
               if (LOG.isDebugEnabled()) {
                 LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
                 LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
@@ -1353,9 +1438,9 @@ public class DFSClient implements FSConstants, java.io.Closeable {
           new BufferedInputStream(NetUtils.getInputStream(sock), 
           new BufferedInputStream(NetUtils.getInputStream(sock), 
                                   bufferSize));
                                   bufferSize));
       
       
-      short status = in.readShort();
-      if (status != DataTransferProtocol.OP_STATUS_SUCCESS) {
-        if (status == DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN) {
+      DataTransferProtocol.Status status = DataTransferProtocol.Status.read(in);
+      if (status != SUCCESS) {
+        if (status == ERROR_ACCESS_TOKEN) {
           throw new InvalidAccessTokenException(
           throw new InvalidAccessTokenException(
               "Got access token error in response to OP_READ_BLOCK "
               "Got access token error in response to OP_READ_BLOCK "
                   + "for file " + file + " for block " + blockId);
                   + "for file " + file + " for block " + blockId);
@@ -1402,9 +1487,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     private void checksumOk(Socket sock) {
     private void checksumOk(Socket sock) {
       try {
       try {
         OutputStream out = NetUtils.getOutputStream(sock, HdfsConstants.WRITE_TIMEOUT);
         OutputStream out = NetUtils.getOutputStream(sock, HdfsConstants.WRITE_TIMEOUT);
-        byte buf[] = { (DataTransferProtocol.OP_STATUS_CHECKSUM_OK >>> 8) & 0xff,
-                       (DataTransferProtocol.OP_STATUS_CHECKSUM_OK) & 0xff };
-        out.write(buf);
+        CHECKSUM_OK.writeOutputStream(out);
         out.flush();
         out.flush();
       } catch (IOException e) {
       } catch (IOException e) {
         // its ok not to be able to send this.
         // its ok not to be able to send this.
@@ -2476,8 +2559,9 @@ public class DFSClient implements FSConstants, java.io.Closeable {
 
 
               // processes response status from all datanodes.
               // processes response status from all datanodes.
               for (int i = 0; i < targets.length && clientRunning; i++) {
               for (int i = 0; i < targets.length && clientRunning; i++) {
-                short reply = blockReplyStream.readShort();
-                if (reply != DataTransferProtocol.OP_STATUS_SUCCESS) {
+                final DataTransferProtocol.Status reply
+                    = DataTransferProtocol.Status.read(blockReplyStream);
+                if (reply != SUCCESS) {
                   errorIndex = i; // first bad datanode
                   errorIndex = i; // first bad datanode
                   throw new IOException("Bad response " + reply +
                   throw new IOException("Bad response " + reply +
                       " for block " + block +
                       " for block " + block +
@@ -2716,7 +2800,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       //
       //
       private boolean createBlockOutputStream(DatanodeInfo[] nodes, String client,
       private boolean createBlockOutputStream(DatanodeInfo[] nodes, String client,
           boolean recoveryFlag) {
           boolean recoveryFlag) {
-        short pipelineStatus = (short)DataTransferProtocol.OP_STATUS_SUCCESS;
+        DataTransferProtocol.Status pipelineStatus = SUCCESS;
         String firstBadLink = "";
         String firstBadLink = "";
         if (LOG.isDebugEnabled()) {
         if (LOG.isDebugEnabled()) {
           for (int i = 0; i < nodes.length; i++) {
           for (int i = 0; i < nodes.length; i++) {
@@ -2755,10 +2839,10 @@ public class DFSClient implements FSConstants, java.io.Closeable {
           out.flush();
           out.flush();
 
 
           // receive ack for connect
           // receive ack for connect
-          pipelineStatus = blockReplyStream.readShort();
+          pipelineStatus = DataTransferProtocol.Status.read(blockReplyStream);
           firstBadLink = Text.readString(blockReplyStream);
           firstBadLink = Text.readString(blockReplyStream);
-          if (pipelineStatus != DataTransferProtocol.OP_STATUS_SUCCESS) {
-            if (pipelineStatus == DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN) {
+          if (pipelineStatus != SUCCESS) {
+            if (pipelineStatus == ERROR_ACCESS_TOKEN) {
               throw new InvalidAccessTokenException(
               throw new InvalidAccessTokenException(
                   "Got access token error for connect ack with firstBadLink as "
                   "Got access token error for connect ack with firstBadLink as "
                       + firstBadLink);
                       + firstBadLink);
@@ -2792,7 +2876,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       }
       }
 
 
       private LocatedBlock locateFollowingBlock(long start) throws IOException {
       private LocatedBlock locateFollowingBlock(long start) throws IOException {
-        int retries = 5;
+        int retries = conf.getInt("dfs.client.block.write.locateFollowingBlock.retries", 5);
         long sleeptime = 400;
         long sleeptime = 400;
         while (true) {
         while (true) {
           long localstart = System.currentTimeMillis();
           long localstart = System.currentTimeMillis();
@@ -2808,26 +2892,32 @@ public class DFSClient implements FSConstants, java.io.Closeable {
               if (ue != e) { 
               if (ue != e) { 
                 throw ue; // no need to retry these exceptions
                 throw ue; // no need to retry these exceptions
               }
               }
-
-              if (--retries == 0 && 
-                  !NotReplicatedYetException.class.getName().
+              
+              
+              if (NotReplicatedYetException.class.getName().
                   equals(e.getClassName())) {
                   equals(e.getClassName())) {
-                throw e;
-              } else {
-                LOG.info(StringUtils.stringifyException(e));
-                if (System.currentTimeMillis() - localstart > 5000) {
-                  LOG.info("Waiting for replication for "
-                      + (System.currentTimeMillis() - localstart) / 1000
-                      + " seconds");
-                }
-                try {
-                  LOG.warn("NotReplicatedYetException sleeping " + src
-                      + " retries left " + retries);
-                  Thread.sleep(sleeptime);
-                  sleeptime *= 2;
-                } catch (InterruptedException ie) {
+                if (retries == 0) { 
+                  throw e;
+                } else {
+                  --retries;
+                  LOG.info(StringUtils.stringifyException(e));
+                  if (System.currentTimeMillis() - localstart > 5000) {
+                    LOG.info("Waiting for replication for "
+                        + (System.currentTimeMillis() - localstart) / 1000
+                        + " seconds");
+                  }
+                  try {
+                    LOG.warn("NotReplicatedYetException sleeping " + src
+                        + " retries left " + retries);
+                    Thread.sleep(sleeptime);
+                    sleeptime *= 2;
+                  } catch (InterruptedException ie) {
+                  }
                 }
                 }
-              }                
+              } else {
+                throw e;
+              }
+
             }
             }
           }
           }
         } 
         } 
@@ -2919,7 +3009,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
 
 
     private DFSOutputStream(String src, long blockSize, Progressable progress,
     private DFSOutputStream(String src, long blockSize, Progressable progress,
         int bytesPerChecksum) throws IOException {
         int bytesPerChecksum) throws IOException {
-      super(new CRC32(), bytesPerChecksum, 4);
+      super(new PureJavaCrc32(), bytesPerChecksum, 4);
       this.src = src;
       this.src = src;
       this.blockSize = blockSize;
       this.blockSize = blockSize;
       this.progress = progress;
       this.progress = progress;

+ 126 - 26
src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java

@@ -17,9 +17,12 @@
  */
  */
 package org.apache.hadoop.hdfs.protocol;
 package org.apache.hadoop.hdfs.protocol;
 
 
+import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
+import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.OutputStream;
 
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.AccessToken;
 import org.apache.hadoop.security.AccessToken;
@@ -42,39 +45,136 @@ public interface DataTransferProtocol {
    */
    */
   public static final int DATA_TRANSFER_VERSION = 16;
   public static final int DATA_TRANSFER_VERSION = 16;
 
 
-  // Processed at datanode stream-handler
-  public static final byte OP_WRITE_BLOCK = (byte) 80;
-  public static final byte OP_READ_BLOCK = (byte) 81;
-  /**
-   * @deprecated As of version 15, OP_READ_METADATA is no longer supported
-   */
-  @Deprecated public static final byte OP_READ_METADATA = (byte) 82;
-  public static final byte OP_REPLACE_BLOCK = (byte) 83;
-  public static final byte OP_COPY_BLOCK = (byte) 84;
-  public static final byte OP_BLOCK_CHECKSUM = (byte) 85;
+  /** Operation */
+  public enum Op {
+    WRITE_BLOCK((byte)80),
+    READ_BLOCK((byte)81),
+    READ_METADATA((byte)82),
+    REPLACE_BLOCK((byte)83),
+    COPY_BLOCK((byte)84),
+    BLOCK_CHECKSUM((byte)85);
+
+    /** The code for this operation. */
+    public final byte code;
+    
+    private Op(byte code) {
+      this.code = code;
+    }
+    
+    private static final int FIRST_CODE = values()[0].code;
+    /** Return the object represented by the code. */
+    private static Op valueOf(byte code) {
+      final int i = (code & 0xff) - FIRST_CODE;
+      return i < 0 || i >= values().length? null: values()[i];
+    }
+
+    /** Read from in */
+    public static Op read(DataInput in) throws IOException {
+      return valueOf(in.readByte());
+    }
+
+    /** Write to out */
+    public void write(DataOutput out) throws IOException {
+      out.write(code);
+    }
+  };
+
+  /** Status */
+  public enum Status {
+    SUCCESS(0),
+    ERROR(1),
+    ERROR_CHECKSUM(2),
+    ERROR_INVALID(3),
+    ERROR_EXISTS(4),
+    ERROR_ACCESS_TOKEN(5),
+    CHECKSUM_OK(6);
+
+    /** The code for this operation. */
+    private final int code;
+    
+    private Status(int code) {
+      this.code = code;
+    }
+
+    private static final int FIRST_CODE = values()[0].code;
+    /** Return the object represented by the code. */
+    private static Status valueOf(int code) {
+      final int i = code - FIRST_CODE;
+      return i < 0 || i >= values().length? null: values()[i];
+    }
+
+    /** Read from in */
+    public static Status read(DataInput in) throws IOException {
+      return valueOf(in.readShort());
+    }
+
+    /** Write to out */
+    public void write(DataOutput out) throws IOException {
+      out.writeShort(code);
+    }
+
+    /** Write to out */
+    public void writeOutputStream(OutputStream out) throws IOException {
+      out.write(new byte[] {(byte)(code >>> 8), (byte)code});
+    }
+  };
   
   
-  public static final int OP_STATUS_SUCCESS = 0;  
-  public static final int OP_STATUS_ERROR = 1;  
-  public static final int OP_STATUS_ERROR_CHECKSUM = 2;  
-  public static final int OP_STATUS_ERROR_INVALID = 3;  
-  public static final int OP_STATUS_ERROR_EXISTS = 4;  
-  public static final int OP_STATUS_ERROR_ACCESS_TOKEN = 5;
-  public static final int OP_STATUS_CHECKSUM_OK = 6;
+  /** @deprecated Deprecated at 0.21.  Use Op.WRITE_BLOCK instead. */
+  @Deprecated
+  public static final byte OP_WRITE_BLOCK = Op.WRITE_BLOCK.code;
+  /** @deprecated Deprecated at 0.21.  Use Op.READ_BLOCK instead. */
+  @Deprecated
+  public static final byte OP_READ_BLOCK = Op.READ_BLOCK.code;
+  /** @deprecated As of version 15, OP_READ_METADATA is no longer supported. */
+  @Deprecated
+  public static final byte OP_READ_METADATA = Op.READ_METADATA.code;
+  /** @deprecated Deprecated at 0.21.  Use Op.REPLACE_BLOCK instead. */
+  @Deprecated
+  public static final byte OP_REPLACE_BLOCK = Op.REPLACE_BLOCK.code;
+  /** @deprecated Deprecated at 0.21.  Use Op.COPY_BLOCK instead. */
+  @Deprecated
+  public static final byte OP_COPY_BLOCK = Op.COPY_BLOCK.code;
+  /** @deprecated Deprecated at 0.21.  Use Op.BLOCK_CHECKSUM instead. */
+  @Deprecated
+  public static final byte OP_BLOCK_CHECKSUM = Op.BLOCK_CHECKSUM.code;
+
+
+  /** @deprecated Deprecated at 0.21.  Use Status.SUCCESS instead. */
+  @Deprecated
+  public static final int OP_STATUS_SUCCESS = Status.SUCCESS.code;  
+  /** @deprecated Deprecated at 0.21.  Use Status.ERROR instead. */
+  @Deprecated
+  public static final int OP_STATUS_ERROR = Status.ERROR.code;
+  /** @deprecated Deprecated at 0.21.  Use Status.ERROR_CHECKSUM instead. */
+  @Deprecated
+  public static final int OP_STATUS_ERROR_CHECKSUM = Status.ERROR_CHECKSUM.code;
+  /** @deprecated Deprecated at 0.21.  Use Status.ERROR_INVALID instead. */
+  @Deprecated
+  public static final int OP_STATUS_ERROR_INVALID = Status.ERROR_INVALID.code;
+  /** @deprecated Deprecated at 0.21.  Use Status.ERROR_EXISTS instead. */
+  @Deprecated
+  public static final int OP_STATUS_ERROR_EXISTS = Status.ERROR_EXISTS.code;
+  /** @deprecated Deprecated at 0.21.  Use Status.ERROR_ACCESS_TOKEN instead.*/
+  @Deprecated
+  public static final int OP_STATUS_ERROR_ACCESS_TOKEN = Status.ERROR_ACCESS_TOKEN.code;
+  /** @deprecated Deprecated at 0.21.  Use Status.CHECKSUM_OK instead. */
+  @Deprecated
+  public static final int OP_STATUS_CHECKSUM_OK = Status.CHECKSUM_OK.code;
 
 
 
 
   /** Sender */
   /** Sender */
   public static class Sender {
   public static class Sender {
     /** Initialize a operation. */
     /** Initialize a operation. */
-    public static void op(DataOutputStream out, int op) throws IOException {
+    public static void op(DataOutputStream out, Op op) throws IOException {
       out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
       out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
-      out.write(op);
+      op.write(out);
     }
     }
 
 
     /** Send OP_READ_BLOCK */
     /** Send OP_READ_BLOCK */
     public static void opReadBlock(DataOutputStream out,
     public static void opReadBlock(DataOutputStream out,
         long blockId, long blockGs, long blockOffset, long blockLen,
         long blockId, long blockGs, long blockOffset, long blockLen,
         String clientName, AccessToken accessToken) throws IOException {
         String clientName, AccessToken accessToken) throws IOException {
-      op(out, OP_READ_BLOCK);
+      op(out, Op.READ_BLOCK);
 
 
       out.writeLong(blockId);
       out.writeLong(blockId);
       out.writeLong(blockGs);
       out.writeLong(blockGs);
@@ -90,7 +190,7 @@ public interface DataTransferProtocol {
         long blockId, long blockGs, int pipelineSize, boolean isRecovery,
         long blockId, long blockGs, int pipelineSize, boolean isRecovery,
         String client, DatanodeInfo src, DatanodeInfo[] targets,
         String client, DatanodeInfo src, DatanodeInfo[] targets,
         AccessToken accesstoken) throws IOException {
         AccessToken accesstoken) throws IOException {
-      op(out, OP_WRITE_BLOCK);
+      op(out, Op.WRITE_BLOCK);
 
 
       out.writeLong(blockId);
       out.writeLong(blockId);
       out.writeLong(blockGs);
       out.writeLong(blockGs);
@@ -114,7 +214,7 @@ public interface DataTransferProtocol {
     public static void opReplaceBlock(DataOutputStream out,
     public static void opReplaceBlock(DataOutputStream out,
         long blockId, long blockGs, String storageId, DatanodeInfo src,
         long blockId, long blockGs, String storageId, DatanodeInfo src,
         AccessToken accesstoken) throws IOException {
         AccessToken accesstoken) throws IOException {
-      op(out, OP_REPLACE_BLOCK);
+      op(out, Op.REPLACE_BLOCK);
 
 
       out.writeLong(blockId);
       out.writeLong(blockId);
       out.writeLong(blockGs);
       out.writeLong(blockGs);
@@ -127,7 +227,7 @@ public interface DataTransferProtocol {
     /** Send OP_COPY_BLOCK */
     /** Send OP_COPY_BLOCK */
     public static void opCopyBlock(DataOutputStream out,
     public static void opCopyBlock(DataOutputStream out,
         long blockId, long blockGs, AccessToken accesstoken) throws IOException {
         long blockId, long blockGs, AccessToken accesstoken) throws IOException {
-      op(out, OP_COPY_BLOCK);
+      op(out, Op.COPY_BLOCK);
 
 
       out.writeLong(blockId);
       out.writeLong(blockId);
       out.writeLong(blockGs);
       out.writeLong(blockGs);
@@ -138,7 +238,7 @@ public interface DataTransferProtocol {
     /** Send OP_BLOCK_CHECKSUM */
     /** Send OP_BLOCK_CHECKSUM */
     public static void opBlockChecksum(DataOutputStream out,
     public static void opBlockChecksum(DataOutputStream out,
         long blockId, long blockGs, AccessToken accesstoken) throws IOException {
         long blockId, long blockGs, AccessToken accesstoken) throws IOException {
-      op(out, OP_BLOCK_CHECKSUM);
+      op(out, Op.BLOCK_CHECKSUM);
 
 
       out.writeLong(blockId);
       out.writeLong(blockId);
       out.writeLong(blockGs);
       out.writeLong(blockGs);
@@ -150,12 +250,12 @@ public interface DataTransferProtocol {
   /** Receiver */
   /** Receiver */
   public static abstract class Receiver {
   public static abstract class Receiver {
     /** Initialize a operation. */
     /** Initialize a operation. */
-    public final byte op(DataInputStream in) throws IOException {
+    public final Op op(DataInputStream in) throws IOException {
       final short version = in.readShort();
       final short version = in.readShort();
       if (version != DATA_TRANSFER_VERSION) {
       if (version != DATA_TRANSFER_VERSION) {
         throw new IOException( "Version Mismatch" );
         throw new IOException( "Version Mismatch" );
       }
       }
-      return in.readByte();
+      return Op.read(in);
     }
     }
 
 
     /** Receive OP_READ_BLOCK */
     /** Receive OP_READ_BLOCK */

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

@@ -17,6 +17,8 @@
  */
  */
 package org.apache.hadoop.hdfs.server.balancer;
 package org.apache.hadoop.hdfs.server.balancer;
 
 
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR_ACCESS_TOKEN;
+
 import java.io.BufferedInputStream;
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
@@ -376,9 +378,9 @@ public class Balancer implements Tool {
     
     
     /* Receive a block copy response from the input stream */ 
     /* Receive a block copy response from the input stream */ 
     private void receiveResponse(DataInputStream in) throws IOException {
     private void receiveResponse(DataInputStream in) throws IOException {
-      short status = in.readShort();
-      if (status != DataTransferProtocol.OP_STATUS_SUCCESS) {
-        if (status == DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN)
+      DataTransferProtocol.Status status = DataTransferProtocol.Status.read(in);
+      if (status != DataTransferProtocol.Status.SUCCESS) {
+        if (status == ERROR_ACCESS_TOKEN)
           throw new IOException("block move failed due to access token error");
           throw new IOException("block move failed due to access token error");
         throw new IOException("block move is failed");
         throw new IOException("block move is failed");
       }
       }

+ 17 - 13
src/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -17,6 +17,10 @@
  */
  */
 package org.apache.hadoop.hdfs.server.datanode;
 package org.apache.hadoop.hdfs.server.datanode;
 
 
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR;
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.SUCCESS;
+import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
+
 import java.io.BufferedOutputStream;
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
@@ -25,7 +29,6 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 import java.util.LinkedList;
 import java.util.LinkedList;
-import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 import java.util.zip.Checksum;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -36,11 +39,12 @@ import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.PureJavaCrc32;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
-import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
 
 
 /** A class that receives a block and writes to its own disk, meanwhile
 /** A class that receives a block and writes to its own disk, meanwhile
  * may copies it to another site. If a throttler is provided,
  * may copies it to another site. If a throttler is provided,
@@ -672,7 +676,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
     }
     }
 
 
     // compute crc of partial chunk from data read in the block file.
     // compute crc of partial chunk from data read in the block file.
-    partialCrc = new CRC32();
+    partialCrc = new PureJavaCrc32();
     partialCrc.update(buf, 0, sizePartialChunk);
     partialCrc.update(buf, 0, sizePartialChunk);
     LOG.info("Read in partial CRC chunk from disk for block " + block);
     LOG.info("Read in partial CRC chunk from disk for block " + block);
 
 
@@ -823,7 +827,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
             }
             }
 
 
             replyOut.writeLong(expected);
             replyOut.writeLong(expected);
-            replyOut.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS);
+            SUCCESS.write(replyOut);
             replyOut.flush();
             replyOut.flush();
         } catch (Exception e) {
         } catch (Exception e) {
           if (running) {
           if (running) {
@@ -854,7 +858,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
       while (running && datanode.shouldRun && !lastPacketInBlock) {
       while (running && datanode.shouldRun && !lastPacketInBlock) {
 
 
         try {
         try {
-            short op = DataTransferProtocol.OP_STATUS_SUCCESS;
+            DataTransferProtocol.Status op = SUCCESS;
             boolean didRead = false;
             boolean didRead = false;
             long expected = -2;
             long expected = -2;
             try { 
             try { 
@@ -919,7 +923,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
             }
             }
             
             
             if (!didRead) {
             if (!didRead) {
-              op = DataTransferProtocol.OP_STATUS_ERROR;
+              op = ERROR;
             }
             }
             
             
             // If this is the last packet in block, then close block
             // If this is the last packet in block, then close block
@@ -948,7 +952,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
 
 
             // send my status back to upstream datanode
             // send my status back to upstream datanode
             replyOut.writeLong(expected); // send seqno upstream
             replyOut.writeLong(expected); // send seqno upstream
-            replyOut.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS);
+            SUCCESS.write(replyOut);
 
 
             LOG.debug("PacketResponder " + numTargets + 
             LOG.debug("PacketResponder " + numTargets + 
                       " for block " + block +
                       " for block " + block +
@@ -958,18 +962,18 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
             // forward responses from downstream datanodes.
             // forward responses from downstream datanodes.
             for (int i = 0; i < numTargets && datanode.shouldRun; i++) {
             for (int i = 0; i < numTargets && datanode.shouldRun; i++) {
               try {
               try {
-                if (op == DataTransferProtocol.OP_STATUS_SUCCESS) {
-                  op = mirrorIn.readShort();
-                  if (op != DataTransferProtocol.OP_STATUS_SUCCESS) {
+                if (op == SUCCESS) {
+                  op = Status.read(mirrorIn);
+                  if (op != SUCCESS) {
                     LOG.debug("PacketResponder for block " + block +
                     LOG.debug("PacketResponder for block " + block +
                               ": error code received from downstream " +
                               ": error code received from downstream " +
                               " datanode[" + i + "] " + op);
                               " datanode[" + i + "] " + op);
                   }
                   }
                 }
                 }
               } catch (Throwable e) {
               } catch (Throwable e) {
-                op = DataTransferProtocol.OP_STATUS_ERROR;
+                op = ERROR;
               }
               }
-              replyOut.writeShort(op);
+              op.write(replyOut);
             }
             }
             replyOut.flush();
             replyOut.flush();
             LOG.debug("PacketResponder " + block + " " + numTargets + 
             LOG.debug("PacketResponder " + block + " " + numTargets + 
@@ -982,7 +986,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
             // If we forwarded an error response from a downstream datanode
             // If we forwarded an error response from a downstream datanode
             // and we are acting on behalf of a client, then we quit. The 
             // and we are acting on behalf of a client, then we quit. The 
             // client will drive the recovery mechanism.
             // client will drive the recovery mechanism.
-            if (op == DataTransferProtocol.OP_STATUS_ERROR && receiver.clientName.length() > 0) {
+            if (op == ERROR && receiver.clientName.length() > 0) {
               running = false;
               running = false;
             }
             }
         } catch (IOException e) {
         } catch (IOException e) {

+ 41 - 40
src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -17,6 +17,12 @@
  */
  */
 package org.apache.hadoop.hdfs.server.datanode;
 package org.apache.hadoop.hdfs.server.datanode;
 
 
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.CHECKSUM_OK;
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR;
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR_ACCESS_TOKEN;
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.SUCCESS;
+import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
+
 import java.io.BufferedInputStream;
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
@@ -42,7 +48,6 @@ import org.apache.hadoop.security.AccessToken;
 import org.apache.hadoop.security.AccessTokenHandler;
 import org.apache.hadoop.security.AccessTokenHandler;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
-import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
 
 
 /**
 /**
  * Thread for processing incoming/outgoing data stream.
  * Thread for processing incoming/outgoing data stream.
@@ -79,7 +84,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
       in = new DataInputStream(
       in = new DataInputStream(
           new BufferedInputStream(NetUtils.getInputStream(s), 
           new BufferedInputStream(NetUtils.getInputStream(s), 
                                   SMALL_BUFFER_SIZE));
                                   SMALL_BUFFER_SIZE));
-      final byte op = op(in);
+      final DataTransferProtocol.Op op = op(in);
       boolean local = s.getInetAddress().equals(s.getLocalAddress());
       boolean local = s.getInetAddress().equals(s.getLocalAddress());
       // Make sure the xciver count is not exceeded
       // Make sure the xciver count is not exceeded
       int curXceiverCount = datanode.getXceiverCount();
       int curXceiverCount = datanode.getXceiverCount();
@@ -90,7 +95,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
       }
       }
       long startTime = DataNode.now();
       long startTime = DataNode.now();
       switch ( op ) {
       switch ( op ) {
-      case DataTransferProtocol.OP_READ_BLOCK:
+      case READ_BLOCK:
         opReadBlock(in);
         opReadBlock(in);
         datanode.myMetrics.readBlockOp.inc(DataNode.now() - startTime);
         datanode.myMetrics.readBlockOp.inc(DataNode.now() - startTime);
         if (local)
         if (local)
@@ -98,7 +103,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
         else
         else
           datanode.myMetrics.readsFromRemoteClient.inc();
           datanode.myMetrics.readsFromRemoteClient.inc();
         break;
         break;
-      case DataTransferProtocol.OP_WRITE_BLOCK:
+      case WRITE_BLOCK:
         opWriteBlock(in);
         opWriteBlock(in);
         datanode.myMetrics.writeBlockOp.inc(DataNode.now() - startTime);
         datanode.myMetrics.writeBlockOp.inc(DataNode.now() - startTime);
         if (local)
         if (local)
@@ -106,16 +111,16 @@ class DataXceiver extends DataTransferProtocol.Receiver
         else
         else
           datanode.myMetrics.writesFromRemoteClient.inc();
           datanode.myMetrics.writesFromRemoteClient.inc();
         break;
         break;
-      case DataTransferProtocol.OP_REPLACE_BLOCK: // for balancing purpose; send to a destination
+      case REPLACE_BLOCK: // for balancing purpose; send to a destination
         opReplaceBlock(in);
         opReplaceBlock(in);
         datanode.myMetrics.replaceBlockOp.inc(DataNode.now() - startTime);
         datanode.myMetrics.replaceBlockOp.inc(DataNode.now() - startTime);
         break;
         break;
-      case DataTransferProtocol.OP_COPY_BLOCK:
+      case COPY_BLOCK:
             // for balancing purpose; send to a proxy source
             // for balancing purpose; send to a proxy source
         opCopyBlock(in);
         opCopyBlock(in);
         datanode.myMetrics.copyBlockOp.inc(DataNode.now() - startTime);
         datanode.myMetrics.copyBlockOp.inc(DataNode.now() - startTime);
         break;
         break;
-      case DataTransferProtocol.OP_BLOCK_CHECKSUM: //get the checksum of a block
+      case BLOCK_CHECKSUM: //get the checksum of a block
         opBlockChecksum(in);
         opBlockChecksum(in);
         datanode.myMetrics.blockChecksumOp.inc(DataNode.now() - startTime);
         datanode.myMetrics.blockChecksumOp.inc(DataNode.now() - startTime);
         break;
         break;
@@ -150,7 +155,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
         && !datanode.accessTokenHandler.checkAccess(accessToken, null, blockId,
         && !datanode.accessTokenHandler.checkAccess(accessToken, null, blockId,
             AccessTokenHandler.AccessMode.READ)) {
             AccessTokenHandler.AccessMode.READ)) {
       try {
       try {
-        out.writeShort(DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN);
+        ERROR_ACCESS_TOKEN.write(out);
         out.flush();
         out.flush();
         throw new IOException("Access token verification failed, on client "
         throw new IOException("Access token verification failed, on client "
             + "request for reading block " + block);
             + "request for reading block " + block);
@@ -172,19 +177,19 @@ class DataXceiver extends DataTransferProtocol.Receiver
         blockSender = new BlockSender(block, startOffset, length,
         blockSender = new BlockSender(block, startOffset, length,
             true, true, false, datanode, clientTraceFmt);
             true, true, false, datanode, clientTraceFmt);
       } catch(IOException e) {
       } catch(IOException e) {
-        out.writeShort(DataTransferProtocol.OP_STATUS_ERROR);
+        ERROR.write(out);
         throw e;
         throw e;
       }
       }
 
 
-      out.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS); // send op status
+      SUCCESS.write(out); // send op status
       long read = blockSender.sendBlock(out, baseStream, null); // send data
       long read = blockSender.sendBlock(out, baseStream, null); // send data
 
 
       if (blockSender.isBlockReadFully()) {
       if (blockSender.isBlockReadFully()) {
         // See if client verification succeeded. 
         // See if client verification succeeded. 
         // This is an optional response from client.
         // This is an optional response from client.
         try {
         try {
-          if (in.readShort() == DataTransferProtocol.OP_STATUS_CHECKSUM_OK  && 
-              datanode.blockScanner != null) {
+          if (DataTransferProtocol.Status.read(in) == CHECKSUM_OK
+              && datanode.blockScanner != null) {
             datanode.blockScanner.verifiedByClient(block);
             datanode.blockScanner.verifiedByClient(block);
           }
           }
         } catch (IOException ignored) {}
         } catch (IOException ignored) {}
@@ -238,7 +243,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
             .getBlockId(), AccessTokenHandler.AccessMode.WRITE)) {
             .getBlockId(), AccessTokenHandler.AccessMode.WRITE)) {
       try {
       try {
         if (client.length() != 0) {
         if (client.length() != 0) {
-          replyOut.writeShort((short)DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN);
+          ERROR_ACCESS_TOKEN.write(replyOut);
           Text.writeString(replyOut, datanode.dnRegistration.getName());
           Text.writeString(replyOut, datanode.dnRegistration.getName());
           replyOut.flush();
           replyOut.flush();
         }
         }
@@ -255,7 +260,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
     BlockReceiver blockReceiver = null; // responsible for data handling
     BlockReceiver blockReceiver = null; // responsible for data handling
     String mirrorNode = null;           // the name:port of next target
     String mirrorNode = null;           // the name:port of next target
     String firstBadLink = "";           // first datanode that failed in connection setup
     String firstBadLink = "";           // first datanode that failed in connection setup
-    short mirrorInStatus = (short)DataTransferProtocol.OP_STATUS_SUCCESS;
+    DataTransferProtocol.Status mirrorInStatus = SUCCESS;
     try {
     try {
       // open a block receiver and check if the block does not exist
       // open a block receiver and check if the block does not exist
       blockReceiver = new BlockReceiver(block, in, 
       blockReceiver = new BlockReceiver(block, in, 
@@ -296,9 +301,9 @@ class DataXceiver extends DataTransferProtocol.Receiver
 
 
           // read connect ack (only for clients, not for replication req)
           // read connect ack (only for clients, not for replication req)
           if (client.length() != 0) {
           if (client.length() != 0) {
-            mirrorInStatus = mirrorIn.readShort();
+            mirrorInStatus = DataTransferProtocol.Status.read(mirrorIn);
             firstBadLink = Text.readString(mirrorIn);
             firstBadLink = Text.readString(mirrorIn);
-            if (LOG.isDebugEnabled() || mirrorInStatus != DataTransferProtocol.OP_STATUS_SUCCESS) {
+            if (LOG.isDebugEnabled() || mirrorInStatus != SUCCESS) {
               LOG.info("Datanode " + targets.length +
               LOG.info("Datanode " + targets.length +
                        " got response for connect ack " +
                        " got response for connect ack " +
                        " from downstream datanode with firstbadlink as " +
                        " from downstream datanode with firstbadlink as " +
@@ -308,7 +313,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
 
 
         } catch (IOException e) {
         } catch (IOException e) {
           if (client.length() != 0) {
           if (client.length() != 0) {
-            replyOut.writeShort((short)DataTransferProtocol.OP_STATUS_ERROR);
+            ERROR.write(replyOut);
             Text.writeString(replyOut, mirrorNode);
             Text.writeString(replyOut, mirrorNode);
             replyOut.flush();
             replyOut.flush();
           }
           }
@@ -331,12 +336,12 @@ class DataXceiver extends DataTransferProtocol.Receiver
 
 
       // send connect ack back to source (only for clients)
       // send connect ack back to source (only for clients)
       if (client.length() != 0) {
       if (client.length() != 0) {
-        if (LOG.isDebugEnabled() || mirrorInStatus != DataTransferProtocol.OP_STATUS_SUCCESS) {
+        if (LOG.isDebugEnabled() || mirrorInStatus != SUCCESS) {
           LOG.info("Datanode " + targets.length +
           LOG.info("Datanode " + targets.length +
                    " forwarding connect ack to upstream firstbadlink is " +
                    " forwarding connect ack to upstream firstbadlink is " +
                    firstBadLink);
                    firstBadLink);
         }
         }
-        replyOut.writeShort(mirrorInStatus);
+        mirrorInStatus.write(replyOut);
         Text.writeString(replyOut, firstBadLink);
         Text.writeString(replyOut, firstBadLink);
         replyOut.flush();
         replyOut.flush();
       }
       }
@@ -387,7 +392,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
         && !datanode.accessTokenHandler.checkAccess(accessToken, null, block
         && !datanode.accessTokenHandler.checkAccess(accessToken, null, block
             .getBlockId(), AccessTokenHandler.AccessMode.READ)) {
             .getBlockId(), AccessTokenHandler.AccessMode.READ)) {
       try {
       try {
-        out.writeShort(DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN);
+        ERROR_ACCESS_TOKEN.write(out);
         out.flush();
         out.flush();
         throw new IOException(
         throw new IOException(
             "Access token verification failed, on getBlockChecksum() "
             "Access token verification failed, on getBlockChecksum() "
@@ -418,7 +423,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
       }
       }
 
 
       //write reply
       //write reply
-      out.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS);
+      SUCCESS.write(out);
       out.writeInt(bytesPerCRC);
       out.writeInt(bytesPerCRC);
       out.writeLong(crcPerBlock);
       out.writeLong(crcPerBlock);
       md5.write(out);
       md5.write(out);
@@ -443,17 +448,14 @@ class DataXceiver extends DataTransferProtocol.Receiver
             AccessTokenHandler.AccessMode.COPY)) {
             AccessTokenHandler.AccessMode.COPY)) {
       LOG.warn("Invalid access token in request from "
       LOG.warn("Invalid access token in request from "
           + s.getRemoteSocketAddress() + " for copying block " + block);
           + s.getRemoteSocketAddress() + " for copying block " + block);
-      sendResponse(s,
-          (short) DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN,
-          datanode.socketWriteTimeout);
+      sendResponse(s, ERROR_ACCESS_TOKEN, datanode.socketWriteTimeout);
       return;
       return;
     }
     }
 
 
     if (!dataXceiverServer.balanceThrottler.acquire()) { // not able to start
     if (!dataXceiverServer.balanceThrottler.acquire()) { // not able to start
       LOG.info("Not able to copy block " + blockId + " to " 
       LOG.info("Not able to copy block " + blockId + " to " 
           + s.getRemoteSocketAddress() + " because threads quota is exceeded.");
           + s.getRemoteSocketAddress() + " because threads quota is exceeded.");
-      sendResponse(s, (short)DataTransferProtocol.OP_STATUS_ERROR, 
-          datanode.socketWriteTimeout);
+      sendResponse(s, ERROR, datanode.socketWriteTimeout);
       return;
       return;
     }
     }
 
 
@@ -473,7 +475,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
           baseStream, SMALL_BUFFER_SIZE));
           baseStream, SMALL_BUFFER_SIZE));
 
 
       // send status first
       // send status first
-      reply.writeShort((short)DataTransferProtocol.OP_STATUS_SUCCESS);
+      SUCCESS.write(reply);
       // send block content to the target
       // send block content to the target
       long read = blockSender.sendBlock(reply, baseStream, 
       long read = blockSender.sendBlock(reply, baseStream, 
                                         dataXceiverServer.balanceThrottler);
                                         dataXceiverServer.balanceThrottler);
@@ -515,22 +517,20 @@ class DataXceiver extends DataTransferProtocol.Receiver
             AccessTokenHandler.AccessMode.REPLACE)) {
             AccessTokenHandler.AccessMode.REPLACE)) {
       LOG.warn("Invalid access token in request from "
       LOG.warn("Invalid access token in request from "
           + s.getRemoteSocketAddress() + " for replacing block " + block);
           + s.getRemoteSocketAddress() + " for replacing block " + block);
-      sendResponse(s, (short)DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN,
-          datanode.socketWriteTimeout);
+      sendResponse(s, ERROR_ACCESS_TOKEN, datanode.socketWriteTimeout);
       return;
       return;
     }
     }
 
 
     if (!dataXceiverServer.balanceThrottler.acquire()) { // not able to start
     if (!dataXceiverServer.balanceThrottler.acquire()) { // not able to start
       LOG.warn("Not able to receive block " + blockId + " from " 
       LOG.warn("Not able to receive block " + blockId + " from " 
           + s.getRemoteSocketAddress() + " because threads quota is exceeded.");
           + s.getRemoteSocketAddress() + " because threads quota is exceeded.");
-      sendResponse(s, (short)DataTransferProtocol.OP_STATUS_ERROR, 
-          datanode.socketWriteTimeout);
+      sendResponse(s, ERROR, datanode.socketWriteTimeout);
       return;
       return;
     }
     }
 
 
     Socket proxySock = null;
     Socket proxySock = null;
     DataOutputStream proxyOut = null;
     DataOutputStream proxyOut = null;
-    short opStatus = DataTransferProtocol.OP_STATUS_SUCCESS;
+    DataTransferProtocol.Status opStatus = SUCCESS;
     BlockReceiver blockReceiver = null;
     BlockReceiver blockReceiver = null;
     DataInputStream proxyReply = null;
     DataInputStream proxyReply = null;
     
     
@@ -554,9 +554,10 @@ class DataXceiver extends DataTransferProtocol.Receiver
       // receive the response from the proxy
       // receive the response from the proxy
       proxyReply = new DataInputStream(new BufferedInputStream(
       proxyReply = new DataInputStream(new BufferedInputStream(
           NetUtils.getInputStream(proxySock), BUFFER_SIZE));
           NetUtils.getInputStream(proxySock), BUFFER_SIZE));
-      short status = proxyReply.readShort();
-      if (status != DataTransferProtocol.OP_STATUS_SUCCESS) {
-        if (status == DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN) {
+      final DataTransferProtocol.Status status
+          = DataTransferProtocol.Status.read(proxyReply);
+      if (status != SUCCESS) {
+        if (status == ERROR_ACCESS_TOKEN) {
           throw new IOException("Copy block " + block + " from "
           throw new IOException("Copy block " + block + " from "
               + proxySock.getRemoteSocketAddress()
               + proxySock.getRemoteSocketAddress()
               + " failed due to access token error");
               + " failed due to access token error");
@@ -581,11 +582,11 @@ class DataXceiver extends DataTransferProtocol.Receiver
           " from " + s.getRemoteSocketAddress());
           " from " + s.getRemoteSocketAddress());
       
       
     } catch (IOException ioe) {
     } catch (IOException ioe) {
-      opStatus = DataTransferProtocol.OP_STATUS_ERROR;
+      opStatus = ERROR;
       throw ioe;
       throw ioe;
     } finally {
     } finally {
       // receive the last byte that indicates the proxy released its thread resource
       // receive the last byte that indicates the proxy released its thread resource
-      if (opStatus == DataTransferProtocol.OP_STATUS_SUCCESS) {
+      if (opStatus == SUCCESS) {
         try {
         try {
           proxyReply.readChar();
           proxyReply.readChar();
         } catch (IOException ignored) {
         } catch (IOException ignored) {
@@ -613,12 +614,12 @@ class DataXceiver extends DataTransferProtocol.Receiver
    * @param opStatus status message to write
    * @param opStatus status message to write
    * @param timeout send timeout
    * @param timeout send timeout
    **/
    **/
-  private void sendResponse(Socket s, short opStatus, long timeout) 
-                                                       throws IOException {
+  private void sendResponse(Socket s, DataTransferProtocol.Status opStatus,
+      long timeout) throws IOException {
     DataOutputStream reply = 
     DataOutputStream reply = 
       new DataOutputStream(NetUtils.getOutputStream(s, timeout));
       new DataOutputStream(NetUtils.getOutputStream(s, timeout));
     try {
     try {
-      reply.writeShort(opStatus);
+      opStatus.write(reply);
       reply.flush();
       reply.flush();
     } finally {
     } finally {
       IOUtils.closeStream(reply);
       IOUtils.closeStream(reply);

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

@@ -65,7 +65,7 @@ class DatanodeJspHelper {
       JspHelper.printGotoForm(out, namenodeInfoPort, target);
       JspHelper.printGotoForm(out, namenodeInfoPort, target);
     } else {
     } else {
       if (!targetStatus.isDir()) { // a file
       if (!targetStatus.isDir()) { // a file
-        List<LocatedBlock> blocks = dfs.namenode.getBlockLocations(dir, 0, 1)
+        List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(dir, 0, 1)
             .getLocatedBlocks();
             .getLocatedBlocks();
 
 
         LocatedBlock firstBlock = null;
         LocatedBlock firstBlock = null;
@@ -205,7 +205,7 @@ class DatanodeJspHelper {
 
 
     final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(),
     final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(),
         JspHelper.conf);
         JspHelper.conf);
-    List<LocatedBlock> blocks = dfs.namenode.getBlockLocations(filename, 0,
+    List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
         Long.MAX_VALUE).getLocatedBlocks();
         Long.MAX_VALUE).getLocatedBlocks();
     // Add the various links for looking at the file contents
     // Add the various links for looking at the file contents
     // URL for downloading the full file
     // URL for downloading the full file
@@ -320,7 +320,7 @@ class DatanodeJspHelper {
     AccessToken accessToken = AccessToken.DUMMY_TOKEN;
     AccessToken accessToken = AccessToken.DUMMY_TOKEN;
     if (JspHelper.conf.getBoolean(
     if (JspHelper.conf.getBoolean(
         AccessTokenHandler.STRING_ENABLE_ACCESS_TOKEN, false)) {
         AccessTokenHandler.STRING_ENABLE_ACCESS_TOKEN, false)) {
-      List<LocatedBlock> blks = dfs.namenode.getBlockLocations(filename, 0,
+      List<LocatedBlock> blks = dfs.getNamenode().getBlockLocations(filename, 0,
           Long.MAX_VALUE).getLocatedBlocks();
           Long.MAX_VALUE).getLocatedBlocks();
       if (blks == null || blks.size() == 0) {
       if (blks == null || blks.size() == 0) {
         out.print("Can't locate file blocks");
         out.print("Can't locate file blocks");
@@ -390,7 +390,7 @@ class DatanodeJspHelper {
     // determine data for the next link
     // determine data for the next link
     if (startOffset + chunkSizeToView >= blockSize) {
     if (startOffset + chunkSizeToView >= blockSize) {
       // we have to go to the next block from this point onwards
       // we have to go to the next block from this point onwards
-      List<LocatedBlock> blocks = dfs.namenode.getBlockLocations(filename, 0,
+      List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
           Long.MAX_VALUE).getLocatedBlocks();
           Long.MAX_VALUE).getLocatedBlocks();
       for (int i = 0; i < blocks.size(); i++) {
       for (int i = 0; i < blocks.size(); i++) {
         if (blocks.get(i).getBlock().getBlockId() == blockId) {
         if (blocks.get(i).getBlock().getBlockId() == blockId) {
@@ -440,7 +440,7 @@ class DatanodeJspHelper {
     int prevPort = req.getServerPort();
     int prevPort = req.getServerPort();
     int prevDatanodePort = datanodePort;
     int prevDatanodePort = datanodePort;
     if (startOffset == 0) {
     if (startOffset == 0) {
-      List<LocatedBlock> blocks = dfs.namenode.getBlockLocations(filename, 0,
+      List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
           Long.MAX_VALUE).getLocatedBlocks();
           Long.MAX_VALUE).getLocatedBlocks();
       for (int i = 0; i < blocks.size(); i++) {
       for (int i = 0; i < blocks.size(); i++) {
         if (blocks.get(i).getBlock().getBlockId() == blockId) {
         if (blocks.get(i).getBlock().getBlockId() == blockId) {
@@ -546,7 +546,7 @@ class DatanodeJspHelper {
     // fetch the block from the datanode that has the last block for this file
     // fetch the block from the datanode that has the last block for this file
     final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(),
     final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(),
         JspHelper.conf);
         JspHelper.conf);
-    List<LocatedBlock> blocks = dfs.namenode.getBlockLocations(filename, 0,
+    List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
         Long.MAX_VALUE).getLocatedBlocks();
         Long.MAX_VALUE).getLocatedBlocks();
     if (blocks == null || blocks.size() == 0) {
     if (blocks == null || blocks.size() == 0) {
       out.print("No datanodes contain blocks of file " + filename);
       out.print("No datanodes contain blocks of file " + filename);

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

@@ -171,7 +171,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       return Block.GRANDFATHER_GENERATION_STAMP;
       return Block.GRANDFATHER_GENERATION_STAMP;
     }
     }
 
 
-    void getVolumeMap(HashMap<Block, DatanodeBlockInfo> volumeMap, FSVolume volume) {
+    void getVolumeMap(HashMap<Block, ReplicaInfo> volumeMap, FSVolume volume) {
       if (children != null) {
       if (children != null) {
         for (int i = 0; i < children.length; i++) {
         for (int i = 0; i < children.length; i++) {
           children[i].getVolumeMap(volumeMap, volume);
           children[i].getVolumeMap(volumeMap, volume);
@@ -183,7 +183,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
         if (Block.isBlockFilename(blockFiles[i])) {
         if (Block.isBlockFilename(blockFiles[i])) {
           long genStamp = getGenerationStampFromFile(blockFiles, blockFiles[i]);
           long genStamp = getGenerationStampFromFile(blockFiles, blockFiles[i]);
           volumeMap.put(new Block(blockFiles[i], blockFiles[i].length(), genStamp), 
           volumeMap.put(new Block(blockFiles[i], blockFiles[i].length(), genStamp), 
-                        new DatanodeBlockInfo(volume, blockFiles[i]));
+                        new ReplicaInfo(volume, blockFiles[i]));
         }
         }
       }
       }
     }
     }
@@ -403,7 +403,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       DiskChecker.checkDir(tmpDir);
       DiskChecker.checkDir(tmpDir);
     }
     }
       
       
-    void getVolumeMap(HashMap<Block, DatanodeBlockInfo> volumeMap) {
+    void getVolumeMap(HashMap<Block, ReplicaInfo> volumeMap) {
       dataDir.getVolumeMap(volumeMap, this);
       dataDir.getVolumeMap(volumeMap, this);
     }
     }
       
       
@@ -496,7 +496,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       return remaining;
       return remaining;
     }
     }
       
       
-    synchronized void getVolumeMap(HashMap<Block, DatanodeBlockInfo> volumeMap) {
+    synchronized void getVolumeMap(HashMap<Block, ReplicaInfo> volumeMap) {
       for (int idx = 0; idx < volumes.length; idx++) {
       for (int idx = 0; idx < volumes.length; idx++) {
         volumes[idx].getVolumeMap(volumeMap);
         volumes[idx].getVolumeMap(volumeMap);
       }
       }
@@ -653,7 +653,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
   FSVolumeSet volumes;
   FSVolumeSet volumes;
   private HashMap<Block,ActiveFile> ongoingCreates = new HashMap<Block,ActiveFile>();
   private HashMap<Block,ActiveFile> ongoingCreates = new HashMap<Block,ActiveFile>();
   private int maxBlocksPerDir = 0;
   private int maxBlocksPerDir = 0;
-  HashMap<Block,DatanodeBlockInfo> volumeMap = null;
+  HashMap<Block,ReplicaInfo> volumeMap = null;
   static  Random random = new Random();
   static  Random random = new Random();
 
 
   // Used for synchronizing access to usage stats
   // Used for synchronizing access to usage stats
@@ -669,7 +669,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       volArray[idx] = new FSVolume(storage.getStorageDir(idx).getCurrentDir(), conf);
       volArray[idx] = new FSVolume(storage.getStorageDir(idx).getCurrentDir(), conf);
     }
     }
     volumes = new FSVolumeSet(volArray);
     volumes = new FSVolumeSet(volArray);
-    volumeMap = new HashMap<Block, DatanodeBlockInfo>();
+    volumeMap = new HashMap<Block, ReplicaInfo>();
     volumes.getVolumeMap(volumeMap);
     volumes.getVolumeMap(volumeMap);
     registerMBean(storage.getStorageID());
     registerMBean(storage.getStorageID());
   }
   }
@@ -742,7 +742,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
   public synchronized BlockInputStreams getTmpInputStreams(Block b, 
   public synchronized BlockInputStreams getTmpInputStreams(Block b, 
                           long blkOffset, long ckoff) throws IOException {
                           long blkOffset, long ckoff) throws IOException {
 
 
-    DatanodeBlockInfo info = volumeMap.get(b);
+    ReplicaInfo info = volumeMap.get(b);
     if (info == null) {
     if (info == null) {
       throw new IOException("Block " + b + " does not exist in volumeMap.");
       throw new IOException("Block " + b + " does not exist in volumeMap.");
     }
     }
@@ -777,7 +777,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
    * @return - true if the specified block was detached
    * @return - true if the specified block was detached
    */
    */
   public boolean detachBlock(Block block, int numLinks) throws IOException {
   public boolean detachBlock(Block block, int numLinks) throws IOException {
-    DatanodeBlockInfo info = null;
+    ReplicaInfo info = null;
 
 
     synchronized (this) {
     synchronized (this) {
       info = volumeMap.get(block);
       info = volumeMap.get(block);
@@ -1006,12 +1006,12 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
         v = volumes.getNextVolume(blockSize);
         v = volumes.getNextVolume(blockSize);
         // create temporary file to hold block in the designated volume
         // create temporary file to hold block in the designated volume
         f = createTmpFile(v, b);
         f = createTmpFile(v, b);
-        volumeMap.put(b, new DatanodeBlockInfo(v));
+        volumeMap.put(b, new ReplicaInfo(v));
       } else if (f != null) {
       } else if (f != null) {
         DataNode.LOG.info("Reopen already-open Block for append " + b);
         DataNode.LOG.info("Reopen already-open Block for append " + b);
         // create or reuse temporary file to hold block in the designated volume
         // create or reuse temporary file to hold block in the designated volume
         v = volumeMap.get(b).getVolume();
         v = volumeMap.get(b).getVolume();
-        volumeMap.put(b, new DatanodeBlockInfo(v));
+        volumeMap.put(b, new ReplicaInfo(v));
       } else {
       } else {
         // reopening block for appending to it.
         // reopening block for appending to it.
         DataNode.LOG.info("Reopen Block for append " + b);
         DataNode.LOG.info("Reopen Block for append " + b);
@@ -1042,7 +1042,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
                                   " to tmp dir " + f);
                                   " to tmp dir " + f);
           }
           }
         }
         }
-        volumeMap.put(b, new DatanodeBlockInfo(v));
+        volumeMap.put(b, new ReplicaInfo(v));
       }
       }
       if (f == null) {
       if (f == null) {
         DataNode.LOG.warn("Block " + b + " reopen failed " +
         DataNode.LOG.warn("Block " + b + " reopen failed " +
@@ -1147,7 +1147,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
         
         
     File dest = null;
     File dest = null;
     dest = v.addBlock(b, f);
     dest = v.addBlock(b, f);
-    volumeMap.put(b, new DatanodeBlockInfo(v, dest));
+    volumeMap.put(b, new ReplicaInfo(v, dest));
     ongoingCreates.remove(b);
     ongoingCreates.remove(b);
   }
   }
 
 
@@ -1248,7 +1248,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   public void validateBlockMetadata(Block b) throws IOException {
   public void validateBlockMetadata(Block b) throws IOException {
-    DatanodeBlockInfo info = volumeMap.get(b);
+    ReplicaInfo info = volumeMap.get(b);
     if (info == null) {
     if (info == null) {
       throw new IOException("Block " + b + " does not exist in volumeMap.");
       throw new IOException("Block " + b + " does not exist in volumeMap.");
     }
     }
@@ -1306,7 +1306,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       FSVolume v;
       FSVolume v;
       synchronized (this) {
       synchronized (this) {
         f = getFile(invalidBlks[i]);
         f = getFile(invalidBlks[i]);
-        DatanodeBlockInfo dinfo = volumeMap.get(invalidBlks[i]);
+        ReplicaInfo dinfo = volumeMap.get(invalidBlks[i]);
         if (dinfo == null) {
         if (dinfo == null) {
           DataNode.LOG.warn("Unexpected error trying to delete block "
           DataNode.LOG.warn("Unexpected error trying to delete block "
                            + invalidBlks[i] + 
                            + invalidBlks[i] + 
@@ -1369,7 +1369,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
    * Turn the block identifier into a filename.
    * Turn the block identifier into a filename.
    */
    */
   public synchronized File getFile(Block b) {
   public synchronized File getFile(Block b) {
-    DatanodeBlockInfo info = volumeMap.get(b);
+    ReplicaInfo info = volumeMap.get(b);
     if (info != null) {
     if (info != null) {
       return info.getFile();
       return info.getFile();
     }
     }
@@ -1448,8 +1448,8 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
    * generation stamp</li>
    * generation stamp</li>
    * <li>If the block length in memory does not match the actual block file length
    * <li>If the block length in memory does not match the actual block file length
    * then mark the block as corrupt and update the block length in memory</li>
    * then mark the block as corrupt and update the block length in memory</li>
-   * <li>If the file in {@link DatanodeBlockInfo} does not match the file on
-   * the disk, update {@link DatanodeBlockInfo} with the correct file</li>
+   * <li>If the file in {@link ReplicaInfo} does not match the file on
+   * the disk, update {@link ReplicaInfo} with the correct file</li>
    * </ul>
    * </ul>
    *
    *
    * @param blockId Block that differs
    * @param blockId Block that differs
@@ -1472,7 +1472,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
           Block.getGenerationStamp(diskMetaFile.getName()) :
           Block.getGenerationStamp(diskMetaFile.getName()) :
             Block.GRANDFATHER_GENERATION_STAMP;
             Block.GRANDFATHER_GENERATION_STAMP;
 
 
-      DatanodeBlockInfo memBlockInfo = volumeMap.get(block);
+      ReplicaInfo memBlockInfo = volumeMap.get(block);
       if (diskFile == null || !diskFile.exists()) {
       if (diskFile == null || !diskFile.exists()) {
         if (memBlockInfo == null) {
         if (memBlockInfo == null) {
           // Block file does not exist and block does not exist in memory
           // Block file does not exist and block does not exist in memory
@@ -1507,7 +1507,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
        */
        */
       if (memBlockInfo == null) {
       if (memBlockInfo == null) {
         // Block is missing in memory - add the block to volumeMap
         // Block is missing in memory - add the block to volumeMap
-        DatanodeBlockInfo diskBlockInfo = new DatanodeBlockInfo(vol, diskFile);
+        ReplicaInfo diskBlockInfo = new ReplicaInfo(vol, diskFile);
         Block diskBlock = new Block(diskFile, diskFile.length(), diskGS);
         Block diskBlock = new Block(diskFile, diskFile.length(), diskGS);
         volumeMap.put(diskBlock, diskBlockInfo);
         volumeMap.put(diskBlock, diskBlockInfo);
         if (datanode.blockScanner != null) {
         if (datanode.blockScanner != null) {
@@ -1540,7 +1540,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
             + memFile.getAbsolutePath()
             + memFile.getAbsolutePath()
             + " does not exist. Updating it to the file found during scan "
             + " does not exist. Updating it to the file found during scan "
             + diskFile.getAbsolutePath());
             + diskFile.getAbsolutePath());
-        DatanodeBlockInfo info = volumeMap.remove(memBlock);
+        ReplicaInfo info = volumeMap.remove(memBlock);
         info.setFile(diskFile);
         info.setFile(diskFile);
         memFile = diskFile;
         memFile = diskFile;
 
 
@@ -1571,7 +1571,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
           DataNode.LOG.warn("Updating generation stamp for block " + blockId
           DataNode.LOG.warn("Updating generation stamp for block " + blockId
               + " from " + memBlock.getGenerationStamp() + " to " + gs);
               + " from " + memBlock.getGenerationStamp() + " to " + gs);
 
 
-          DatanodeBlockInfo info = volumeMap.remove(memBlock);
+          ReplicaInfo info = volumeMap.remove(memBlock);
           memBlock.setGenerationStamp(gs);
           memBlock.setGenerationStamp(gs);
           volumeMap.put(memBlock, info);
           volumeMap.put(memBlock, info);
         }
         }
@@ -1583,7 +1583,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
         corruptBlock = new Block(memBlock);
         corruptBlock = new Block(memBlock);
         DataNode.LOG.warn("Updating size of block " + blockId + " from "
         DataNode.LOG.warn("Updating size of block " + blockId + " from "
             + memBlock.getNumBytes() + " to " + memFile.length());
             + memBlock.getNumBytes() + " to " + memFile.length());
-        DatanodeBlockInfo info = volumeMap.remove(memBlock);
+        ReplicaInfo info = volumeMap.remove(memBlock);
         memBlock.setNumBytes(memFile.length());
         memBlock.setNumBytes(memFile.length());
         volumeMap.put(memBlock, info);
         volumeMap.put(memBlock, info);
       }
       }

+ 3 - 3
src/java/org/apache/hadoop/hdfs/server/datanode/DatanodeBlockInfo.java → src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java

@@ -32,19 +32,19 @@ import org.apache.hadoop.io.IOUtils;
  * This class is used by the datanode to maintain the map from a block 
  * This class is used by the datanode to maintain the map from a block 
  * to its metadata.
  * to its metadata.
  */
  */
-class DatanodeBlockInfo {
+class ReplicaInfo {
 
 
   private FSVolume volume;       // volume where the block belongs
   private FSVolume volume;       // volume where the block belongs
   private File     file;         // block file
   private File     file;         // block file
   private boolean detached;      // copy-on-write done for block
   private boolean detached;      // copy-on-write done for block
 
 
-  DatanodeBlockInfo(FSVolume vol, File file) {
+  ReplicaInfo(FSVolume vol, File file) {
     this.volume = vol;
     this.volume = vol;
     this.file = file;
     this.file = file;
     detached = false;
     detached = false;
   }
   }
 
 
-  DatanodeBlockInfo(FSVolume vol) {
+  ReplicaInfo(FSVolume vol) {
     this.volume = vol;
     this.volume = vol;
     this.file = null;
     this.file = null;
     detached = false;
     detached = false;

+ 276 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfo.java

@@ -0,0 +1,276 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/**
+ * Internal class for block metadata.
+ */
+class BlockInfo extends Block {
+  private INodeFile inode;
+
+  /**
+   * This array contains triplets of references.
+   * For each i-th datanode the block belongs to
+   * triplets[3*i] is the reference to the DatanodeDescriptor
+   * and triplets[3*i+1] and triplets[3*i+2] are references 
+   * to the previous and the next blocks, respectively, in the 
+   * list of blocks belonging to this data-node.
+   */
+  private Object[] triplets;
+
+  BlockInfo(Block blk, int replication) {
+    super(blk);
+    this.triplets = new Object[3*replication];
+    this.inode = null;
+  }
+
+  INodeFile getINode() {
+    return inode;
+  }
+
+  void setINode(INodeFile inode) {
+    this.inode = inode;
+  }
+
+  DatanodeDescriptor getDatanode(int index) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
+    DatanodeDescriptor node = (DatanodeDescriptor)triplets[index*3];
+    assert node == null || 
+        DatanodeDescriptor.class.getName().equals(node.getClass().getName()) : 
+              "DatanodeDescriptor is expected at " + index*3;
+    return node;
+  }
+
+  BlockInfo getPrevious(int index) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
+    BlockInfo info = (BlockInfo)triplets[index*3+1];
+    assert info == null || 
+        BlockInfo.class.getName().equals(info.getClass().getName()) : 
+              "BlockInfo is expected at " + index*3;
+    return info;
+  }
+
+  BlockInfo getNext(int index) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
+    BlockInfo info = (BlockInfo)triplets[index*3+2];
+    assert info == null || 
+        BlockInfo.class.getName().equals(info.getClass().getName()) : 
+              "BlockInfo is expected at " + index*3;
+    return info;
+  }
+
+  void setDatanode(int index, DatanodeDescriptor node) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
+    triplets[index*3] = node;
+  }
+
+  void setPrevious(int index, BlockInfo to) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
+    triplets[index*3+1] = to;
+  }
+
+  void setNext(int index, BlockInfo to) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
+    triplets[index*3+2] = to;
+  }
+
+  int getCapacity() {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
+    return triplets.length / 3;
+  }
+
+  /**
+   * Ensure that there is enough  space to include num more triplets.
+   * @return first free triplet index.
+   */
+  private int ensureCapacity(int num) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    int last = numNodes();
+    if(triplets.length >= (last+num)*3)
+      return last;
+    /* Not enough space left. Create a new array. Should normally 
+     * happen only when replication is manually increased by the user. */
+    Object[] old = triplets;
+    triplets = new Object[(last+num)*3];
+    for(int i=0; i < last*3; i++) {
+      triplets[i] = old[i];
+    }
+    return last;
+  }
+
+  /**
+   * Count the number of data-nodes the block belongs to.
+   */
+  int numNodes() {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
+    for(int idx = getCapacity()-1; idx >= 0; idx--) {
+      if(getDatanode(idx) != null)
+        return idx+1;
+    }
+    return 0;
+  }
+
+  /**
+   * Add data-node this block belongs to.
+   */
+  boolean addNode(DatanodeDescriptor node) {
+    if(findDatanode(node) >= 0) // the node is already there
+      return false;
+    // find the last null node
+    int lastNode = ensureCapacity(1);
+    setDatanode(lastNode, node);
+    setNext(lastNode, null);
+    setPrevious(lastNode, null);
+    return true;
+  }
+
+  /**
+   * Remove data-node from the block.
+   */
+  boolean removeNode(DatanodeDescriptor node) {
+    int dnIndex = findDatanode(node);
+    if(dnIndex < 0) // the node is not found
+      return false;
+    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
+      "Block is still in the list and must be removed first.";
+    // find the last not null node
+    int lastNode = numNodes()-1; 
+    // replace current node triplet by the lastNode one 
+    setDatanode(dnIndex, getDatanode(lastNode));
+    setNext(dnIndex, getNext(lastNode)); 
+    setPrevious(dnIndex, getPrevious(lastNode)); 
+    // set the last triplet to null
+    setDatanode(lastNode, null);
+    setNext(lastNode, null); 
+    setPrevious(lastNode, null); 
+    return true;
+  }
+
+  /**
+   * Find specified DatanodeDescriptor.
+   * @param dn
+   * @return index or -1 if not found.
+   */
+  int findDatanode(DatanodeDescriptor dn) {
+    int len = getCapacity();
+    for(int idx = 0; idx < len; idx++) {
+      DatanodeDescriptor cur = getDatanode(idx);
+      if(cur == dn)
+        return idx;
+      if(cur == null)
+        break;
+    }
+    return -1;
+  }
+
+  /**
+   * Insert this block into the head of the list of blocks 
+   * related to the specified DatanodeDescriptor.
+   * If the head is null then form a new list.
+   * @return current block as the new head of the list.
+   */
+  BlockInfo listInsert(BlockInfo head, DatanodeDescriptor dn) {
+    int dnIndex = this.findDatanode(dn);
+    assert dnIndex >= 0 : "Data node is not found: current";
+    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
+            "Block is already in the list and cannot be inserted.";
+    this.setPrevious(dnIndex, null);
+    this.setNext(dnIndex, head);
+    if(head != null)
+      head.setPrevious(head.findDatanode(dn), this);
+    return this;
+  }
+
+  /**
+   * Remove this block from the list of blocks 
+   * related to the specified DatanodeDescriptor.
+   * If this block is the head of the list then return the next block as 
+   * the new head.
+   * @return the new head of the list or null if the list becomes
+   * empty after deletion.
+   */
+  BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
+    if(head == null)
+      return null;
+    int dnIndex = this.findDatanode(dn);
+    if(dnIndex < 0) // this block is not on the data-node list
+      return head;
+
+    BlockInfo next = this.getNext(dnIndex);
+    BlockInfo prev = this.getPrevious(dnIndex);
+    this.setNext(dnIndex, null);
+    this.setPrevious(dnIndex, null);
+    if(prev != null)
+      prev.setNext(prev.findDatanode(dn), next);
+    if(next != null)
+      next.setPrevious(next.findDatanode(dn), prev);
+    if(this == head)  // removing the head
+      head = next;
+    return head;
+  }
+
+  int listCount(DatanodeDescriptor dn) {
+    int count = 0;
+    for(BlockInfo cur = this; cur != null;
+          cur = cur.getNext(cur.findDatanode(dn)))
+      count++;
+    return count;
+  }
+
+  boolean listIsConsistent(DatanodeDescriptor dn) {
+    // going forward
+    int count = 0;
+    BlockInfo next, nextPrev;
+    BlockInfo cur = this;
+    while(cur != null) {
+      next = cur.getNext(cur.findDatanode(dn));
+      if(next != null) {
+        nextPrev = next.getPrevious(next.findDatanode(dn));
+        if(cur != nextPrev) {
+          System.out.println("Inconsistent list: cur->next->prev != cur");
+          return false;
+        }
+      }
+      cur = next;
+      count++;
+    }
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    // Super implementation is sufficient
+    return super.hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    // Sufficient to rely on super's implementation
+    return (this == obj) || super.equals(obj);
+  }
+}

+ 74 - 80
src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
 import org.apache.hadoop.hdfs.server.namenode.UnderReplicatedBlocks.BlockIterator;
 import org.apache.hadoop.hdfs.server.namenode.UnderReplicatedBlocks.BlockIterator;
 import org.apache.hadoop.security.AccessTokenHandler;
 import org.apache.hadoop.security.AccessTokenHandler;
@@ -234,7 +233,7 @@ public class BlockManager {
   /**
   /**
    * Get all valid locations of the block
    * Get all valid locations of the block
    */
    */
-  ArrayList<String> addBlock(Block block) {
+  ArrayList<String> getValidLocations(Block block) {
     ArrayList<String> machineSet =
     ArrayList<String> machineSet =
       new ArrayList<String>(blocksMap.numNodes(block));
       new ArrayList<String>(blocksMap.numNodes(block));
     for(Iterator<DatanodeDescriptor> it =
     for(Iterator<DatanodeDescriptor> it =
@@ -249,7 +248,6 @@ public class BlockManager {
     return machineSet;
     return machineSet;
   }
   }
 
 
-
   List<LocatedBlock> getBlockLocations(Block[] blocks, long offset,
   List<LocatedBlock> getBlockLocations(Block[] blocks, long offset,
       long length, int nrBlocksToReturn) throws IOException {
       long length, int nrBlocksToReturn) throws IOException {
     int curBlk = 0;
     int curBlk = 0;
@@ -397,43 +395,50 @@ public class BlockManager {
     }
     }
   }
   }
 
 
-  void markBlockAsCorrupt(Block blk, DatanodeInfo dn) throws IOException {
+  void findAndMarkBlockAsCorrupt(Block blk,
+                                 DatanodeInfo dn) throws IOException {
+    BlockInfo storedBlock = getStoredBlock(blk);
+    if (storedBlock == null) {
+      // Check if the replica is in the blockMap, if not
+      // ignore the request for now. This could happen when BlockScanner
+      // thread of Datanode reports bad block before Block reports are sent
+      // by the Datanode on startup
+      NameNode.stateChangeLog.info("BLOCK* NameSystem.markBlockAsCorrupt: " +
+                                   "block " + blk + " could not be marked as " +
+                                   "corrupt as it does not exist in blocksMap");
+      return;
+    }
+    markBlockAsCorrupt(storedBlock, dn);
+  }
+
+  private void markBlockAsCorrupt(BlockInfo storedBlock,
+                                  DatanodeInfo dn) throws IOException {
+    assert storedBlock != null : "storedBlock should not be null";
     DatanodeDescriptor node = namesystem.getDatanode(dn);
     DatanodeDescriptor node = namesystem.getDatanode(dn);
     if (node == null) {
     if (node == null) {
-      throw new IOException("Cannot mark block" + blk.getBlockName() +
+      throw new IOException("Cannot mark block " + 
+                            storedBlock.getBlockName() +
                             " as corrupt because datanode " + dn.getName() +
                             " as corrupt because datanode " + dn.getName() +
                             " does not exist. ");
                             " does not exist. ");
     }
     }
 
 
-    final BlockInfo storedBlockInfo = blocksMap.getStoredBlock(blk);
-    if (storedBlockInfo == null) {
-      // Check if the replica is in the blockMap, if not
-      // ignore the request for now. This could happen when BlockScanner
-      // thread of Datanode reports bad block before Block reports are sent
-      // by the Datanode on startup
+    INodeFile inode = storedBlock.getINode();
+    if (inode == null) {
       NameNode.stateChangeLog.info("BLOCK NameSystem.markBlockAsCorrupt: " +
       NameNode.stateChangeLog.info("BLOCK NameSystem.markBlockAsCorrupt: " +
-                                   "block " + blk + " could not be marked " +
-                                   "as corrupt as it does not exists in " +
-                                   "blocksMap");
+                                   "block " + storedBlock +
+                                   " could not be marked as corrupt as it" +
+                                   " does not belong to any file");
+      addToInvalidates(storedBlock, node);
+      return;
+    } 
+    // Add this replica to corruptReplicas Map
+    corruptReplicas.addToCorruptReplicasMap(storedBlock, node);
+    if (countNodes(storedBlock).liveReplicas() > inode.getReplication()) {
+      // the block is over-replicated so invalidate the replicas immediately
+      invalidateBlock(storedBlock, node);
     } else {
     } else {
-      INodeFile inode = storedBlockInfo.getINode();
-      if (inode == null) {
-        NameNode.stateChangeLog.info("BLOCK NameSystem.markBlockAsCorrupt: " +
-                                     "block " + blk + " could not be marked " +
-                                     "as corrupt as it does not belong to " +
-                                     "any file");
-        addToInvalidates(storedBlockInfo, node);
-        return;
-      } 
-      // Add this replica to corruptReplicas Map
-      corruptReplicas.addToCorruptReplicasMap(storedBlockInfo, node);
-      if (countNodes(storedBlockInfo).liveReplicas() > inode.getReplication()) {
-        // the block is over-replicated so invalidate the replicas immediately
-        invalidateBlock(storedBlockInfo, node);
-      } else {
-        // add the block to neededReplication
-        updateNeededReplications(storedBlockInfo, -1, 0);
-      }
+      // add the block to neededReplication
+      updateNeededReplications(storedBlock, -1, 0);
     }
     }
   }
   }
 
 
@@ -844,8 +849,9 @@ public class BlockManager {
    * needed replications if this takes care of the problem.
    * needed replications if this takes care of the problem.
    * @return the block that is stored in blockMap.
    * @return the block that is stored in blockMap.
    */
    */
-  private Block addStoredBlock(Block block, DatanodeDescriptor node,
-      DatanodeDescriptor delNodeHint) {
+  private Block addStoredBlock(final Block block,
+                               DatanodeDescriptor node,
+                               DatanodeDescriptor delNodeHint) {
     BlockInfo storedBlock = blocksMap.getStoredBlock(block);
     BlockInfo storedBlock = blocksMap.getStoredBlock(block);
     if (storedBlock == null || storedBlock.getINode() == null) {
     if (storedBlock == null || storedBlock.getINode() == null) {
       // If this block does not belong to anyfile, then we are done.
       // If this block does not belong to anyfile, then we are done.
@@ -858,30 +864,32 @@ public class BlockManager {
       // it will happen in next block report otherwise.
       // it will happen in next block report otherwise.
       return block;
       return block;
     }
     }
+    assert storedBlock != null : "Block must be stored by now";
+    INodeFile fileINode = storedBlock.getINode();
+    assert fileINode != null : "Block must belong to a file";
 
 
     // add block to the data-node
     // add block to the data-node
     boolean added = node.addBlock(storedBlock);
     boolean added = node.addBlock(storedBlock);
 
 
-    assert storedBlock != null : "Block must be stored by now";
-
     if (block != storedBlock) {
     if (block != storedBlock) {
-      if (block.getNumBytes() >= 0) {
-        long cursize = storedBlock.getNumBytes();
+      long cursize = storedBlock.getNumBytes();
+      long newsize = block.getNumBytes();
+      if (newsize >= 0) {
         if (cursize == 0) {
         if (cursize == 0) {
-          storedBlock.setNumBytes(block.getNumBytes());
-        } else if (cursize != block.getNumBytes()) {
+          storedBlock.setNumBytes(newsize);
+        } else if (cursize != newsize) {
           FSNamesystem.LOG.warn("Inconsistent size for block " + block +
           FSNamesystem.LOG.warn("Inconsistent size for block " + block +
                    " reported from " + node.getName() +
                    " reported from " + node.getName() +
                    " current size is " + cursize +
                    " current size is " + cursize +
-                   " reported size is " + block.getNumBytes());
+                   " reported size is " + newsize);
           try {
           try {
-            if (cursize > block.getNumBytes()) {
+            if (cursize > newsize) {
               // new replica is smaller in size than existing block.
               // new replica is smaller in size than existing block.
               // Mark the new replica as corrupt.
               // Mark the new replica as corrupt.
               FSNamesystem.LOG.warn("Mark new replica "
               FSNamesystem.LOG.warn("Mark new replica "
                   + block + " from " + node.getName() + " as corrupt "
                   + block + " from " + node.getName() + " as corrupt "
                   + "because length is shorter than existing ones");
                   + "because length is shorter than existing ones");
-              markBlockAsCorrupt(block, node);
+              markBlockAsCorrupt(storedBlock, node);
             } else {
             } else {
               // new replica is larger in size than existing block.
               // new replica is larger in size than existing block.
               // Mark pre-existing replicas as corrupt.
               // Mark pre-existing replicas as corrupt.
@@ -899,19 +907,12 @@ public class BlockManager {
                 FSNamesystem.LOG.warn("Mark existing replica "
                 FSNamesystem.LOG.warn("Mark existing replica "
                         + block + " from " + node.getName() + " as corrupt "
                         + block + " from " + node.getName() + " as corrupt "
                         + "because its length is shorter than the new one");
                         + "because its length is shorter than the new one");
-                markBlockAsCorrupt(block, nodes[j]);
+                markBlockAsCorrupt(storedBlock, nodes[j]);
               }
               }
               //
               //
               // change the size of block in blocksMap
               // change the size of block in blocksMap
               //
               //
-              storedBlock = blocksMap.getStoredBlock(block); // extra look up!
-              if (storedBlock == null) {
-                FSNamesystem.LOG.warn("Block " + block + " reported from "
-                    + node.getName()
-                    + " does not exist in blockMap. Surprise! Surprise!");
-              } else {
-                storedBlock.setNumBytes(block.getNumBytes());
-              }
+              storedBlock.setNumBytes(newsize);
             }
             }
           } catch (IOException e) {
           } catch (IOException e) {
             FSNamesystem.LOG.warn("Error in deleting bad block " + block + e);
             FSNamesystem.LOG.warn("Error in deleting bad block " + block + e);
@@ -919,17 +920,15 @@ public class BlockManager {
         }
         }
 
 
         // Updated space consumed if required.
         // Updated space consumed if required.
-        INodeFile file = (storedBlock != null) ? storedBlock.getINode() : null;
-        long diff = (file == null) ? 0 :
-                    (file.getPreferredBlockSize() - storedBlock.getNumBytes());
+        long diff = fileINode.getPreferredBlockSize() - storedBlock.getNumBytes();
         
         
-        if (diff > 0 && file.isUnderConstruction() &&
+        if (diff > 0 && fileINode.isUnderConstruction() &&
             cursize < storedBlock.getNumBytes()) {
             cursize < storedBlock.getNumBytes()) {
           try {
           try {
             String path = /* For finding parents */
             String path = /* For finding parents */
-            namesystem.leaseManager.findPath((INodeFileUnderConstruction) file);
+            namesystem.leaseManager.findPath((INodeFileUnderConstruction)fileINode);
             namesystem.dir.updateSpaceConsumed(path, 0, -diff
             namesystem.dir.updateSpaceConsumed(path, 0, -diff
-                * file.getReplication());
+                * fileINode.getReplication());
           } catch (IOException e) {
           } catch (IOException e) {
             FSNamesystem.LOG
             FSNamesystem.LOG
                 .warn("Unexpected exception while updating disk space : "
                 .warn("Unexpected exception while updating disk space : "
@@ -937,12 +936,9 @@ public class BlockManager {
           }
           }
         }
         }
       }
       }
-      block = storedBlock;
     }
     }
-    assert storedBlock == block : "Block must be stored by now";
 
 
     int curReplicaDelta = 0;
     int curReplicaDelta = 0;
-
     if (added) {
     if (added) {
       curReplicaDelta = 1;
       curReplicaDelta = 1;
       //
       //
@@ -952,20 +948,20 @@ public class BlockManager {
       //
       //
       if (!namesystem.isInSafeMode()) {
       if (!namesystem.isInSafeMode()) {
         NameNode.stateChangeLog.info("BLOCK* NameSystem.addStoredBlock: "
         NameNode.stateChangeLog.info("BLOCK* NameSystem.addStoredBlock: "
-            + "blockMap updated: " + node.getName() + " is added to " + block
-            + " size " + block.getNumBytes());
+            + "blockMap updated: " + node.getName() + " is added to " + 
+            storedBlock + " size " + storedBlock.getNumBytes());
       }
       }
     } else {
     } else {
       NameNode.stateChangeLog.warn("BLOCK* NameSystem.addStoredBlock: "
       NameNode.stateChangeLog.warn("BLOCK* NameSystem.addStoredBlock: "
-          + "Redundant addStoredBlock request received for " + block + " on "
-          + node.getName() + " size " + block.getNumBytes());
+          + "Redundant addStoredBlock request received for " + storedBlock
+          + " on " + node.getName() + " size " + storedBlock.getNumBytes());
     }
     }
 
 
     // filter out containingNodes that are marked for decommission.
     // filter out containingNodes that are marked for decommission.
     NumberReplicas num = countNodes(storedBlock);
     NumberReplicas num = countNodes(storedBlock);
     int numLiveReplicas = num.liveReplicas();
     int numLiveReplicas = num.liveReplicas();
     int numCurrentReplica = numLiveReplicas
     int numCurrentReplica = numLiveReplicas
-      + pendingReplications.getNumReplicas(block);
+      + pendingReplications.getNumReplicas(storedBlock);
 
 
     // check whether safe replication is reached for the block
     // check whether safe replication is reached for the block
     namesystem.incrementSafeBlockCount(numCurrentReplica);
     namesystem.incrementSafeBlockCount(numCurrentReplica);
@@ -974,39 +970,37 @@ public class BlockManager {
     // if file is being actively written to, then do not check
     // if file is being actively written to, then do not check
     // replication-factor here. It will be checked when the file is closed.
     // replication-factor here. It will be checked when the file is closed.
     //
     //
-    INodeFile fileINode = null;
-    fileINode = storedBlock.getINode();
     if (fileINode.isUnderConstruction()) {
     if (fileINode.isUnderConstruction()) {
-      return block;
+      return storedBlock;
     }
     }
 
 
     // do not handle mis-replicated blocks during startup
     // do not handle mis-replicated blocks during startup
     if (namesystem.isInSafeMode())
     if (namesystem.isInSafeMode())
-      return block;
+      return storedBlock;
 
 
     // handle underReplication/overReplication
     // handle underReplication/overReplication
     short fileReplication = fileINode.getReplication();
     short fileReplication = fileINode.getReplication();
     if (numCurrentReplica >= fileReplication) {
     if (numCurrentReplica >= fileReplication) {
-      neededReplications.remove(block, numCurrentReplica,
+      neededReplications.remove(storedBlock, numCurrentReplica,
           num.decommissionedReplicas, fileReplication);
           num.decommissionedReplicas, fileReplication);
     } else {
     } else {
-      updateNeededReplications(block, curReplicaDelta, 0);
+      updateNeededReplications(storedBlock, curReplicaDelta, 0);
     }
     }
     if (numCurrentReplica > fileReplication) {
     if (numCurrentReplica > fileReplication) {
-      processOverReplicatedBlock(block, fileReplication, node, delNodeHint);
+      processOverReplicatedBlock(storedBlock, fileReplication, node, delNodeHint);
     }
     }
     // If the file replication has reached desired value
     // If the file replication has reached desired value
     // we can remove any corrupt replicas the block may have
     // we can remove any corrupt replicas the block may have
-    int corruptReplicasCount = corruptReplicas.numCorruptReplicas(block);
+    int corruptReplicasCount = corruptReplicas.numCorruptReplicas(storedBlock);
     int numCorruptNodes = num.corruptReplicas();
     int numCorruptNodes = num.corruptReplicas();
     if (numCorruptNodes != corruptReplicasCount) {
     if (numCorruptNodes != corruptReplicasCount) {
       FSNamesystem.LOG.warn("Inconsistent number of corrupt replicas for " +
       FSNamesystem.LOG.warn("Inconsistent number of corrupt replicas for " +
-          block + "blockMap has " + numCorruptNodes + 
+          storedBlock + "blockMap has " + numCorruptNodes + 
           " but corrupt replicas map has " + corruptReplicasCount);
           " but corrupt replicas map has " + corruptReplicasCount);
     }
     }
     if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication))
     if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication))
-      invalidateCorruptReplicas(block);
-    return block;
+      invalidateCorruptReplicas(storedBlock);
+    return storedBlock;
   }
   }
 
 
   /**
   /**
@@ -1051,7 +1045,7 @@ public class BlockManager {
     long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0;
     long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0;
     synchronized (namesystem) {
     synchronized (namesystem) {
       neededReplications.clear();
       neededReplications.clear();
-      for (BlocksMap.BlockInfo block : blocksMap.getBlocks()) {
+      for (BlockInfo block : blocksMap.getBlocks()) {
         INodeFile fileINode = block.getINode();
         INodeFile fileINode = block.getINode();
         if (fileINode == null) {
         if (fileINode == null) {
           // block does not belong to any file
           // block does not belong to any file
@@ -1415,13 +1409,13 @@ public class BlockManager {
     blocksMap.removeBlock(block);
     blocksMap.removeBlock(block);
   }
   }
   
   
-  public int getCapacity() {
+  int getCapacity() {
     synchronized(namesystem) {
     synchronized(namesystem) {
       return blocksMap.getCapacity();
       return blocksMap.getCapacity();
     }
     }
   }
   }
   
   
-  public float getLoadFactor() {
+  float getLoadFactor() {
     return blocksMap.getLoadFactor();
     return blocksMap.getLoadFactor();
   }
   }
 }
 }

+ 12 - 281
src/java/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java

@@ -30,259 +30,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
  * the datanodes that store the block.
  * the datanodes that store the block.
  */
  */
 class BlocksMap {
 class BlocksMap {
-        
-  /**
-   * Internal class for block metadata.
-   */
-  static class BlockInfo extends Block {
-    private INodeFile inode;
-
-    /**
-     * This array contains triplets of references.
-     * For each i-th datanode the block belongs to
-     * triplets[3*i] is the reference to the DatanodeDescriptor
-     * and triplets[3*i+1] and triplets[3*i+2] are references 
-     * to the previous and the next blocks, respectively, in the 
-     * list of blocks belonging to this data-node.
-     */
-    private Object[] triplets;
-
-    BlockInfo(Block blk, int replication) {
-      super(blk);
-      this.triplets = new Object[3*replication];
-      this.inode = null;
-    }
-
-    INodeFile getINode() {
-      return inode;
-    }
-
-    DatanodeDescriptor getDatanode(int index) {
-      assert this.triplets != null : "BlockInfo is not initialized";
-      assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
-      DatanodeDescriptor node = (DatanodeDescriptor)triplets[index*3];
-      assert node == null || 
-          DatanodeDescriptor.class.getName().equals(node.getClass().getName()) : 
-                "DatanodeDescriptor is expected at " + index*3;
-      return node;
-    }
-
-    BlockInfo getPrevious(int index) {
-      assert this.triplets != null : "BlockInfo is not initialized";
-      assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
-      BlockInfo info = (BlockInfo)triplets[index*3+1];
-      assert info == null || 
-          BlockInfo.class.getName().equals(info.getClass().getName()) : 
-                "BlockInfo is expected at " + index*3;
-      return info;
-    }
-
-    BlockInfo getNext(int index) {
-      assert this.triplets != null : "BlockInfo is not initialized";
-      assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
-      BlockInfo info = (BlockInfo)triplets[index*3+2];
-      assert info == null || 
-          BlockInfo.class.getName().equals(info.getClass().getName()) : 
-                "BlockInfo is expected at " + index*3;
-      return info;
-    }
-
-    void setDatanode(int index, DatanodeDescriptor node) {
-      assert this.triplets != null : "BlockInfo is not initialized";
-      assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
-      triplets[index*3] = node;
-    }
-
-    void setPrevious(int index, BlockInfo to) {
-      assert this.triplets != null : "BlockInfo is not initialized";
-      assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
-      triplets[index*3+1] = to;
-    }
-
-    void setNext(int index, BlockInfo to) {
-      assert this.triplets != null : "BlockInfo is not initialized";
-      assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
-      triplets[index*3+2] = to;
-    }
-
-    private int getCapacity() {
-      assert this.triplets != null : "BlockInfo is not initialized";
-      assert triplets.length % 3 == 0 : "Malformed BlockInfo";
-      return triplets.length / 3;
-    }
-
-    /**
-     * Ensure that there is enough  space to include num more triplets.
-     *      * @return first free triplet index.
-     */
-    private int ensureCapacity(int num) {
-      assert this.triplets != null : "BlockInfo is not initialized";
-      int last = numNodes();
-      if(triplets.length >= (last+num)*3)
-        return last;
-      /* Not enough space left. Create a new array. Should normally 
-       * happen only when replication is manually increased by the user. */
-      Object[] old = triplets;
-      triplets = new Object[(last+num)*3];
-      for(int i=0; i < last*3; i++) {
-        triplets[i] = old[i];
-      }
-      return last;
-    }
-
-    /**
-     * Count the number of data-nodes the block belongs to.
-     */
-    int numNodes() {
-      assert this.triplets != null : "BlockInfo is not initialized";
-      assert triplets.length % 3 == 0 : "Malformed BlockInfo";
-      for(int idx = getCapacity()-1; idx >= 0; idx--) {
-        if(getDatanode(idx) != null)
-          return idx+1;
-      }
-      return 0;
-    }
-
-    /**
-     * Add data-node this block belongs to.
-     */
-    boolean addNode(DatanodeDescriptor node) {
-      if(findDatanode(node) >= 0) // the node is already there
-        return false;
-      // find the last null node
-      int lastNode = ensureCapacity(1);
-      setDatanode(lastNode, node);
-      setNext(lastNode, null);
-      setPrevious(lastNode, null);
-      return true;
-    }
-
-    /**
-     * Remove data-node from the block.
-     */
-    boolean removeNode(DatanodeDescriptor node) {
-      int dnIndex = findDatanode(node);
-      if(dnIndex < 0) // the node is not found
-        return false;
-      assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
-        "Block is still in the list and must be removed first.";
-      // find the last not null node
-      int lastNode = numNodes()-1; 
-      // replace current node triplet by the lastNode one 
-      setDatanode(dnIndex, getDatanode(lastNode));
-      setNext(dnIndex, getNext(lastNode)); 
-      setPrevious(dnIndex, getPrevious(lastNode)); 
-      // set the last triplet to null
-      setDatanode(lastNode, null);
-      setNext(lastNode, null); 
-      setPrevious(lastNode, null); 
-      return true;
-    }
-
-    /**
-     * Find specified DatanodeDescriptor.
-     * @param dn
-     * @return index or -1 if not found.
-     */
-    int findDatanode(DatanodeDescriptor dn) {
-      int len = getCapacity();
-      for(int idx = 0; idx < len; idx++) {
-        DatanodeDescriptor cur = getDatanode(idx);
-        if(cur == dn)
-          return idx;
-        if(cur == null)
-          break;
-      }
-      return -1;
-    }
-
-    /**
-     * Insert this block into the head of the list of blocks 
-     * related to the specified DatanodeDescriptor.
-     * If the head is null then form a new list.
-     * @return current block as the new head of the list.
-     */
-    BlockInfo listInsert(BlockInfo head, DatanodeDescriptor dn) {
-      int dnIndex = this.findDatanode(dn);
-      assert dnIndex >= 0 : "Data node is not found: current";
-      assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
-              "Block is already in the list and cannot be inserted.";
-      this.setPrevious(dnIndex, null);
-      this.setNext(dnIndex, head);
-      if(head != null)
-        head.setPrevious(head.findDatanode(dn), this);
-      return this;
-    }
-
-    /**
-     * Remove this block from the list of blocks 
-     * related to the specified DatanodeDescriptor.
-     * If this block is the head of the list then return the next block as 
-     * the new head.
-     * @return the new head of the list or null if the list becomes
-     * empty after deletion.
-     */
-    BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
-      if(head == null)
-        return null;
-      int dnIndex = this.findDatanode(dn);
-      if(dnIndex < 0) // this block is not on the data-node list
-        return head;
-
-      BlockInfo next = this.getNext(dnIndex);
-      BlockInfo prev = this.getPrevious(dnIndex);
-      this.setNext(dnIndex, null);
-      this.setPrevious(dnIndex, null);
-      if(prev != null)
-        prev.setNext(prev.findDatanode(dn), next);
-      if(next != null)
-        next.setPrevious(next.findDatanode(dn), prev);
-      if(this == head)  // removing the head
-        head = next;
-      return head;
-    }
-
-    int listCount(DatanodeDescriptor dn) {
-      int count = 0;
-      for(BlockInfo cur = this; cur != null;
-            cur = cur.getNext(cur.findDatanode(dn)))
-        count++;
-      return count;
-    }
-
-    boolean listIsConsistent(DatanodeDescriptor dn) {
-      // going forward
-      int count = 0;
-      BlockInfo next, nextPrev;
-      BlockInfo cur = this;
-      while(cur != null) {
-        next = cur.getNext(cur.findDatanode(dn));
-        if(next != null) {
-          nextPrev = next.getPrevious(next.findDatanode(dn));
-          if(cur != nextPrev) {
-            System.out.println("Inconsistent list: cur->next->prev != cur");
-            return false;
-          }
-        }
-        cur = next;
-        count++;
-      }
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      // Super implementation is sufficient
-      return super.hashCode();
-    }
-    
-    @Override
-    public boolean equals(Object obj) {
-      // Sufficient to rely on super's implementation
-      return (this == obj) || super.equals(obj);
-    }
-  }
-
   private static class NodeIterator implements Iterator<DatanodeDescriptor> {
   private static class NodeIterator implements Iterator<DatanodeDescriptor> {
     private BlockInfo blockInfo;
     private BlockInfo blockInfo;
     private int nextIdx = 0;
     private int nextIdx = 0;
@@ -320,29 +67,22 @@ class BlocksMap {
     this.map = new HashMap<BlockInfo, BlockInfo>(initialCapacity, loadFactor);
     this.map = new HashMap<BlockInfo, BlockInfo>(initialCapacity, loadFactor);
   }
   }
 
 
-  /**
-   * Add BlockInfo if mapping does not exist.
-   */
-  private BlockInfo checkBlockInfo(Block b, int replication) {
-    BlockInfo info = map.get(b);
-    if (info == null) {
-      info = new BlockInfo(b, replication);
-      map.put(info, info);
-    }
-    return info;
-  }
-
   INodeFile getINode(Block b) {
   INodeFile getINode(Block b) {
     BlockInfo info = map.get(b);
     BlockInfo info = map.get(b);
-    return (info != null) ? info.inode : null;
+    return (info != null) ? info.getINode() : null;
   }
   }
 
 
   /**
   /**
    * Add block b belonging to the specified file inode to the map.
    * Add block b belonging to the specified file inode to the map.
    */
    */
   BlockInfo addINode(Block b, INodeFile iNode) {
   BlockInfo addINode(Block b, INodeFile iNode) {
-    BlockInfo info = checkBlockInfo(b, iNode.getReplication());
-    info.inode = iNode;
+    int replication = iNode.getReplication();
+    BlockInfo info = map.get(b);
+    if (info == null) {
+      info = new BlockInfo(b, replication);
+      map.put(info, info);
+    }
+    info.setINode(iNode);
     return info;
     return info;
   }
   }
 
 
@@ -356,7 +96,7 @@ class BlocksMap {
     if (blockInfo == null)
     if (blockInfo == null)
       return;
       return;
 
 
-    blockInfo.inode = null;
+    blockInfo.setINode(null);
     for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
     for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
       DatanodeDescriptor dn = blockInfo.getDatanode(idx);
       DatanodeDescriptor dn = blockInfo.getDatanode(idx);
       dn.removeBlock(blockInfo); // remove from the list and wipe the location
       dn.removeBlock(blockInfo); // remove from the list and wipe the location
@@ -379,15 +119,6 @@ class BlocksMap {
     return info == null ? 0 : info.numNodes();
     return info == null ? 0 : info.numNodes();
   }
   }
 
 
-  /** returns true if the node does not already exists and is added.
-   * false if the node already exists.*/
-  boolean addNode(Block b, DatanodeDescriptor node, int replication) {
-    // insert into the map if not there yet
-    BlockInfo info = checkBlockInfo(b, replication);
-    // add block to the data-node list and the node to the block info
-    return node.addBlock(info);
-  }
-
   /**
   /**
    * Remove data-node reference from the block.
    * Remove data-node reference from the block.
    * Remove the block from the block map
    * Remove the block from the block map
@@ -402,7 +133,7 @@ class BlocksMap {
     boolean removed = node.removeBlock(info);
     boolean removed = node.removeBlock(info);
 
 
     if (info.getDatanode(0) == null     // no datanodes left
     if (info.getDatanode(0) == null     // no datanodes left
-              && info.inode == null) {  // does not belong to a file
+              && info.getINode() == null) {  // does not belong to a file
       map.remove(b);  // remove block from the map
       map.remove(b);  // remove block from the map
     }
     }
     return removed;
     return removed;
@@ -437,7 +168,7 @@ class BlocksMap {
   }
   }
   
   
   /** Get the capacity of the HashMap that stores blocks */
   /** Get the capacity of the HashMap that stores blocks */
-  public int getCapacity() {
+  int getCapacity() {
     // Capacity doubles every time the map size reaches the threshold
     // Capacity doubles every time the map size reaches the threshold
     while (map.size() > (int)(capacity * loadFactor)) {
     while (map.size() > (int)(capacity * loadFactor)) {
       capacity <<= 1;
       capacity <<= 1;
@@ -446,7 +177,7 @@ class BlocksMap {
   }
   }
   
   
   /** Get the load factor of the map */
   /** Get the load factor of the map */
-  public float getLoadFactor() {
+  float getLoadFactor() {
     return loadFactor;
     return loadFactor;
   }
   }
 }
 }

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

@@ -25,7 +25,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;

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

@@ -33,7 +33,6 @@ import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 
 
 /*************************************************
 /*************************************************
  * FSDirectory stores the filesystem directory state.
  * FSDirectory stores the filesystem directory state.
@@ -277,8 +276,7 @@ class FSDirectory implements Closeable {
                   fileNode.getPreferredBlockSize()*fileNode.getReplication());
                   fileNode.getPreferredBlockSize()*fileNode.getReplication());
       
       
       // associate the new list of blocks with this file
       // associate the new list of blocks with this file
-      getBlockManager().addINode(block, fileNode);
-      BlockInfo blockInfo = getBlockManager().getStoredBlock(block);
+      BlockInfo blockInfo = getBlockManager().addINode(block, fileNode);
       fileNode.addBlock(blockInfo);
       fileNode.addBlock(blockInfo);
 
 
       NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
       NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
@@ -308,8 +306,10 @@ class FSDirectory implements Closeable {
    */
    */
   void closeFile(String path, INodeFile file) {
   void closeFile(String path, INodeFile file) {
     waitForReady();
     waitForReady();
+    long now = FSNamesystem.now();
     synchronized (rootDir) {
     synchronized (rootDir) {
       // file is closed
       // file is closed
+      file.setModificationTimeForce(now);
       fsImage.getEditLog().logCloseFile(path, file);
       fsImage.getEditLog().logCloseFile(path, file);
       if (NameNode.stateChangeLog.isDebugEnabled()) {
       if (NameNode.stateChangeLog.isDebugEnabled()) {
         NameNode.stateChangeLog.debug("DIR* FSDirectory.closeFile: "
         NameNode.stateChangeLog.debug("DIR* FSDirectory.closeFile: "

+ 90 - 83
src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -320,7 +320,7 @@ public class FSEditLog {
    * @param es - stream to remove
    * @param es - stream to remove
    * @return the matching stream
    * @return the matching stream
    */
    */
-  public StorageDirectory getStorage(EditLogOutputStream es) {
+  StorageDirectory getStorage(EditLogOutputStream es) {
     String parentStorageDir = ((EditLogFileOutputStream)es).getFile()
     String parentStorageDir = ((EditLogFileOutputStream)es).getFile()
     .getParentFile().getParentFile().getAbsolutePath();
     .getParentFile().getParentFile().getAbsolutePath();
 
 
@@ -339,7 +339,7 @@ public class FSEditLog {
    * @param sd
    * @param sd
    * @return the matching stream
    * @return the matching stream
    */
    */
-  public EditLogOutputStream getEditsStream(StorageDirectory sd) {
+  synchronized EditLogOutputStream getEditsStream(StorageDirectory sd) {
 	for (EditLogOutputStream es : editStreams) {
 	for (EditLogOutputStream es : editStreams) {
 	  File parentStorageDir = ((EditLogFileOutputStream)es).getFile()
 	  File parentStorageDir = ((EditLogFileOutputStream)es).getFile()
 	  .getParentFile().getParentFile();
 	  .getParentFile().getParentFile();
@@ -780,68 +780,76 @@ public class FSEditLog {
 
 
     // Fetch the transactionId of this thread. 
     // Fetch the transactionId of this thread. 
     long mytxid = myTransactionId.get().txid;
     long mytxid = myTransactionId.get().txid;
-
-    synchronized (this) {
-      assert editStreams.size() > 0 : "no editlog streams";
-      printStatistics(false);
-
-      // if somebody is already syncing, then wait
-      while (mytxid > synctxid && isSyncRunning) {
-        try {
-          wait(1000);
-        } catch (InterruptedException ie) { 
+    EditLogOutputStream streams[] = null;
+    boolean sync = false;
+    try {
+      synchronized (this) {
+        assert editStreams.size() > 0 : "no editlog streams";
+        printStatistics(false);
+  
+        // if somebody is already syncing, then wait
+        while (mytxid > synctxid && isSyncRunning) {
+          try {
+            wait(1000);
+          } catch (InterruptedException ie) { 
+          }
         }
         }
-      }
-
-      //
-      // If this transaction was already flushed, then nothing to do
-      //
-      if (mytxid <= synctxid) {
-        numTransactionsBatchedInSync++;
-        if (metrics != null) // Metrics is non-null only when used inside name node
-          metrics.transactionsBatchedInSync.inc();
-        return;
-      }
-   
-      // now, this thread will do the sync
-      syncStart = txid;
-      isSyncRunning = true;   
-
-      // swap buffers
-      for(EditLogOutputStream eStream : editStreams) {
-        eStream.setReadyToFlush();
-      }
-    }
-
-    // do the sync
-    long start = FSNamesystem.now();
-    for (int idx = 0; idx < editStreams.size(); idx++) {
-      EditLogOutputStream eStream = editStreams.get(idx);
-      try {
-        eStream.flush();
-      } catch (IOException ie) {
+  
         //
         //
-        // remember the streams that encountered an error.
+        // If this transaction was already flushed, then nothing to do
         //
         //
-        if (errorStreams == null) {
-          errorStreams = new ArrayList<EditLogOutputStream>(1);
+        if (mytxid <= synctxid) {
+          numTransactionsBatchedInSync++;
+          if (metrics != null) // Metrics is non-null only when used inside name node
+            metrics.transactionsBatchedInSync.inc();
+          return;
         }
         }
-        errorStreams.add(eStream);
-        FSNamesystem.LOG.error("Unable to sync edit log. " +
-                               "Fatal Error.");
+     
+        // now, this thread will do the sync
+        syncStart = txid;
+        isSyncRunning = true;
+        sync = true;
+  
+        // swap buffers
+        for(EditLogOutputStream eStream : editStreams) {
+          eStream.setReadyToFlush();
+        }
+        streams = 
+          editStreams.toArray(new EditLogOutputStream[editStreams.size()]);
       }
       }
+  
+      // do the sync
+      long start = FSNamesystem.now();
+      for (int idx = 0; idx < streams.length; idx++) {
+        EditLogOutputStream eStream = streams[idx];
+        try {
+          eStream.flush();
+        } catch (IOException ie) {
+          //
+          // remember the streams that encountered an error.
+          //
+          if (errorStreams == null) {
+            errorStreams = new ArrayList<EditLogOutputStream>(1);
+          }
+          errorStreams.add(eStream);
+          FSNamesystem.LOG.error("Unable to sync edit log. " +
+                                 "Fatal Error.");
+        }
+      }
+      long elapsed = FSNamesystem.now() - start;
+      processIOError(errorStreams, true);
+  
+      if (metrics != null) // Metrics non-null only when used inside name node
+        metrics.syncs.inc(elapsed);
+    } finally {
+      synchronized (this) {
+        synctxid = syncStart;
+        if (sync) {
+          isSyncRunning = false;
+        }
+        this.notifyAll();
+     }
     }
     }
-    long elapsed = FSNamesystem.now() - start;
-
-    synchronized (this) {
-       processIOError(errorStreams, true);
-       synctxid = syncStart;
-       isSyncRunning = false;
-       this.notifyAll();
-    }
-
-    if (metrics != null) // Metrics is non-null only when used inside name node
-      metrics.syncs.inc(elapsed);
   }
   }
 
 
   //
   //
@@ -1030,14 +1038,6 @@ public class FSEditLog {
     return size;
     return size;
   }
   }
   
   
-  public String listEditsStreams() {
-    StringBuffer buf = new StringBuffer();
-    for (EditLogOutputStream os : editStreams) {
-      buf.append(os.getName()  + ";");
-    }
-    return buf.toString();
-  }
-
   /**
   /**
    * Closes the current edit log and opens edits.new. 
    * Closes the current edit log and opens edits.new. 
    */
    */
@@ -1272,7 +1272,7 @@ public class FSEditLog {
    * @param nnReg this (active) name-node registration.
    * @param nnReg this (active) name-node registration.
    * @throws IOException
    * @throws IOException
    */
    */
-  void logJSpoolStart(NamenodeRegistration bnReg, // backup node
+  synchronized void logJSpoolStart(NamenodeRegistration bnReg, // backup node
                       NamenodeRegistration nnReg) // active name-node
                       NamenodeRegistration nnReg) // active name-node
   throws IOException {
   throws IOException {
     if(bnReg.isRole(NamenodeRole.CHECKPOINT))
     if(bnReg.isRole(NamenodeRole.CHECKPOINT))
@@ -1331,22 +1331,27 @@ public class FSEditLog {
     }
     }
 
 
     public boolean hasNext() {
     public boolean hasNext() {
-      if(editStreams == null || 
-         editStreams.isEmpty() || nextIndex >= editStreams.size())
-        return false;
-      while(nextIndex < editStreams.size()
-            && !editStreams.get(nextIndex).getType().isOfType(type))
-        nextIndex++;
-      return nextIndex < editStreams.size();
+      synchronized(FSEditLog.this) {
+        if(editStreams == null || 
+           editStreams.isEmpty() || nextIndex >= editStreams.size())
+          return false;
+        while(nextIndex < editStreams.size()
+              && !editStreams.get(nextIndex).getType().isOfType(type))
+          nextIndex++;
+        return nextIndex < editStreams.size();
+      }
     }
     }
 
 
     public EditLogOutputStream next() {
     public EditLogOutputStream next() {
-      EditLogOutputStream stream = editStreams.get(nextIndex);
-      prevIndex = nextIndex;
-      nextIndex++;
-      while(nextIndex < editStreams.size()
-          && !editStreams.get(nextIndex).getType().isOfType(type))
-      nextIndex++;
+      EditLogOutputStream stream = null;
+      synchronized(FSEditLog.this) {
+        stream = editStreams.get(nextIndex);
+        prevIndex = nextIndex;
+        nextIndex++;
+        while(nextIndex < editStreams.size()
+            && !editStreams.get(nextIndex).getType().isOfType(type))
+        nextIndex++;
+      }
       return stream;
       return stream;
     }
     }
 
 
@@ -1357,9 +1362,11 @@ public class FSEditLog {
     }
     }
 
 
     void replace(EditLogOutputStream newStream) {
     void replace(EditLogOutputStream newStream) {
-      assert 0 <= prevIndex && prevIndex < editStreams.size() :
-                                                        "Index out of bound.";
-      editStreams.set(prevIndex, newStream);
+      synchronized (FSEditLog.this) {
+        assert 0 <= prevIndex && prevIndex < editStreams.size() :
+                                                          "Index out of bound.";
+        editStreams.set(prevIndex, newStream);
+      }
     }
     }
   }
   }
 
 

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

@@ -58,7 +58,6 @@ import org.apache.hadoop.hdfs.server.common.UpgradeManager;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.JournalStream.JournalType;
 import org.apache.hadoop.hdfs.server.namenode.JournalStream.JournalType;
 import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;

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

@@ -26,7 +26,6 @@ import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMetrics;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMetrics;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
@@ -582,7 +581,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
    * return the length of the added block; 0 if the block is not added
    * return the length of the added block; 0 if the block is not added
    */
    */
   private long addBlock(Block block, List<BlockWithLocations> results) {
   private long addBlock(Block block, List<BlockWithLocations> results) {
-    ArrayList<String> machineSet = blockManager.addBlock(block);
+    ArrayList<String> machineSet = blockManager.getValidLocations(block);
     if(machineSet.size() == 0) {
     if(machineSet.size() == 0) {
       return 0;
       return 0;
     } else {
     } else {
@@ -1338,7 +1337,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
    */
    */
   public synchronized void markBlockAsCorrupt(Block blk, DatanodeInfo dn)
   public synchronized void markBlockAsCorrupt(Block blk, DatanodeInfo dn)
     throws IOException {
     throws IOException {
-    blockManager.markBlockAsCorrupt(blk, dn);
+    blockManager.findAndMarkBlockAsCorrupt(blk, dn);
   }
   }
 
 
 
 

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

@@ -24,7 +24,6 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 
 
 class INodeFile extends INode {
 class INodeFile extends INode {
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);

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

@@ -21,7 +21,6 @@ import java.io.IOException;
 
 
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 
 
 
 
 class INodeFileUnderConstruction extends INodeFile {
 class INodeFileUnderConstruction extends INodeFile {

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

@@ -818,6 +818,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   /**
   /**
    * Returns the size of the current edit log.
    * Returns the size of the current edit log.
    */
    */
+  @Deprecated
   public long getEditLogSize() throws IOException {
   public long getEditLogSize() throws IOException {
     return namesystem.getEditLogSize();
     return namesystem.getEditLogSize();
   }
   }
@@ -825,6 +826,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   /**
   /**
    * Roll the edit log.
    * Roll the edit log.
    */
    */
+  @Deprecated
   public CheckpointSignature rollEditLog() throws IOException {
   public CheckpointSignature rollEditLog() throws IOException {
     return namesystem.rollEditLog();
     return namesystem.rollEditLog();
   }
   }
@@ -832,6 +834,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   /**
   /**
    * Roll the image 
    * Roll the image 
    */
    */
+  @Deprecated
   public void rollFsImage() throws IOException {
   public void rollFsImage() throws IOException {
     namesystem.rollFSImage();
     namesystem.rollFSImage();
   }
   }
@@ -1150,9 +1153,11 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
       case FORMAT:
       case FORMAT:
         boolean aborted = format(conf, true);
         boolean aborted = format(conf, true);
         System.exit(aborted ? 1 : 0);
         System.exit(aborted ? 1 : 0);
+        return null; // avoid javac warning
       case FINALIZE:
       case FINALIZE:
         aborted = finalize(conf, true);
         aborted = finalize(conf, true);
         System.exit(aborted ? 1 : 0);
         System.exit(aborted ? 1 : 0);
+        return null; // avoid javac warning
       case BACKUP:
       case BACKUP:
       case CHECKPOINT:
       case CHECKPOINT:
         return new BackupNode(conf, startOpt.toNodeRole());
         return new BackupNode(conf, startOpt.toNodeRole());

+ 10 - 2
src/test/findbugsExcludeFile.xml

@@ -207,9 +207,17 @@
        <Bug pattern="OBL_UNSATISFIED_OBLIGATION" />
        <Bug pattern="OBL_UNSATISFIED_OBLIGATION" />
      </Match>
      </Match>
 
 
+     <!--
+       CreateBlockWriteStreams and getTmpInputStreams are pretty much like a stream constructor.
+       The newly created streams are not supposed to be closed in the constructor. So ignore
+       the OBL warning.
+     -->
      <Match>
      <Match>
-       <Class name="org.apache.hadoop.examples.ContextFactory" />
-       <Method name="setAttributes" />
+       <Class name="org.apache.hadoop.hdfs.server.datanode.FSDataset" />
+       <Or>
+         <Method name="createBlockWriteStreams" />
+         <Method name="getTmpInputStreams" />
+       </Or>
        <Bug pattern="OBL_UNSATISFIED_OBLIGATION" />
        <Bug pattern="OBL_UNSATISFIED_OBLIGATION" />
      </Match>
      </Match>
 
 

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestAbandonBlock.java

@@ -50,10 +50,10 @@ public class TestAbandonBlock extends junit.framework.TestCase {
   
   
       //try reading the block by someone
       //try reading the block by someone
       DFSClient dfsclient = new DFSClient(CONF);
       DFSClient dfsclient = new DFSClient(CONF);
-      LocatedBlocks blocks = dfsclient.namenode.getBlockLocations(src, 0, 1);
+      LocatedBlocks blocks = dfsclient.getNamenode().getBlockLocations(src, 0, 1);
       LocatedBlock b = blocks.get(0); 
       LocatedBlock b = blocks.get(0); 
       try {
       try {
-        dfsclient.namenode.abandonBlock(b.getBlock(), src, "someone");
+        dfsclient.getNamenode().abandonBlock(b.getBlock(), src, "someone");
         //previous line should throw an exception.
         //previous line should throw an exception.
         assertTrue(false);
         assertTrue(false);
       }
       }

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

@@ -21,10 +21,18 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStream;
 
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.server.common.*;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.security.AccessControlException;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
@@ -34,6 +42,8 @@ import junit.framework.TestCase;
  * properly in case of errors.
  * properly in case of errors.
  */
  */
 public class TestDFSClientRetries extends TestCase {
 public class TestDFSClientRetries extends TestCase {
+  public static final Log LOG =
+    LogFactory.getLog(TestDFSClientRetries.class.getName());
   
   
   // writes 'len' bytes of data to out.
   // writes 'len' bytes of data to out.
   private static void writeData(OutputStream out, int len) throws IOException {
   private static void writeData(OutputStream out, int len) throws IOException {
@@ -97,4 +107,132 @@ public class TestDFSClientRetries extends TestCase {
   }
   }
   
   
   // more tests related to different failure cases can be added here.
   // more tests related to different failure cases can be added here.
+  
+  class TestNameNode implements ClientProtocol
+  {
+    int num_calls = 0;
+    
+    // The total number of calls that can be made to addBlock
+    // before an exception is thrown
+    int num_calls_allowed; 
+    public final String ADD_BLOCK_EXCEPTION = "Testing exception thrown from"
+                                             + "TestDFSClientRetries::"
+                                             + "TestNameNode::addBlock";
+    public final String RETRY_CONFIG
+          = "dfs.client.block.write.locateFollowingBlock.retries";
+          
+    public TestNameNode(Configuration conf) throws IOException
+    {
+      // +1 because the configuration value is the number of retries and
+      // the first call is not a retry (e.g., 2 retries == 3 total
+      // calls allowed)
+      this.num_calls_allowed = conf.getInt(RETRY_CONFIG, 5) + 1;
+    }
+
+    public long getProtocolVersion(String protocol, 
+                                     long clientVersion)
+    throws IOException
+    {
+      return versionID;
+    }
+
+    public LocatedBlock addBlock(String src, String clientName)
+    throws IOException
+    {
+      num_calls++;
+      if (num_calls > num_calls_allowed) { 
+        throw new IOException("addBlock called more times than "
+                              + RETRY_CONFIG
+                              + " allows.");
+      } else {
+          throw new RemoteException(NotReplicatedYetException.class.getName(),
+                                    ADD_BLOCK_EXCEPTION);
+      }
+    }
+    
+    
+    // The following methods are stub methods that are not needed by this mock class
+    
+    public LocatedBlocks  getBlockLocations(String src, long offset, long length) throws IOException { return null; }
+    
+    public void create(String src, FsPermission masked, String clientName, EnumSetWritable<CreateFlag> flag, short replication, long blockSize) throws IOException {}
+    
+    public LocatedBlock append(String src, String clientName) throws IOException { return null; }
+
+    public boolean setReplication(String src, short replication) throws IOException { return false; }
+
+    public void setPermission(String src, FsPermission permission) throws IOException {}
+
+    public void setOwner(String src, String username, String groupname) throws IOException {}
+
+    public void abandonBlock(Block b, String src, String holder) throws IOException {}
+
+    public boolean complete(String src, String clientName) throws IOException { return false; }
+
+    public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {}
+
+    public boolean rename(String src, String dst) throws IOException { return false; }
+
+    public boolean delete(String src) throws IOException { return false; }
+
+    public boolean delete(String src, boolean recursive) throws IOException { return false; }
+
+    public boolean mkdirs(String src, FsPermission masked) throws IOException { return false; }
+
+    public FileStatus[] getListing(String src) throws IOException { return null; }
+
+    public void renewLease(String clientName) throws IOException {}
+
+    public long[] getStats() throws IOException { return null; }
+
+    public DatanodeInfo[] getDatanodeReport(FSConstants.DatanodeReportType type) throws IOException { return null; }
+
+    public long getPreferredBlockSize(String filename) throws IOException { return 0; }
+
+    public boolean setSafeMode(FSConstants.SafeModeAction action) throws IOException { return false; }
+
+    public void saveNamespace() throws IOException {}
+
+    public boolean restoreFailedStorage(String arg) throws AccessControlException { return false; }
+
+    public void refreshNodes() throws IOException {}
+
+    public void finalizeUpgrade() throws IOException {}
+
+    public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action) throws IOException { return null; }
+
+    public void metaSave(String filename) throws IOException {}
+
+    public FileStatus getFileInfo(String src) throws IOException { return null; }
+
+    public ContentSummary getContentSummary(String path) throws IOException { return null; }
+
+    public void setQuota(String path, long namespaceQuota, long diskspaceQuota) throws IOException {}
+
+    public void fsync(String src, String client) throws IOException {}
+
+    public void setTimes(String src, long mtime, long atime) throws IOException {}
+
+  }
+  
+  public void testNotYetReplicatedErrors() throws IOException
+  {   
+    Configuration conf = new Configuration();
+    
+    // allow 1 retry (2 total calls)
+    conf.setInt("dfs.client.block.write.locateFollowingBlock.retries", 1);
+        
+    TestNameNode tnn = new TestNameNode(conf);
+    DFSClient client = new DFSClient(tnn, tnn, conf, null);
+    OutputStream os = client.create("testfile", true);
+    os.write(20); // write one random byte
+    
+    try {
+      os.close();
+    } catch (Exception e) {
+      assertTrue("Retries are not being stopped correctly",
+           e.getMessage().equals(tnn.ADD_BLOCK_EXCEPTION));
+    }
+  }
+  
 }
 }

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

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.Random;
 import java.util.Random;
 
 
 import javax.security.auth.login.LoginException;
 import javax.security.auth.login.LoginException;
@@ -159,9 +160,9 @@ public class TestDFSPermission extends TestCase {
     // create the file/directory
     // create the file/directory
     switch (op) {
     switch (op) {
     case CREATE:
     case CREATE:
-      FSDataOutputStream out = fs.create(name, permission, true, conf.getInt(
-          "io.file.buffer.size", 4096), fs.getDefaultReplication(), fs
-          .getDefaultBlockSize(), null);
+      FSDataOutputStream out = fs.create(name, permission, EnumSet.of(CreateFlag.OVERWRITE), 
+          conf.getInt("io.file.buffer.size", 4096),
+          fs.getDefaultReplication(), fs.getDefaultBlockSize(), null);
       out.close();
       out.close();
       break;
       break;
     case MKDIRS:
     case MKDIRS:

+ 25 - 20
src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java

@@ -17,6 +17,11 @@
  */
  */
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Op.READ_BLOCK;
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Op.WRITE_BLOCK;
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR;
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.SUCCESS;
+
 import java.io.ByteArrayOutputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
@@ -168,13 +173,13 @@ public class TestDataTransferProtocol extends TestCase {
     // bad ops
     // bad ops
     sendBuf.reset();
     sendBuf.reset();
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    sendOut.writeByte(DataTransferProtocol.OP_WRITE_BLOCK - 1);
+    sendOut.writeByte(WRITE_BLOCK.code - 1);
     sendRecvData("Wrong Op Code", true);
     sendRecvData("Wrong Op Code", true);
     
     
     /* Test OP_WRITE_BLOCK */
     /* Test OP_WRITE_BLOCK */
     sendBuf.reset();
     sendBuf.reset();
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    sendOut.writeByte(DataTransferProtocol.OP_WRITE_BLOCK);
+    WRITE_BLOCK.write(sendOut);
     sendOut.writeLong(newBlockId); // block id
     sendOut.writeLong(newBlockId); // block id
     sendOut.writeLong(0);          // generation stamp
     sendOut.writeLong(0);          // generation stamp
     sendOut.writeInt(0);           // targets in pipeline 
     sendOut.writeInt(0);           // targets in pipeline 
@@ -188,13 +193,13 @@ public class TestDataTransferProtocol extends TestCase {
     // bad bytes per checksum
     // bad bytes per checksum
     sendOut.writeInt(-1-random.nextInt(oneMil));
     sendOut.writeInt(-1-random.nextInt(oneMil));
     recvBuf.reset();
     recvBuf.reset();
-    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_ERROR);
+    ERROR.write(recvOut);
     sendRecvData("wrong bytesPerChecksum while writing", true);
     sendRecvData("wrong bytesPerChecksum while writing", true);
 
 
     sendBuf.reset();
     sendBuf.reset();
     recvBuf.reset();
     recvBuf.reset();
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    sendOut.writeByte(DataTransferProtocol.OP_WRITE_BLOCK);
+    WRITE_BLOCK.write(sendOut);
     sendOut.writeLong(newBlockId);
     sendOut.writeLong(newBlockId);
     sendOut.writeLong(0);          // generation stamp
     sendOut.writeLong(0);          // generation stamp
     sendOut.writeInt(0);           // targets in pipeline 
     sendOut.writeInt(0);           // targets in pipeline 
@@ -204,13 +209,13 @@ public class TestDataTransferProtocol extends TestCase {
 
 
     // bad number of targets
     // bad number of targets
     sendOut.writeInt(-1-random.nextInt(oneMil));
     sendOut.writeInt(-1-random.nextInt(oneMil));
-    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_ERROR);
+    ERROR.write(recvOut);
     sendRecvData("bad targets len while writing block " + newBlockId, true);
     sendRecvData("bad targets len while writing block " + newBlockId, true);
 
 
     sendBuf.reset();
     sendBuf.reset();
     recvBuf.reset();
     recvBuf.reset();
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    sendOut.writeByte(DataTransferProtocol.OP_WRITE_BLOCK);
+    WRITE_BLOCK.write(sendOut);
     sendOut.writeLong(++newBlockId);
     sendOut.writeLong(++newBlockId);
     sendOut.writeLong(0);          // generation stamp
     sendOut.writeLong(0);          // generation stamp
     sendOut.writeInt(0);           // targets in pipeline 
     sendOut.writeInt(0);           // targets in pipeline 
@@ -228,10 +233,10 @@ public class TestDataTransferProtocol extends TestCase {
     
     
     // bad data chunk length
     // bad data chunk length
     sendOut.writeInt(-1-random.nextInt(oneMil));
     sendOut.writeInt(-1-random.nextInt(oneMil));
-    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_SUCCESS);
+    SUCCESS.write(recvOut);
     Text.writeString(recvOut, ""); // first bad node
     Text.writeString(recvOut, ""); // first bad node
     recvOut.writeLong(100);        // sequencenumber
     recvOut.writeLong(100);        // sequencenumber
-    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_ERROR);
+    ERROR.write(recvOut);
     sendRecvData("negative DATA_CHUNK len while writing block " + newBlockId, 
     sendRecvData("negative DATA_CHUNK len while writing block " + newBlockId, 
                  true);
                  true);
 
 
@@ -239,7 +244,7 @@ public class TestDataTransferProtocol extends TestCase {
     sendBuf.reset();
     sendBuf.reset();
     recvBuf.reset();
     recvBuf.reset();
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    sendOut.writeByte(DataTransferProtocol.OP_WRITE_BLOCK);
+    WRITE_BLOCK.write(sendOut);
     sendOut.writeLong(++newBlockId);
     sendOut.writeLong(++newBlockId);
     sendOut.writeLong(0);          // generation stamp
     sendOut.writeLong(0);          // generation stamp
     sendOut.writeInt(0);           // targets in pipeline 
     sendOut.writeInt(0);           // targets in pipeline 
@@ -258,10 +263,10 @@ public class TestDataTransferProtocol extends TestCase {
     sendOut.writeInt(0);           // chunk length
     sendOut.writeInt(0);           // chunk length
     sendOut.writeInt(0);           // zero checksum
     sendOut.writeInt(0);           // zero checksum
     //ok finally write a block with 0 len
     //ok finally write a block with 0 len
-    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_SUCCESS);
+    SUCCESS.write(recvOut);
     Text.writeString(recvOut, ""); // first bad node
     Text.writeString(recvOut, ""); // first bad node
     recvOut.writeLong(100);        // sequencenumber
     recvOut.writeLong(100);        // sequencenumber
-    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_SUCCESS);
+    SUCCESS.write(recvOut);
     sendRecvData("Writing a zero len block blockid " + newBlockId, false);
     sendRecvData("Writing a zero len block blockid " + newBlockId, false);
     
     
     /* Test OP_READ_BLOCK */
     /* Test OP_READ_BLOCK */
@@ -270,13 +275,13 @@ public class TestDataTransferProtocol extends TestCase {
     sendBuf.reset();
     sendBuf.reset();
     recvBuf.reset();
     recvBuf.reset();
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    sendOut.writeByte(DataTransferProtocol.OP_READ_BLOCK);
+    READ_BLOCK.write(sendOut);
     newBlockId = firstBlock.getBlockId()-1;
     newBlockId = firstBlock.getBlockId()-1;
     sendOut.writeLong(newBlockId);
     sendOut.writeLong(newBlockId);
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(0L);
     sendOut.writeLong(0L);
     sendOut.writeLong(fileLen);
     sendOut.writeLong(fileLen);
-    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_ERROR);
+    ERROR.write(recvOut);
     Text.writeString(sendOut, "cl");
     Text.writeString(sendOut, "cl");
     AccessToken.DUMMY_TOKEN.write(sendOut);
     AccessToken.DUMMY_TOKEN.write(sendOut);
     sendRecvData("Wrong block ID " + newBlockId + " for read", false); 
     sendRecvData("Wrong block ID " + newBlockId + " for read", false); 
@@ -284,7 +289,7 @@ public class TestDataTransferProtocol extends TestCase {
     // negative block start offset
     // negative block start offset
     sendBuf.reset();
     sendBuf.reset();
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    sendOut.writeByte(DataTransferProtocol.OP_READ_BLOCK);
+    READ_BLOCK.write(sendOut);
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(-1L);
     sendOut.writeLong(-1L);
@@ -297,7 +302,7 @@ public class TestDataTransferProtocol extends TestCase {
     // bad block start offset
     // bad block start offset
     sendBuf.reset();
     sendBuf.reset();
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    sendOut.writeByte(DataTransferProtocol.OP_READ_BLOCK);
+    READ_BLOCK.write(sendOut);
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(fileLen);
     sendOut.writeLong(fileLen);
@@ -309,10 +314,10 @@ public class TestDataTransferProtocol extends TestCase {
     
     
     // negative length is ok. Datanode assumes we want to read the whole block.
     // negative length is ok. Datanode assumes we want to read the whole block.
     recvBuf.reset();
     recvBuf.reset();
-    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_SUCCESS);    
+    SUCCESS.write(recvOut);    
     sendBuf.reset();
     sendBuf.reset();
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    sendOut.writeByte(DataTransferProtocol.OP_READ_BLOCK);
+    READ_BLOCK.write(sendOut);
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(0);
     sendOut.writeLong(0);
@@ -324,10 +329,10 @@ public class TestDataTransferProtocol extends TestCase {
     
     
     // length is more than size of block.
     // length is more than size of block.
     recvBuf.reset();
     recvBuf.reset();
-    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_ERROR);    
+    ERROR.write(recvOut);    
     sendBuf.reset();
     sendBuf.reset();
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    sendOut.writeByte(DataTransferProtocol.OP_READ_BLOCK);
+    READ_BLOCK.write(sendOut);
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(0);
     sendOut.writeLong(0);
@@ -340,7 +345,7 @@ public class TestDataTransferProtocol extends TestCase {
     //At the end of all this, read the file to make sure that succeeds finally.
     //At the end of all this, read the file to make sure that succeeds finally.
     sendBuf.reset();
     sendBuf.reset();
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    sendOut.writeByte(DataTransferProtocol.OP_READ_BLOCK);
+    READ_BLOCK.write(sendOut);
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(0);
     sendOut.writeLong(0);

+ 8 - 8
src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java

@@ -171,7 +171,7 @@ public class TestDatanodeBlockScanner extends TestCase {
     dfsClient = new DFSClient(new InetSocketAddress("localhost", 
     dfsClient = new DFSClient(new InetSocketAddress("localhost", 
                                         cluster.getNameNodePort()), conf);
                                         cluster.getNameNodePort()), conf);
     do {
     do {
-      blocks = dfsClient.namenode.
+      blocks = dfsClient.getNamenode().
                    getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
                    getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
       blockCount = blocks.get(0).getLocations().length;
       blockCount = blocks.get(0).getLocations().length;
       try {
       try {
@@ -190,7 +190,7 @@ public class TestDatanodeBlockScanner extends TestCase {
 
 
     // We have 2 good replicas and block is not corrupt
     // We have 2 good replicas and block is not corrupt
     do {
     do {
-      blocks = dfsClient.namenode.
+      blocks = dfsClient.getNamenode().
                    getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
                    getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
       blockCount = blocks.get(0).getLocations().length;
       blockCount = blocks.get(0).getLocations().length;
       try {
       try {
@@ -218,7 +218,7 @@ public class TestDatanodeBlockScanner extends TestCase {
     // We now have the blocks to be marked as corrupt and we get back all
     // We now have the blocks to be marked as corrupt and we get back all
     // its replicas
     // its replicas
     do {
     do {
-      blocks = dfsClient.namenode.
+      blocks = dfsClient.getNamenode().
                    getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
                    getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
       blockCount = blocks.get(0).getLocations().length;
       blockCount = blocks.get(0).getLocations().length;
       try {
       try {
@@ -282,7 +282,7 @@ public class TestDatanodeBlockScanner extends TestCase {
     
     
     dfsClient = new DFSClient(new InetSocketAddress("localhost", 
     dfsClient = new DFSClient(new InetSocketAddress("localhost", 
                                         cluster.getNameNodePort()), conf);
                                         cluster.getNameNodePort()), conf);
-    blocks = dfsClient.namenode.
+    blocks = dfsClient.getNamenode().
                getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
                getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
     replicaCount = blocks.get(0).getLocations().length;
     replicaCount = blocks.get(0).getLocations().length;
 
 
@@ -294,7 +294,7 @@ public class TestDatanodeBlockScanner extends TestCase {
         Thread.sleep(1000);
         Thread.sleep(1000);
       } catch (InterruptedException ignore) {
       } catch (InterruptedException ignore) {
       }
       }
-      blocks = dfsClient.namenode.
+      blocks = dfsClient.getNamenode().
                    getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
                    getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
       replicaCount = blocks.get(0).getLocations().length;
       replicaCount = blocks.get(0).getLocations().length;
     }
     }
@@ -332,7 +332,7 @@ public class TestDatanodeBlockScanner extends TestCase {
     }
     }
     
     
     // Loop until the block recovers after replication
     // Loop until the block recovers after replication
-    blocks = dfsClient.namenode.
+    blocks = dfsClient.getNamenode().
                getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
                getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
     replicaCount = blocks.get(0).getLocations().length;
     replicaCount = blocks.get(0).getLocations().length;
     while (replicaCount != numReplicas) {
     while (replicaCount != numReplicas) {
@@ -341,7 +341,7 @@ public class TestDatanodeBlockScanner extends TestCase {
         Thread.sleep(1000);
         Thread.sleep(1000);
       } catch (InterruptedException ignore) {
       } catch (InterruptedException ignore) {
       }
       }
-      blocks = dfsClient.namenode.
+      blocks = dfsClient.getNamenode().
                  getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
                  getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
       replicaCount = blocks.get(0).getLocations().length;
       replicaCount = blocks.get(0).getLocations().length;
     }
     }
@@ -358,7 +358,7 @@ public class TestDatanodeBlockScanner extends TestCase {
       }
       }
       corruptReplicaSize = cluster.getNamesystem().
       corruptReplicaSize = cluster.getNamesystem().
                             numCorruptReplicas(blk);
                             numCorruptReplicas(blk);
-      blocks = dfsClient.namenode.
+      blocks = dfsClient.getNamenode().
                  getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
                  getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
       replicaCount = blocks.get(0).getLocations().length;
       replicaCount = blocks.get(0).getLocations().length;
     }
     }

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

@@ -21,12 +21,14 @@ import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.io.RandomAccessFile;
 import java.io.RandomAccessFile;
+import java.util.EnumSet;
 import java.util.Random;
 import java.util.Random;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
@@ -54,7 +56,7 @@ public class TestFSInputChecker extends TestCase {
   private void writeFile(FileSystem fileSys, Path name) throws IOException {
   private void writeFile(FileSystem fileSys, Path name) throws IOException {
     // create and write a file that contains three blocks of data
     // create and write a file that contains three blocks of data
     FSDataOutputStream stm = fileSys.create(name, new FsPermission((short)0777),
     FSDataOutputStream stm = fileSys.create(name, new FsPermission((short)0777),
-        true, fileSys.getConf().getInt("io.file.buffer.size", 4096),
+        EnumSet.of(CreateFlag.OVERWRITE), fileSys.getConf().getInt("io.file.buffer.size", 4096),
         NUM_OF_DATANODES, BLOCK_SIZE, null);
         NUM_OF_DATANODES, BLOCK_SIZE, null);
     stm.write(expected);
     stm.write(expected);
     stm.close();
     stm.close();

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

@@ -168,7 +168,7 @@ public class TestFileAppend extends TestCase {
       assertTrue("There should be only one datanode but found " + dn.length,
       assertTrue("There should be only one datanode but found " + dn.length,
                   dn.length == 1);
                   dn.length == 1);
 
 
-      LocatedBlocks locations = client.namenode.getBlockLocations(
+      LocatedBlocks locations = client.getNamenode().getBlockLocations(
                                   file1.toString(), 0, Long.MAX_VALUE);
                                   file1.toString(), 0, Long.MAX_VALUE);
       List<LocatedBlock> blocks = locations.getLocatedBlocks();
       List<LocatedBlock> blocks = locations.getLocatedBlocks();
       FSDataset dataset = (FSDataset) dn[0].data;
       FSDataset dataset = (FSDataset) dn[0].data;

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend3.java

@@ -160,7 +160,7 @@ public class TestFileAppend3 extends junit.framework.TestCase {
 
 
     //b. Log into one datanode that has one replica of this block.
     //b. Log into one datanode that has one replica of this block.
     //   Find the block file on this datanode and truncate it to zero size.
     //   Find the block file on this datanode and truncate it to zero size.
-    final LocatedBlocks locatedblocks = fs.dfs.namenode.getBlockLocations(p.toString(), 0L, len1);
+    final LocatedBlocks locatedblocks = fs.dfs.getNamenode().getBlockLocations(p.toString(), 0L, len1);
     assertEquals(1, locatedblocks.locatedBlockCount());
     assertEquals(1, locatedblocks.locatedBlockCount());
     final LocatedBlock lb = locatedblocks.get(0);
     final LocatedBlock lb = locatedblocks.get(0);
     final Block blk = lb.getBlock();
     final Block blk = lb.getBlock();
@@ -224,7 +224,7 @@ public class TestFileAppend3 extends junit.framework.TestCase {
 
 
     //check block sizes 
     //check block sizes 
     final long len = fs.getFileStatus(pnew).getLen();
     final long len = fs.getFileStatus(pnew).getLen();
-    final LocatedBlocks locatedblocks = fs.dfs.namenode.getBlockLocations(pnew.toString(), 0L, len);
+    final LocatedBlocks locatedblocks = fs.dfs.getNamenode().getBlockLocations(pnew.toString(), 0L, len);
     final int numblock = locatedblocks.locatedBlockCount();
     final int numblock = locatedblocks.locatedBlockCount();
     for(int i = 0; i < numblock; i++) {
     for(int i = 0; i < numblock; i++) {
       final LocatedBlock lb = locatedblocks.get(i);
       final LocatedBlock lb = locatedblocks.get(i);

+ 8 - 8
src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -372,7 +372,7 @@ public class TestFileCreation extends junit.framework.TestCase {
 
 
       // verify that no blocks are associated with this file
       // verify that no blocks are associated with this file
       // bad block allocations were cleaned up earlier.
       // bad block allocations were cleaned up earlier.
-      LocatedBlocks locations = client.namenode.getBlockLocations(
+      LocatedBlocks locations = client.getNamenode().getBlockLocations(
                                   file1.toString(), 0, Long.MAX_VALUE);
                                   file1.toString(), 0, Long.MAX_VALUE);
       System.out.println("locations = " + locations.locatedBlockCount());
       System.out.println("locations = " + locations.locatedBlockCount());
       assertTrue("Error blocks were not cleaned up",
       assertTrue("Error blocks were not cleaned up",
@@ -411,18 +411,18 @@ public class TestFileCreation extends junit.framework.TestCase {
       System.out.println("testFileCreationError2: "
       System.out.println("testFileCreationError2: "
                          + "Created file filestatus.dat with one replicas.");
                          + "Created file filestatus.dat with one replicas.");
 
 
-      LocatedBlocks locations = client.namenode.getBlockLocations(
+      LocatedBlocks locations = client.getNamenode().getBlockLocations(
                                   file1.toString(), 0, Long.MAX_VALUE);
                                   file1.toString(), 0, Long.MAX_VALUE);
       System.out.println("testFileCreationError2: "
       System.out.println("testFileCreationError2: "
           + "The file has " + locations.locatedBlockCount() + " blocks.");
           + "The file has " + locations.locatedBlockCount() + " blocks.");
 
 
       // add another block to the file
       // add another block to the file
-      LocatedBlock location = client.namenode.addBlock(file1.toString(), 
+      LocatedBlock location = client.getNamenode().addBlock(file1.toString(), 
           client.clientName);
           client.clientName);
       System.out.println("testFileCreationError2: "
       System.out.println("testFileCreationError2: "
           + "Added block " + location.getBlock());
           + "Added block " + location.getBlock());
 
 
-      locations = client.namenode.getBlockLocations(file1.toString(), 
+      locations = client.getNamenode().getBlockLocations(file1.toString(), 
                                                     0, Long.MAX_VALUE);
                                                     0, Long.MAX_VALUE);
       int count = locations.locatedBlockCount();
       int count = locations.locatedBlockCount();
       System.out.println("testFileCreationError2: "
       System.out.println("testFileCreationError2: "
@@ -439,7 +439,7 @@ public class TestFileCreation extends junit.framework.TestCase {
       }
       }
 
 
       // verify that the last block was synchronized.
       // verify that the last block was synchronized.
-      locations = client.namenode.getBlockLocations(file1.toString(), 
+      locations = client.getNamenode().getBlockLocations(file1.toString(), 
                                                     0, Long.MAX_VALUE);
                                                     0, Long.MAX_VALUE);
       System.out.println("testFileCreationError2: "
       System.out.println("testFileCreationError2: "
           + "locations = " + locations.locatedBlockCount());
           + "locations = " + locations.locatedBlockCount());
@@ -567,14 +567,14 @@ public class TestFileCreation extends junit.framework.TestCase {
 
 
       // verify that new block is associated with this file
       // verify that new block is associated with this file
       DFSClient client = ((DistributedFileSystem)fs).dfs;
       DFSClient client = ((DistributedFileSystem)fs).dfs;
-      LocatedBlocks locations = client.namenode.getBlockLocations(
+      LocatedBlocks locations = client.getNamenode().getBlockLocations(
                                   file1.toString(), 0, Long.MAX_VALUE);
                                   file1.toString(), 0, Long.MAX_VALUE);
       System.out.println("locations = " + locations.locatedBlockCount());
       System.out.println("locations = " + locations.locatedBlockCount());
       assertTrue("Error blocks were not cleaned up for file " + file1,
       assertTrue("Error blocks were not cleaned up for file " + file1,
                  locations.locatedBlockCount() == 3);
                  locations.locatedBlockCount() == 3);
 
 
       // verify filestatus2.dat
       // verify filestatus2.dat
-      locations = client.namenode.getBlockLocations(
+      locations = client.getNamenode().getBlockLocations(
                                   file2.toString(), 0, Long.MAX_VALUE);
                                   file2.toString(), 0, Long.MAX_VALUE);
       System.out.println("locations = " + locations.locatedBlockCount());
       System.out.println("locations = " + locations.locatedBlockCount());
       assertTrue("Error blocks were not cleaned up for file " + file2,
       assertTrue("Error blocks were not cleaned up for file " + file2,
@@ -790,7 +790,7 @@ public class TestFileCreation extends junit.framework.TestCase {
       // wait for the lease to expire
       // wait for the lease to expire
       try {Thread.sleep(5 * leasePeriod);} catch (InterruptedException e) {}
       try {Thread.sleep(5 * leasePeriod);} catch (InterruptedException e) {}
 
 
-      LocatedBlocks locations = dfs.dfs.namenode.getBlockLocations(
+      LocatedBlocks locations = dfs.dfs.getNamenode().getBlockLocations(
           f, 0, Long.MAX_VALUE);
           f, 0, Long.MAX_VALUE);
       assertEquals(1, locations.locatedBlockCount());
       assertEquals(1, locations.locatedBlockCount());
       LocatedBlock locatedblock = locations.getLocatedBlocks().get(0);
       LocatedBlock locatedblock = locations.getLocatedBlocks().get(0);

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

@@ -76,7 +76,7 @@ public class TestGetBlocks extends TestCase {
       boolean notWritten;
       boolean notWritten;
       do {
       do {
         DFSClient dfsclient = new DFSClient(CONF);
         DFSClient dfsclient = new DFSClient(CONF);
-        locatedBlocks = dfsclient.namenode.
+        locatedBlocks = dfsclient.getNamenode().
           getBlockLocations("/tmp.txt", 0, fileLen).getLocatedBlocks();
           getBlockLocations("/tmp.txt", 0, fileLen).getLocatedBlocks();
         assertEquals(2, locatedBlocks.size());
         assertEquals(2, locatedBlocks.size());
         notWritten = false;
         notWritten = false;

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java

@@ -148,7 +148,7 @@ public class TestInjectionForSimulatedStorage extends TestCase {
       writeFile(cluster.getFileSystem(), testPath, numDataNodes);
       writeFile(cluster.getFileSystem(), testPath, numDataNodes);
 
 
       
       
-      waitForBlockReplication(testFile, dfsClient.namenode, numDataNodes, 20);
+      waitForBlockReplication(testFile, dfsClient.getNamenode(), numDataNodes, 20);
 
 
       
       
       Block[][] blocksList = cluster.getAllBlockReports();
       Block[][] blocksList = cluster.getAllBlockReports();
@@ -188,7 +188,7 @@ public class TestInjectionForSimulatedStorage extends TestCase {
                                   cluster.getNameNodePort()),
                                   cluster.getNameNodePort()),
                                   conf);
                                   conf);
       
       
-      waitForBlockReplication(testFile, dfsClient.namenode, numDataNodes, -1);
+      waitForBlockReplication(testFile, dfsClient.getNamenode(), numDataNodes, -1);
       
       
     } finally {
     } finally {
       if (cluster != null) {
       if (cluster != null) {

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

@@ -75,7 +75,7 @@ public class TestLeaseRecovery extends junit.framework.TestCase {
 
 
       //get block info for the last block
       //get block info for the last block
       LocatedBlock locatedblock = TestInterDatanodeProtocol.getLastLocatedBlock(
       LocatedBlock locatedblock = TestInterDatanodeProtocol.getLastLocatedBlock(
-          dfs.dfs.namenode, filestr);
+          dfs.dfs.getNamenode(), filestr);
       DatanodeInfo[] datanodeinfos = locatedblock.getLocations();
       DatanodeInfo[] datanodeinfos = locatedblock.getLocations();
       assertEquals(REPLICATION_NUM, datanodeinfos.length);
       assertEquals(REPLICATION_NUM, datanodeinfos.length);
 
 

+ 20 - 16
src/test/hdfs/org/apache/hadoop/hdfs/TestReplication.java

@@ -17,26 +17,30 @@
  */
  */
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
-import junit.framework.TestCase;
-import java.io.*;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.Random;
 import java.util.Random;
-import java.net.*;
+
+import junit.framework.TestCase;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.BlockLocation;
 
 
 /**
 /**
  * This class tests the replication of a DFS file.
  * This class tests the replication of a DFS file.
@@ -167,7 +171,7 @@ public class TestReplication extends TestCase {
     fs.setReplication(file1, (short)2);
     fs.setReplication(file1, (short)2);
   
   
     // Now get block details and check if the block is corrupt
     // Now get block details and check if the block is corrupt
-    blocks = dfsClient.namenode.
+    blocks = dfsClient.getNamenode().
               getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
               getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
     while (blocks.get(0).isCorrupt() != true) {
     while (blocks.get(0).isCorrupt() != true) {
       try {
       try {
@@ -175,7 +179,7 @@ public class TestReplication extends TestCase {
         Thread.sleep(1000);
         Thread.sleep(1000);
       } catch (InterruptedException ie) {
       } catch (InterruptedException ie) {
       }
       }
-      blocks = dfsClient.namenode.
+      blocks = dfsClient.getNamenode().
                 getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
                 getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
     }
     }
     replicaCount = blocks.get(0).getLocations().length;
     replicaCount = blocks.get(0).getLocations().length;
@@ -317,10 +321,10 @@ public class TestReplication extends TestCase {
       out.write(buffer);
       out.write(buffer);
       out.close();
       out.close();
       
       
-      waitForBlockReplication(testFile, dfsClient.namenode, numDataNodes, -1);
+      waitForBlockReplication(testFile, dfsClient.getNamenode(), numDataNodes, -1);
 
 
       // get first block of the file.
       // get first block of the file.
-      String block = dfsClient.namenode.
+      String block = dfsClient.getNamenode().
                        getBlockLocations(testFile, 0, Long.MAX_VALUE).
                        getBlockLocations(testFile, 0, Long.MAX_VALUE).
                        get(0).getBlock().getBlockName();
                        get(0).getBlock().getBlockName();
       
       
@@ -382,7 +386,7 @@ public class TestReplication extends TestCase {
                                   cluster.getNameNodePort()),
                                   cluster.getNameNodePort()),
                                   conf);
                                   conf);
       
       
-      waitForBlockReplication(testFile, dfsClient.namenode, numDataNodes, -1);
+      waitForBlockReplication(testFile, dfsClient.getNamenode(), numDataNodes, -1);
       
       
     } finally {
     } finally {
       if (cluster != null) {
       if (cluster != null) {
@@ -432,19 +436,19 @@ public class TestReplication extends TestCase {
     // block replication triggers corrupt block detection
     // block replication triggers corrupt block detection
     DFSClient dfsClient = new DFSClient(new InetSocketAddress("localhost", 
     DFSClient dfsClient = new DFSClient(new InetSocketAddress("localhost", 
         cluster.getNameNodePort()), fs.getConf());
         cluster.getNameNodePort()), fs.getConf());
-    LocatedBlocks blocks = dfsClient.namenode.getBlockLocations(
+    LocatedBlocks blocks = dfsClient.getNamenode().getBlockLocations(
         fileName.toString(), 0, fileLen);
         fileName.toString(), 0, fileLen);
     if (lenDelta < 0) { // replica truncated
     if (lenDelta < 0) { // replica truncated
     	while (!blocks.get(0).isCorrupt() || 
     	while (!blocks.get(0).isCorrupt() || 
     			REPLICATION_FACTOR != blocks.get(0).getLocations().length) {
     			REPLICATION_FACTOR != blocks.get(0).getLocations().length) {
     		Thread.sleep(100);
     		Thread.sleep(100);
-    		blocks = dfsClient.namenode.getBlockLocations(
+    		blocks = dfsClient.getNamenode().getBlockLocations(
     				fileName.toString(), 0, fileLen);
     				fileName.toString(), 0, fileLen);
     	}
     	}
     } else { // no corruption detected; block replicated
     } else { // no corruption detected; block replicated
     	while (REPLICATION_FACTOR+1 != blocks.get(0).getLocations().length) {
     	while (REPLICATION_FACTOR+1 != blocks.get(0).getLocations().length) {
     		Thread.sleep(100);
     		Thread.sleep(100);
-    		blocks = dfsClient.namenode.getBlockLocations(
+    		blocks = dfsClient.getNamenode().getBlockLocations(
     				fileName.toString(), 0, fileLen);
     				fileName.toString(), 0, fileLen);
     	}
     	}
     }
     }

+ 58 - 0
src/test/hdfs/org/apache/hadoop/hdfs/TestSetTimes.java

@@ -183,6 +183,64 @@ public class TestSetTimes extends TestCase {
     }
     }
   }
   }
 
 
+  /**
+   * Tests mod time change at close in DFS.
+   */
+  public void testTimesAtClose() throws IOException {
+    Configuration conf = new Configuration();
+    final int MAX_IDLE_TIME = 2000; // 2s
+    int replicas = 1;
+
+    // parameter initialization
+    conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
+    conf.setInt("heartbeat.recheck.interval", 1000);
+    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt("dfs.datanode.handler.count", 50);
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, true, null);
+    cluster.waitActive();
+    InetSocketAddress addr = new InetSocketAddress("localhost",
+                                                     cluster.getNameNodePort());
+    DFSClient client = new DFSClient(addr, conf);
+    DatanodeInfo[] info = client.datanodeReport(DatanodeReportType.LIVE);
+    assertEquals("Number of Datanodes ", numDatanodes, info.length);
+    FileSystem fileSys = cluster.getFileSystem();
+    assertTrue(fileSys instanceof DistributedFileSystem);
+
+    try {
+      // create a new file and write to it
+      Path file1 = new Path("/simple.dat");
+      FSDataOutputStream stm = writeFile(fileSys, file1, replicas);
+      System.out.println("Created and wrote file simple.dat");
+      FileStatus statBeforeClose = fileSys.getFileStatus(file1);
+      long mtimeBeforeClose = statBeforeClose.getModificationTime();
+      String mdateBeforeClose = dateForm.format(new Date(
+                                                     mtimeBeforeClose));
+      System.out.println("mtime on " + file1 + " before close is "
+                  + mdateBeforeClose + " (" + mtimeBeforeClose + ")");
+      assertTrue(mtimeBeforeClose != 0);
+
+      //close file after writing
+      stm.close();
+      System.out.println("Closed file.");
+      FileStatus statAfterClose = fileSys.getFileStatus(file1);
+      long mtimeAfterClose = statAfterClose.getModificationTime();
+      String mdateAfterClose = dateForm.format(new Date(mtimeAfterClose));
+      System.out.println("mtime on " + file1 + " after close is "
+                  + mdateAfterClose + " (" + mtimeAfterClose + ")");
+      assertTrue(mtimeAfterClose != 0);
+      assertTrue(mtimeBeforeClose != mtimeAfterClose);
+
+      cleanupFile(fileSys, file1);
+    } catch (IOException e) {
+      info = client.datanodeReport(DatanodeReportType.ALL);
+      printDatanodeReport(info);
+      throw e;
+    } finally {
+      fileSys.close();
+      cluster.shutdown();
+    }
+  }
+
   public static void main(String[] args) throws Exception {
   public static void main(String[] args) throws Exception {
     new TestSetTimes().testTimes();
     new TestSetTimes().testTimes();
   }
   }

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

@@ -17,6 +17,9 @@
  */
  */
 package org.apache.hadoop.hdfs.server.datanode;
 package org.apache.hadoop.hdfs.server.datanode;
 
 
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Op.REPLACE_BLOCK;
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.*;
+
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
@@ -44,7 +47,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.common.Util;
-import org.apache.hadoop.hdfs.server.datanode.BlockTransferThrottler;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessToken;
 import org.apache.hadoop.security.AccessToken;
@@ -108,7 +110,7 @@ public class TestBlockReplacement extends TestCase {
       InetSocketAddress addr = new InetSocketAddress("localhost",
       InetSocketAddress addr = new InetSocketAddress("localhost",
           cluster.getNameNodePort());
           cluster.getNameNodePort());
       DFSClient client = new DFSClient(addr, CONF);
       DFSClient client = new DFSClient(addr, CONF);
-      List<LocatedBlock> locatedBlocks = client.namenode.
+      List<LocatedBlock> locatedBlocks = client.getNamenode().
         getBlockLocations("/tmp.txt", 0, DEFAULT_BLOCK_SIZE).getLocatedBlocks();
         getBlockLocations("/tmp.txt", 0, DEFAULT_BLOCK_SIZE).getLocatedBlocks();
       assertEquals(1, locatedBlocks.size());
       assertEquals(1, locatedBlocks.size());
       LocatedBlock block = locatedBlocks.get(0);
       LocatedBlock block = locatedBlocks.get(0);
@@ -192,7 +194,7 @@ public class TestBlockReplacement extends TestCase {
         Thread.sleep(100);
         Thread.sleep(100);
       } catch(InterruptedException e) {
       } catch(InterruptedException e) {
       }
       }
-      List<LocatedBlock> blocks = client.namenode.
+      List<LocatedBlock> blocks = client.getNamenode().
       getBlockLocations(fileName, 0, fileLen).getLocatedBlocks();
       getBlockLocations(fileName, 0, fileLen).getLocatedBlocks();
       assertEquals(1, blocks.size());
       assertEquals(1, blocks.size());
       DatanodeInfo[] nodes = blocks.get(0).getLocations();
       DatanodeInfo[] nodes = blocks.get(0).getLocations();
@@ -227,7 +229,7 @@ public class TestBlockReplacement extends TestCase {
     // sendRequest
     // sendRequest
     DataOutputStream out = new DataOutputStream(sock.getOutputStream());
     DataOutputStream out = new DataOutputStream(sock.getOutputStream());
     out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
     out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
-    out.writeByte(DataTransferProtocol.OP_REPLACE_BLOCK);
+    REPLACE_BLOCK.write(out);
     out.writeLong(block.getBlockId());
     out.writeLong(block.getBlockId());
     out.writeLong(block.getGenerationStamp());
     out.writeLong(block.getGenerationStamp());
     Text.writeString(out, source.getStorageID());
     Text.writeString(out, source.getStorageID());
@@ -237,11 +239,7 @@ public class TestBlockReplacement extends TestCase {
     // receiveResponse
     // receiveResponse
     DataInputStream reply = new DataInputStream(sock.getInputStream());
     DataInputStream reply = new DataInputStream(sock.getInputStream());
 
 
-    short status = reply.readShort();
-    if(status == DataTransferProtocol.OP_STATUS_SUCCESS) {
-      return true;
-    }
-    return false;
+    return DataTransferProtocol.Status.read(reply) == SUCCESS;
   }
   }
 
 
   /**
   /**

+ 5 - 5
src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java

@@ -67,7 +67,7 @@ public class TestDirectoryScanner extends TestCase {
   /** Truncate a block file */
   /** Truncate a block file */
   private long truncateBlockFile() throws IOException {
   private long truncateBlockFile() throws IOException {
     synchronized (fds) {
     synchronized (fds) {
-      for (Entry<Block, DatanodeBlockInfo> entry : fds.volumeMap.entrySet()) {
+      for (Entry<Block, ReplicaInfo> entry : fds.volumeMap.entrySet()) {
         Block b = entry.getKey();
         Block b = entry.getKey();
         File f = entry.getValue().getFile();
         File f = entry.getValue().getFile();
         File mf = FSDataset.getMetaFile(f, b);
         File mf = FSDataset.getMetaFile(f, b);
@@ -87,7 +87,7 @@ public class TestDirectoryScanner extends TestCase {
   /** Delete a block file */
   /** Delete a block file */
   private long deleteBlockFile() {
   private long deleteBlockFile() {
     synchronized(fds) {
     synchronized(fds) {
-      for (Entry<Block, DatanodeBlockInfo> entry : fds.volumeMap.entrySet()) {
+      for (Entry<Block, ReplicaInfo> entry : fds.volumeMap.entrySet()) {
         Block b = entry.getKey();
         Block b = entry.getKey();
         File f = entry.getValue().getFile();
         File f = entry.getValue().getFile();
         File mf = FSDataset.getMetaFile(f, b);
         File mf = FSDataset.getMetaFile(f, b);
@@ -104,7 +104,7 @@ public class TestDirectoryScanner extends TestCase {
   /** Delete block meta file */
   /** Delete block meta file */
   private long deleteMetaFile() {
   private long deleteMetaFile() {
     synchronized(fds) {
     synchronized(fds) {
-      for (Entry<Block, DatanodeBlockInfo> entry : fds.volumeMap.entrySet()) {
+      for (Entry<Block, ReplicaInfo> entry : fds.volumeMap.entrySet()) {
         Block b = entry.getKey();
         Block b = entry.getKey();
         String blkfile = entry.getValue().getFile().getAbsolutePath();
         String blkfile = entry.getValue().getFile().getAbsolutePath();
         long genStamp = b.getGenerationStamp();
         long genStamp = b.getGenerationStamp();
@@ -126,7 +126,7 @@ public class TestDirectoryScanner extends TestCase {
     while (true) {
     while (true) {
       id = rand.nextLong();
       id = rand.nextLong();
       Block b = new Block(id);
       Block b = new Block(id);
-      DatanodeBlockInfo info = null;
+      ReplicaInfo info = null;
       synchronized(fds) {
       synchronized(fds) {
         info = fds.volumeMap.get(b);
         info = fds.volumeMap.get(b);
       }
       }
@@ -326,7 +326,7 @@ public class TestDirectoryScanner extends TestCase {
   private void verifyAddition(long blockId, long genStamp, long size) {
   private void verifyAddition(long blockId, long genStamp, long size) {
     Block memBlock = fds.getBlockKey(blockId);
     Block memBlock = fds.getBlockKey(blockId);
     assertNotNull(memBlock);
     assertNotNull(memBlock);
-    DatanodeBlockInfo blockInfo;
+    ReplicaInfo blockInfo;
     synchronized(fds) {
     synchronized(fds) {
       blockInfo = fds.volumeMap.get(memBlock);
       blockInfo = fds.volumeMap.get(memBlock);
     }
     }

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

@@ -17,11 +17,15 @@
  */
  */
 package org.apache.hadoop.hdfs.server.datanode;
 package org.apache.hadoop.hdfs.server.datanode;
 
 
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Op.WRITE_BLOCK;
+
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.File;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.Socket;
 
 
+import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -33,8 +37,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.AccessToken;
 import org.apache.hadoop.security.AccessToken;
 
 
-import junit.framework.TestCase;
-
 /** Test if a datanode can correctly handle errors during block read/write*/
 /** Test if a datanode can correctly handle errors during block read/write*/
 public class TestDiskError extends TestCase {
 public class TestDiskError extends TestCase {
   public void testShutdown() throws Exception {
   public void testShutdown() throws Exception {
@@ -112,7 +114,7 @@ public class TestDiskError extends TestCase {
           s.getOutputStream());
           s.getOutputStream());
 
 
       out.writeShort( DataTransferProtocol.DATA_TRANSFER_VERSION );
       out.writeShort( DataTransferProtocol.DATA_TRANSFER_VERSION );
-      out.write( DataTransferProtocol.OP_WRITE_BLOCK );
+      WRITE_BLOCK.write(out);
       out.writeLong( block.getBlock().getBlockId());
       out.writeLong( block.getBlock().getBlockId());
       out.writeLong( block.getBlock().getGenerationStamp() );
       out.writeLong( block.getBlock().getGenerationStamp() );
       out.writeInt(1);
       out.writeInt(1);

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

@@ -83,7 +83,7 @@ public class TestInterDatanodeProtocol extends junit.framework.TestCase {
       assertTrue(dfs.getClient().exists(filestr));
       assertTrue(dfs.getClient().exists(filestr));
 
 
       //get block info
       //get block info
-      LocatedBlock locatedblock = getLastLocatedBlock(dfs.getClient().namenode, filestr);
+      LocatedBlock locatedblock = getLastLocatedBlock(dfs.getClient().getNamenode(), filestr);
       DatanodeInfo[] datanodeinfo = locatedblock.getLocations();
       DatanodeInfo[] datanodeinfo = locatedblock.getLocations();
       assertTrue(datanodeinfo.length > 0);
       assertTrue(datanodeinfo.length > 0);
 
 

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

@@ -19,15 +19,12 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
 
 
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 
 
 /**
 /**
  * 
  * 

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

@@ -175,7 +175,7 @@ public class TestFsck extends TestCase {
       String[] fileNames = util.getFileNames(topDir);
       String[] fileNames = util.getFileNames(topDir);
       DFSClient dfsClient = new DFSClient(new InetSocketAddress("localhost",
       DFSClient dfsClient = new DFSClient(new InetSocketAddress("localhost",
                                           cluster.getNameNodePort()), conf);
                                           cluster.getNameNodePort()), conf);
-      String block = dfsClient.namenode.
+      String block = dfsClient.getNamenode().
                       getBlockLocations(fileNames[0], 0, Long.MAX_VALUE).
                       getBlockLocations(fileNames[0], 0, Long.MAX_VALUE).
                       get(0).getBlock().getBlockName();
                       get(0).getBlock().getBlockName();
       File baseDir = new File(System.getProperty("test.build.data",
       File baseDir = new File(System.getProperty("test.build.data",
@@ -315,7 +315,7 @@ public class TestFsck extends TestCase {
 
 
     dfsClient = new DFSClient(new InetSocketAddress("localhost",
     dfsClient = new DFSClient(new InetSocketAddress("localhost",
                                cluster.getNameNodePort()), conf);
                                cluster.getNameNodePort()), conf);
-    blocks = dfsClient.namenode.
+    blocks = dfsClient.getNamenode().
                getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
                getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
     replicaCount = blocks.get(0).getLocations().length;
     replicaCount = blocks.get(0).getLocations().length;
     while (replicaCount != 3) {
     while (replicaCount != 3) {
@@ -323,7 +323,7 @@ public class TestFsck extends TestCase {
         Thread.sleep(100);
         Thread.sleep(100);
       } catch (InterruptedException ignore) {
       } catch (InterruptedException ignore) {
       }
       }
-      blocks = dfsClient.namenode.
+      blocks = dfsClient.getNamenode().
                 getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
                 getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
       replicaCount = blocks.get(0).getLocations().length;
       replicaCount = blocks.get(0).getLocations().length;
     }
     }

+ 2 - 1
src/test/hdfs/org/apache/hadoop/security/TestPermission.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.security;
 package org.apache.hadoop.security;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.Random;
 import java.util.Random;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -86,7 +87,7 @@ public class TestPermission extends TestCase {
 
 
       FsPermission filePerm = new FsPermission((short)0444);
       FsPermission filePerm = new FsPermission((short)0444);
       FSDataOutputStream out = fs.create(new Path("/b1/b2/b3.txt"), filePerm,
       FSDataOutputStream out = fs.create(new Path("/b1/b2/b3.txt"), filePerm,
-          true, conf.getInt("io.file.buffer.size", 4096),
+          EnumSet.of(CreateFlag.OVERWRITE), conf.getInt("io.file.buffer.size", 4096),
           fs.getDefaultReplication(), fs.getDefaultBlockSize(), null);
           fs.getDefaultReplication(), fs.getDefaultBlockSize(), null);
       out.write(123);
       out.write(123);
       out.close();
       out.close();