You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ra...@apache.org on 2008/09/24 00:32:54 UTC

svn commit: r698377 [1/2] - in /hadoop/core/branches/branch-0.19: ./ src/hdfs/org/apache/hadoop/hdfs/ src/hdfs/org/apache/hadoop/hdfs/protocol/ src/hdfs/org/apache/hadoop/hdfs/server/balancer/ src/hdfs/org/apache/hadoop/hdfs/server/common/ src/hdfs/org...

Author: rangadi
Date: Tue Sep 23 15:32:53 2008
New Revision: 698377

URL: http://svn.apache.org/viewvc?rev=698377&view=rev
Log:
Merging HADOOP-4006 committed to trunk (r 697306) to 0.19.
Original log :
HADOOP-4006. Clean up FSConstants and move some of the constants to 
better places. (Sanjay Radia via rangadi)

Added:
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java
      - copied unchanged from r697306, hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/HdfsConstants.java
      - copied unchanged from r697306, hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/common/HdfsConstants.java
Modified:
    hadoop/core/branches/branch-0.19/   (props changed)
    hadoop/core/branches/branch-0.19/CHANGES.txt
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/protocol/FSConstants.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/balancer/Balancer.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeManager.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeObjectCollection.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/Upgradeable.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/UpgradeManagerNamenode.java
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/UpgradeObjectNamenode.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/DataNodeCluster.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/MiniDFSCluster.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/NNThroughputBenchmark.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSFinalize.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSRollback.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSUpgrade.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/UpgradeUtilities.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java

Propchange: hadoop/core/branches/branch-0.19/
------------------------------------------------------------------------------
    svn:mergeinfo = /hadoop/core/trunk:697306

Modified: hadoop/core/branches/branch-0.19/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/CHANGES.txt?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/CHANGES.txt (original)
+++ hadoop/core/branches/branch-0.19/CHANGES.txt Tue Sep 23 15:32:53 2008
@@ -414,6 +414,9 @@
 
     HADOOP-4227. Remove the deprecated class org.apache.hadoop.fs.ShellCommand.
     (szetszwo)
+
+    HADOOP-4006. Clean up FSConstants and move some of the constants to 
+    better places. (Sanjay Radia via rangadi)
  
   OPTIMIZATIONS
 

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java Tue Sep 23 15:32:53 2008
@@ -29,6 +29,7 @@
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.hdfs.DistributedFileSystem.DiskStatus;
 import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -157,9 +158,9 @@
     this.conf = conf;
     this.stats = stats;
     this.socketTimeout = conf.getInt("dfs.socket.timeout", 
-                                     FSConstants.READ_TIMEOUT);
+                                     HdfsConstants.READ_TIMEOUT);
     this.datanodeWriteTimeout = conf.getInt("dfs.datanode.socket.write.timeout",
-                                            FSConstants.WRITE_TIMEOUT);
+                                            HdfsConstants.WRITE_TIMEOUT);
     this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
     // dfs.write.packet.size is an internal config variable
     this.writePacketSize = conf.getInt("dfs.write.packet.size", 64*1024);
@@ -630,16 +631,17 @@
         try {
           if (LOG.isDebugEnabled()) {
             LOG.debug("write to " + datanodes[j].getName() + ": "
-                + OP_BLOCK_CHECKSUM + ", block=" + block);
+                + DataTransferProtocol.OP_BLOCK_CHECKSUM +
+                ", block=" + block);
           }
-          out.writeShort(DATA_TRANSFER_VERSION);
-          out.write(OP_BLOCK_CHECKSUM);
+          out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
+          out.write(DataTransferProtocol.OP_BLOCK_CHECKSUM);
           out.writeLong(block.getBlockId());
           out.writeLong(block.getGenerationStamp());
           out.flush();
          
           final short reply = in.readShort();
-          if (reply != OP_STATUS_SUCCESS) {
+          if (reply != DataTransferProtocol.OP_STATUS_SUCCESS) {
             throw new IOException("Bad response " + reply + " for block "
                 + block + " from datanode " + datanodes[j].getName());
           }
@@ -1254,11 +1256,11 @@
                                        throws IOException {
       // in and out will be closed when sock is closed (by the caller)
       DataOutputStream out = new DataOutputStream(
-        new BufferedOutputStream(NetUtils.getOutputStream(sock,WRITE_TIMEOUT)));
+        new BufferedOutputStream(NetUtils.getOutputStream(sock,HdfsConstants.WRITE_TIMEOUT)));
 
       //write the header.
-      out.writeShort( DATA_TRANSFER_VERSION );
-      out.write( OP_READ_BLOCK );
+      out.writeShort( DataTransferProtocol.DATA_TRANSFER_VERSION );
+      out.write( DataTransferProtocol.OP_READ_BLOCK );
       out.writeLong( blockId );
       out.writeLong( genStamp );
       out.writeLong( startOffset );
@@ -1274,7 +1276,7 @@
           new BufferedInputStream(NetUtils.getInputStream(sock), 
                                   bufferSize));
       
-      if ( in.readShort() != OP_STATUS_SUCCESS ) {
+      if ( in.readShort() != DataTransferProtocol.OP_STATUS_SUCCESS ) {
         throw new IOException("Got error in response to OP_READ_BLOCK " +
                               "for file " + file + 
                               " for block " + blockId);
@@ -1316,9 +1318,9 @@
      */ 
     private void checksumOk(Socket sock) {
       try {
-        OutputStream out = NetUtils.getOutputStream(sock, WRITE_TIMEOUT);
-        byte buf[] = { (OP_STATUS_CHECKSUM_OK >>> 8) & 0xff,
-                       (OP_STATUS_CHECKSUM_OK) & 0xff };
+        OutputStream out = NetUtils.getOutputStream(sock, HdfsConstants.WRITE_TIMEOUT);
+        byte buf[] = { (DataTransferProtocol.OP_STATUS_CHECKSUM_OK >>> 8) & 0xff,
+                       (DataTransferProtocol.OP_STATUS_CHECKSUM_OK) & 0xff };
         out.write(buf);
         out.flush();
       } catch (IOException e) {
@@ -2320,7 +2322,7 @@
             // processes response status from all datanodes.
             for (int i = 0; i < targets.length && clientRunning; i++) {
               short reply = blockReplyStream.readShort();
-              if (reply != OP_STATUS_SUCCESS) {
+              if (reply != DataTransferProtocol.OP_STATUS_SUCCESS) {
                 errorIndex = i; // first bad datanode
                 throw new IOException("Bad response " + reply +
                                       " for block " + block +
@@ -2715,7 +2717,7 @@
         s.setSoTimeout(timeoutValue);
         s.setSendBufferSize(DEFAULT_DATA_SOCKET_SIZE);
         LOG.debug("Send buf size " + s.getSendBufferSize());
-        long writeTimeout = WRITE_TIMEOUT_EXTENSION * nodes.length +
+        long writeTimeout = HdfsConstants.WRITE_TIMEOUT_EXTENSION * nodes.length +
                             datanodeWriteTimeout;
 
         //
@@ -2726,8 +2728,8 @@
                                      DataNode.SMALL_BUFFER_SIZE));
         blockReplyStream = new DataInputStream(NetUtils.getInputStream(s));
 
-        out.writeShort( DATA_TRANSFER_VERSION );
-        out.write( OP_WRITE_BLOCK );
+        out.writeShort( DataTransferProtocol.DATA_TRANSFER_VERSION );
+        out.write( DataTransferProtocol.OP_WRITE_BLOCK );
         out.writeLong( block.getBlockId() );
         out.writeLong( block.getGenerationStamp() );
         out.writeInt( nodes.length );

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/protocol/FSConstants.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/protocol/FSConstants.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/protocol/FSConstants.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/protocol/FSConstants.java Tue Sep 23 15:32:53 2008
@@ -26,93 +26,6 @@
 public interface FSConstants {
   public static int MIN_BLOCKS_FOR_WRITE = 5;
 
-  //
-  // IPC Opcodes 
-  //
-  // Processed at namenode
-  public static final byte OP_ERROR = (byte) 0;
-  public static final byte OP_HEARTBEAT = (byte) 1;
-  public static final byte OP_BLOCKRECEIVED = (byte) 2;
-  public static final byte OP_BLOCKREPORT = (byte) 3;
-  public static final byte OP_TRANSFERDATA = (byte) 4;
-
-  // Processed at namenode, from client
-  public static final byte OP_CLIENT_OPEN = (byte) 20;
-  public static final byte OP_CLIENT_STARTFILE = (byte) 21;
-  public static final byte OP_CLIENT_ADDBLOCK = (byte) 22;
-  public static final byte OP_CLIENT_RENAMETO = (byte) 23;
-  public static final byte OP_CLIENT_DELETE = (byte) 24;  
-  public static final byte OP_CLIENT_COMPLETEFILE = (byte) 25;
-  public static final byte OP_CLIENT_LISTING = (byte) 26;
-  public static final byte OP_CLIENT_OBTAINLOCK = (byte) 27;
-  public static final byte OP_CLIENT_RELEASELOCK = (byte) 28;
-  public static final byte OP_CLIENT_EXISTS = (byte) 29;
-  public static final byte OP_CLIENT_ISDIR = (byte) 30;
-  public static final byte OP_CLIENT_MKDIRS = (byte) 31;
-  public static final byte OP_CLIENT_RENEW_LEASE = (byte) 32;
-  public static final byte OP_CLIENT_ABANDONBLOCK = (byte) 33;
-  public static final byte OP_CLIENT_RAWSTATS = (byte) 34;
-  public static final byte OP_CLIENT_DATANODEREPORT = (byte) 35;
-  public static final byte OP_CLIENT_DATANODE_HINTS = (byte) 36;
-    
-  // Processed at datanode, back from namenode
-  public static final byte OP_ACK = (byte) 40;
-  public static final byte OP_TRANSFERBLOCKS = (byte) 41;    
-  public static final byte OP_INVALIDATE_BLOCKS = (byte) 42;
-  public static final byte OP_FAILURE = (byte) 43;
-
-  // Processed at client, back from namenode
-  public static final byte OP_CLIENT_OPEN_ACK = (byte) 60;
-  public static final byte OP_CLIENT_STARTFILE_ACK = (byte) 61;
-  public static final byte OP_CLIENT_ADDBLOCK_ACK = (byte) 62;
-  public static final byte OP_CLIENT_RENAMETO_ACK = (byte) 63;
-  public static final byte OP_CLIENT_DELETE_ACK = (byte) 64;
-  public static final byte OP_CLIENT_COMPLETEFILE_ACK = (byte) 65;
-  public static final byte OP_CLIENT_TRYAGAIN = (byte) 66;
-  public static final byte OP_CLIENT_LISTING_ACK = (byte) 67;
-  public static final byte OP_CLIENT_OBTAINLOCK_ACK = (byte) 68;
-  public static final byte OP_CLIENT_RELEASELOCK_ACK = (byte) 69;
-  public static final byte OP_CLIENT_EXISTS_ACK = (byte) 70;  
-  public static final byte OP_CLIENT_ISDIR_ACK = (byte) 71;
-  public static final byte OP_CLIENT_MKDIRS_ACK = (byte) 72;
-  public static final byte OP_CLIENT_RENEW_LEASE_ACK = (byte) 73;    
-  public static final byte OP_CLIENT_ABANDONBLOCK_ACK = (byte) 74;
-  public static final byte OP_CLIENT_RAWSTATS_ACK = (byte) 75;
-  public static final byte OP_CLIENT_DATANODEREPORT_ACK = (byte) 76;
-  public static final byte OP_CLIENT_DATANODE_HINTS_ACK = (byte) 77;
-
-  // Processed at datanode stream-handler
-  public static final byte OP_WRITE_BLOCK = (byte) 80;
-  public static final byte OP_READ_BLOCK = (byte) 81;
-  public static final byte OP_READ_METADATA = (byte) 82;
-  public static final byte OP_REPLACE_BLOCK = (byte) 83;
-  public static final byte OP_COPY_BLOCK = (byte) 84;
-  public static final byte OP_BLOCK_CHECKSUM = (byte) 85;
-  
-  public static final int OP_STATUS_SUCCESS = 0;  
-  public static final int OP_STATUS_ERROR = 1;  
-  public static final int OP_STATUS_ERROR_CHECKSUM = 2;  
-  public static final int OP_STATUS_ERROR_INVALID = 3;  
-  public static final int OP_STATUS_ERROR_EXISTS = 4;  
-  public static final int OP_STATUS_CHECKSUM_OK = 5;  
-
-  
-  /** Version for data transfers between clients and datanodes
-   * This should change when serialization of DatanodeInfo, not just
-   * when protocol changes. It is not very obvious. 
-   */
-  /*
-   * Version 13:
-   *    Added a new operation, OP_BLOCK_CHECKSUM, for obtaining
-   *    the checksum of a block from a datanode.
-   */
-  public static final int DATA_TRANSFER_VERSION = 13;
-
-  // Return codes for file create
-  public static final int OPERATION_FAILED = 0;
-  public static final int STILL_WAITING = 1;
-  public static final int COMPLETE_SUCCESS = 2;
-
   // Chunk the block Invalidate message
   public static final int BLOCK_INVALIDATE_CHUNK = 100;
 
@@ -128,10 +41,7 @@
   public static long BLOCKREPORT_INITIAL_DELAY = 0;
   public static final long LEASE_SOFTLIMIT_PERIOD = 60 * 1000;
   public static final long LEASE_HARDLIMIT_PERIOD = 60 * LEASE_SOFTLIMIT_PERIOD;
-  public static int READ_TIMEOUT = 60 * 1000;
-  public static int WRITE_TIMEOUT = 8 * 60 * 1000;  
-  public static int WRITE_TIMEOUT_EXTENSION = 5 * 1000; //for write pipeline
-
+  
   // We need to limit the length and depth of a path in the filesystem.  HADOOP-438
   // Currently we set the maximum length to 8k characters and the maximum depth to 1k.  
   public static int MAX_PATH_LENGTH = 8000;
@@ -149,34 +59,9 @@
   // SafeMode actions
   public enum SafeModeAction{ SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET; }
 
-  // Startup options
-  public enum StartupOption{
-    FORMAT  ("-format"),
-    REGULAR ("-regular"),
-    UPGRADE ("-upgrade"),
-    ROLLBACK("-rollback"),
-    FINALIZE("-finalize"),
-    IMPORT  ("-importCheckpoint");
-    
-    private String name = null;
-    private StartupOption(String arg) {this.name = arg;}
-    public String getName() {return name;}
-  }
-
   // type of the datanode report
   public static enum DatanodeReportType {ALL, LIVE, DEAD }
 
-  // checkpoint states
-  public enum CheckpointStates{ START, ROLLED_EDITS, UPLOAD_START, UPLOAD_DONE; }
-
-  /**
-   * Type of the node
-   */
-  static public enum NodeType {
-    NAME_NODE,
-    DATA_NODE;
-  }
-
   /**
    * Distributed upgrade actions:
    * 

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/balancer/Balancer.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/balancer/Balancer.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/balancer/Balancer.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/balancer/Balancer.java Tue Sep 23 15:32:53 2008
@@ -51,6 +51,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.protocol.*;
@@ -306,9 +307,9 @@
       DataInputStream in = null;
       try {
         sock.connect(DataNode.createSocketAddr(
-            proxySource.datanode.getName()), FSConstants.READ_TIMEOUT);
+            proxySource.datanode.getName()), HdfsConstants.READ_TIMEOUT);
         long bandwidth = conf.getLong("dfs.balance.bandwidthPerSec", 1024L*1024);
-        sock.setSoTimeout(2*FSConstants.READ_TIMEOUT+
+        sock.setSoTimeout(2*HdfsConstants.READ_TIMEOUT+
             (int)(block.getNumBytes()*1500/bandwidth));
         out = new DataOutputStream( new BufferedOutputStream(
             sock.getOutputStream(), FSConstants.BUFFER_SIZE));
@@ -357,8 +358,8 @@
     
     /* Send a block copy request to the outputstream*/
     private void sendRequest(DataOutputStream out) throws IOException {
-      out.writeShort(FSConstants.DATA_TRANSFER_VERSION);
-      out.writeByte(FSConstants.OP_COPY_BLOCK);
+      out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
+      out.writeByte(DataTransferProtocol.OP_COPY_BLOCK);
       out.writeLong(block.getBlock().getBlockId());
       out.writeLong(block.getBlock().getGenerationStamp());
       Text.writeString(out, source.getStorageID());
@@ -369,7 +370,7 @@
     /* Receive a block copy response from the input stream */ 
     private void receiveResponse(DataInputStream in) throws IOException {
       short status = in.readShort();
-      if (status != FSConstants.OP_STATUS_SUCCESS) {
+      if (status != DataTransferProtocol.OP_STATUS_SUCCESS) {
         throw new IOException("Moving block "+block.getBlockId()+
             " from "+source.getName() + " to " +
             target.getName() + " through " +

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java Tue Sep 23 15:32:53 2008
@@ -32,8 +32,8 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
-import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
-import org.apache.hadoop.hdfs.protocol.FSConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
@@ -362,9 +362,9 @@
 
       this.lock(); // lock storage if it exists
 
-      if (startOpt == StartupOption.FORMAT)
+      if (startOpt == HdfsConstants.StartupOption.FORMAT)
         return StorageState.NOT_FORMATTED;
-      if (startOpt != StartupOption.IMPORT) {
+      if (startOpt != HdfsConstants.StartupOption.IMPORT) {
         //make sure no conversion is required
         checkConversionNeeded(this);
       }

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeManager.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeManager.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeManager.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeManager.java Tue Sep 23 15:32:53 2008
@@ -83,7 +83,7 @@
     return false;
   }
 
-  public abstract FSConstants.NodeType getType();
+  public abstract HdfsConstants.NodeType getType();
   public abstract boolean startUpgrade() throws IOException;
   public abstract void completeUpgrade() throws IOException;
 }

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeObjectCollection.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeObjectCollection.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeObjectCollection.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/UpgradeObjectCollection.java Tue Sep 23 15:32:53 2008
@@ -38,7 +38,7 @@
 
   static class UOSignature implements Comparable<UOSignature> {
     int version;
-    FSConstants.NodeType type;
+    HdfsConstants.NodeType type;
     String className;
 
     UOSignature(Upgradeable uo) {
@@ -51,7 +51,7 @@
       return version;
     }
 
-    FSConstants.NodeType getType() {
+    HdfsConstants.NodeType getType() {
       return type;
     }
 
@@ -109,7 +109,7 @@
   }
 
   public static SortedSet<Upgradeable> getDistributedUpgrades(int versionFrom, 
-                                                       FSConstants.NodeType type
+                                                       HdfsConstants.NodeType type
                                                        ) throws IOException {
     assert FSConstants.LAYOUT_VERSION <= versionFrom : "Incorrect version " 
       + versionFrom + ". Expected to be <= " + FSConstants.LAYOUT_VERSION;

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/Upgradeable.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/Upgradeable.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/Upgradeable.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/common/Upgradeable.java Tue Sep 23 15:32:53 2008
@@ -19,7 +19,6 @@
 
 import java.io.IOException;
 
-import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 
 /**
@@ -41,7 +40,7 @@
    * Get the type of the software component, which this object is upgrading.
    * @return type
    */
-  FSConstants.NodeType getType();
+  HdfsConstants.NodeType getType();
 
   /**
    * Description of the upgrade object for displaying.

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java Tue Sep 23 15:32:53 2008
@@ -33,6 +33,7 @@
 import org.apache.hadoop.fs.FSInputChecker;
 import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -794,7 +795,7 @@
             }
 
             replyOut.writeLong(expected);
-            replyOut.writeShort(OP_STATUS_SUCCESS);
+            replyOut.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS);
             replyOut.flush();
         } catch (Exception e) {
           if (running) {
@@ -824,7 +825,7 @@
       while (running && datanode.shouldRun && !lastPacketInBlock) {
 
         try {
-            short op = OP_STATUS_SUCCESS;
+            short op = DataTransferProtocol.OP_STATUS_SUCCESS;
             boolean didRead = false;
             long expected = -2;
             try { 
@@ -889,7 +890,7 @@
             }
             
             if (!didRead) {
-              op = OP_STATUS_ERROR;
+              op = DataTransferProtocol.OP_STATUS_ERROR;
             }
             
             // If this is the last packet in block, then close block
@@ -916,7 +917,7 @@
 
             // send my status back to upstream datanode
             replyOut.writeLong(expected); // send seqno upstream
-            replyOut.writeShort(OP_STATUS_SUCCESS);
+            replyOut.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS);
 
             LOG.debug("PacketResponder " + numTargets + 
                       " for block " + block +
@@ -926,16 +927,16 @@
             // forward responses from downstream datanodes.
             for (int i = 0; i < numTargets && datanode.shouldRun; i++) {
               try {
-                if (op == OP_STATUS_SUCCESS) {
+                if (op == DataTransferProtocol.OP_STATUS_SUCCESS) {
                   op = mirrorIn.readShort();
-                  if (op != OP_STATUS_SUCCESS) {
+                  if (op != DataTransferProtocol.OP_STATUS_SUCCESS) {
                     LOG.debug("PacketResponder for block " + block +
                               ": error code received from downstream " +
                               " datanode[" + i + "] " + op);
                   }
                 }
               } catch (Throwable e) {
-                op = OP_STATUS_ERROR;
+                op = DataTransferProtocol.OP_STATUS_ERROR;
               }
               replyOut.writeShort(op);
             }
@@ -950,7 +951,7 @@
             // If we forwarded an error response from a downstream datanode
             // and we are acting on behalf of a client, then we quit. The 
             // client will drive the recovery mechanism.
-            if (op == OP_STATUS_ERROR && receiver.clientName.length() > 0) {
+            if (op == DataTransferProtocol.OP_STATUS_ERROR && receiver.clientName.length() > 0) {
               running = false;
             }
         } catch (IOException e) {

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java Tue Sep 23 15:32:53 2008
@@ -48,11 +48,14 @@
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.UnregisteredDatanodeException;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -239,9 +242,9 @@
     InetSocketAddress nameNodeAddr = NameNode.getAddress(conf);
     
     this.socketTimeout =  conf.getInt("dfs.socket.timeout",
-                                      FSConstants.READ_TIMEOUT);
+                                      HdfsConstants.READ_TIMEOUT);
     this.socketWriteTimeout = conf.getInt("dfs.datanode.socket.write.timeout",
-                                          FSConstants.WRITE_TIMEOUT);
+                                          HdfsConstants.WRITE_TIMEOUT);
     /* Based on results on different platforms, we might need set the default 
      * to false on some of them. */
     this.transferToAllowed = conf.getBoolean("dfs.datanode.transferTo.allowed", 
@@ -1046,7 +1049,7 @@
         sock.setSoTimeout(targets.length * socketTimeout);
 
         long writeTimeout = socketWriteTimeout + 
-                            WRITE_TIMEOUT_EXTENSION * (targets.length-1);
+                            HdfsConstants.WRITE_TIMEOUT_EXTENSION * (targets.length-1);
         OutputStream baseStream = NetUtils.getOutputStream(sock, writeTimeout);
         out = new DataOutputStream(new BufferedOutputStream(baseStream, 
                                                             SMALL_BUFFER_SIZE));
@@ -1058,8 +1061,8 @@
         //
         // Header info
         //
-        out.writeShort(DATA_TRANSFER_VERSION);
-        out.writeByte(OP_WRITE_BLOCK);
+        out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
+        out.writeByte(DataTransferProtocol.OP_WRITE_BLOCK);
         out.writeLong(b.getBlockId());
         out.writeLong(b.getGenerationStamp());
         out.writeInt(0);           // no pipelining

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataStorage.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataStorage.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataStorage.java Tue Sep 23 15:32:53 2008
@@ -33,8 +33,9 @@
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
-import org.apache.hadoop.hdfs.protocol.FSConstants.NodeType;
-import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java Tue Sep 23 15:32:53 2008
@@ -29,8 +29,10 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.MetaDataInputStream;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
@@ -74,7 +76,7 @@
           new BufferedInputStream(NetUtils.getInputStream(s), 
                                   SMALL_BUFFER_SIZE));
       short version = in.readShort();
-      if ( version != DATA_TRANSFER_VERSION ) {
+      if ( version != DataTransferProtocol.DATA_TRANSFER_VERSION ) {
         throw new IOException( "Version Mismatch" );
       }
       boolean local = s.getInetAddress().equals(s.getLocalAddress());
@@ -88,7 +90,7 @@
       }
       long startTime = DataNode.now();
       switch ( op ) {
-      case OP_READ_BLOCK:
+      case DataTransferProtocol.OP_READ_BLOCK:
         readBlock( in );
         datanode.myMetrics.readBlockOp.inc(DataNode.now() - startTime);
         if (local)
@@ -96,7 +98,7 @@
         else
           datanode.myMetrics.readsFromRemoteClient.inc();
         break;
-      case OP_WRITE_BLOCK:
+      case DataTransferProtocol.OP_WRITE_BLOCK:
         writeBlock( in );
         datanode.myMetrics.writeBlockOp.inc(DataNode.now() - startTime);
         if (local)
@@ -104,19 +106,20 @@
         else
           datanode.myMetrics.writesFromRemoteClient.inc();
         break;
-      case OP_READ_METADATA:
+      case DataTransferProtocol.OP_READ_METADATA:
         readMetadata( in );
         datanode.myMetrics.readMetadataOp.inc(DataNode.now() - startTime);
         break;
-      case OP_REPLACE_BLOCK: // for balancing purpose; send to a destination
+      case DataTransferProtocol.OP_REPLACE_BLOCK: // for balancing purpose; send to a destination
         replaceBlock(in);
         datanode.myMetrics.replaceBlockOp.inc(DataNode.now() - startTime);
         break;
-      case OP_COPY_BLOCK: // for balancing purpose; send to a proxy source
+      case DataTransferProtocol.OP_COPY_BLOCK:
+            // for balancing purpose; send to a proxy source
         copyBlock(in);
         datanode.myMetrics.copyBlockOp.inc(DataNode.now() - startTime);
         break;
-      case OP_BLOCK_CHECKSUM: //get the checksum of a block
+      case DataTransferProtocol.OP_BLOCK_CHECKSUM: //get the checksum of a block
         getBlockChecksum(in);
         datanode.myMetrics.blockChecksumOp.inc(DataNode.now() - startTime);
         break;
@@ -168,18 +171,18 @@
         blockSender = new BlockSender(block, startOffset, length,
             true, true, false, datanode, clientTraceFmt);
       } catch(IOException e) {
-        out.writeShort(OP_STATUS_ERROR);
+        out.writeShort(DataTransferProtocol.OP_STATUS_ERROR);
         throw e;
       }
 
-      out.writeShort(DataNode.OP_STATUS_SUCCESS); // send op status
+      out.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS); // send op status
       long read = blockSender.sendBlock(out, baseStream, null); // send data
 
       if (blockSender.isBlockReadFully()) {
         // See if client verification succeeded. 
         // This is an optional response from client.
         try {
-          if (in.readShort() == OP_STATUS_CHECKSUM_OK  && 
+          if (in.readShort() == DataTransferProtocol.OP_STATUS_CHECKSUM_OK  && 
               datanode.blockScanner != null) {
             datanode.blockScanner.verifiedByClient(block);
           }
@@ -274,7 +277,7 @@
         try {
           int timeoutValue = numTargets * datanode.socketTimeout;
           int writeTimeout = datanode.socketWriteTimeout + 
-                             (WRITE_TIMEOUT_EXTENSION * numTargets);
+                             (HdfsConstants.WRITE_TIMEOUT_EXTENSION * numTargets);
           mirrorSock.connect(mirrorTarget, timeoutValue);
           mirrorSock.setSoTimeout(timeoutValue);
           mirrorSock.setSendBufferSize(DEFAULT_DATA_SOCKET_SIZE);
@@ -285,8 +288,8 @@
           mirrorIn = new DataInputStream(NetUtils.getInputStream(mirrorSock));
 
           // Write header: Copied from DFSClient.java!
-          mirrorOut.writeShort( DATA_TRANSFER_VERSION );
-          mirrorOut.write( OP_WRITE_BLOCK );
+          mirrorOut.writeShort( DataTransferProtocol.DATA_TRANSFER_VERSION );
+          mirrorOut.write( DataTransferProtocol.OP_WRITE_BLOCK );
           mirrorOut.writeLong( block.getBlockId() );
           mirrorOut.writeLong( block.getGenerationStamp() );
           mirrorOut.writeInt( pipelineSize );
@@ -407,7 +410,7 @@
       out = new DataOutputStream(
                 NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
       
-      out.writeByte(OP_STATUS_SUCCESS);
+      out.writeByte(DataTransferProtocol.OP_STATUS_SUCCESS);
       out.writeInt(buf.length);
       out.write(buf);
       
@@ -450,7 +453,7 @@
       //write reply
       out = new DataOutputStream(
           NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
-      out.writeShort(OP_STATUS_SUCCESS);
+      out.writeShort(DataTransferProtocol.OP_STATUS_SUCCESS);
       out.writeInt(bytesPerCRC);
       out.writeLong(crcPerBlock);
       md5.write(out);
@@ -477,7 +480,7 @@
     target.readFields(in);
 
     Socket targetSock = null;
-    short opStatus = OP_STATUS_SUCCESS;
+    short opStatus = DataTransferProtocol.OP_STATUS_SUCCESS;
     BlockSender blockSender = null;
     DataOutputStream targetOut = null;
     try {
@@ -501,8 +504,8 @@
 
       /* send request to the target */
       // fist write header info
-      targetOut.writeShort(DATA_TRANSFER_VERSION); // transfer version
-      targetOut.writeByte(OP_REPLACE_BLOCK); // op code
+      targetOut.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION); // transfer version
+      targetOut.writeByte(DataTransferProtocol.OP_REPLACE_BLOCK); // op code
       targetOut.writeLong(block.getBlockId()); // block id
       targetOut.writeLong(block.getGenerationStamp()); // block id
       Text.writeString( targetOut, source); // del hint
@@ -519,7 +522,7 @@
 
       LOG.info("Copied block " + block + " to " + targetAddr);
     } catch (IOException ioe) {
-      opStatus = OP_STATUS_ERROR;
+      opStatus = DataTransferProtocol.OP_STATUS_ERROR;
       LOG.warn("Got exception while serving " + block + " to "
           + target.getName() + ": " + StringUtils.stringifyException(ioe));
       throw ioe;
@@ -553,7 +556,7 @@
         in.readLong()); // block id & len
     String sourceID = Text.readString(in);
 
-    short opStatus = OP_STATUS_SUCCESS;
+    short opStatus = DataTransferProtocol.OP_STATUS_SUCCESS;
     BlockReceiver blockReceiver = null;
     try {
       // open a block receiver and check if the block does not exist
@@ -571,7 +574,7 @@
       LOG.info("Moved block " + block + 
           " from " + s.getRemoteSocketAddress());
     } catch (IOException ioe) {
-      opStatus = OP_STATUS_ERROR;
+      opStatus = DataTransferProtocol.OP_STATUS_ERROR;
       throw ioe;
     } finally {
       // send response back
@@ -597,7 +600,7 @@
     try {
       for (int i = 0; i < numTargets; i++) {
         short opStatus = reply.readShort();
-        if(opStatus != OP_STATUS_SUCCESS) {
+        if(opStatus != DataTransferProtocol.OP_STATUS_SUCCESS) {
           throw new IOException("operation failed at "+
               s.getInetAddress());
         } 

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java Tue Sep 23 15:32:53 2008
@@ -20,6 +20,7 @@
 import java.io.IOException;
 
 import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeManager;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
@@ -43,8 +44,8 @@
     this.dataNode = dataNode;
   }
 
-  public FSConstants.NodeType getType() {
-    return FSConstants.NodeType.DATA_NODE;
+  public HdfsConstants.NodeType getType() {
+    return HdfsConstants.NodeType.DATA_NODE;
   }
 
   synchronized void initializeUpgrade(NamespaceInfo nsInfo) throws IOException {

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java Tue Sep 23 15:32:53 2008
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeObject;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@@ -33,8 +34,8 @@
 public abstract class UpgradeObjectDatanode extends UpgradeObject implements Runnable {
   private DataNode dataNode = null;
 
-  public FSConstants.NodeType getType() {
-    return FSConstants.NodeType.DATA_NODE;
+  public HdfsConstants.NodeType getType() {
+    return HdfsConstants.NodeType.DATA_NODE;
   }
 
   protected DataNode getDatanode() {

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java Tue Sep 23 15:32:53 2008
@@ -31,6 +31,7 @@
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;;
 import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 
 /*************************************************

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java Tue Sep 23 15:32:53 2008
@@ -47,14 +47,14 @@
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
-import org.apache.hadoop.hdfs.protocol.FSConstants.CheckpointStates;
-import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
-import org.apache.hadoop.hdfs.protocol.FSConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLog.EditLogFileInputStream;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -83,7 +83,9 @@
     private NameNodeFile(String name) {this.fileName = name;}
     String getName() {return fileName;}
   }
-  
+
+  // checkpoint states
+  enum CheckpointStates{START, ROLLED_EDITS, UPLOAD_START, UPLOAD_DONE; }
   /**
    * Implementation of StorageDirType specific to namenode storage
    * A Storage directory could be of type IMAGE which stores only fsimage,
@@ -119,7 +121,7 @@
   /**
    * Can fs-image be rolled?
    */
-  volatile private CheckpointStates ckptState = CheckpointStates.START; 
+  volatile private CheckpointStates ckptState = FSImage.CheckpointStates.START; 
 
   /**
    * Used for saving the image to disk
@@ -1320,7 +1322,7 @@
         it.remove();
       }
     }
-    ckptState = CheckpointStates.START;
+    ckptState = FSImage.CheckpointStates.START;
   }
 
   CheckpointSignature rollEditLog() throws IOException {
@@ -1349,7 +1351,7 @@
                             ". Checkpoint Aborted.");
     }
     sig.validateStorageInfo(this);
-    ckptState = CheckpointStates.UPLOAD_START;
+    ckptState = FSImage.CheckpointStates.UPLOAD_START;
   }
 
   /**

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Tue Sep 23 15:32:53 2008
@@ -23,6 +23,7 @@
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
@@ -1349,13 +1350,21 @@
    * Before we return, we make sure that all the file's blocks have 
    * been reported by datanodes and are replicated correctly.
    */
-  public int completeFile(String src, String holder) throws IOException {
-    int status = completeFileInternal(src, holder);
+  
+  enum CompleteFileStatus {
+    OPERATION_FAILED,
+    STILL_WAITING,
+    COMPLETE_SUCCESS
+  }
+  
+  public CompleteFileStatus completeFile(String src, String holder) throws IOException {
+    CompleteFileStatus status = completeFileInternal(src, holder);
     getEditLog().logSync();
     return status;
   }
 
-  private synchronized int completeFileInternal(String src, 
+
+  private synchronized CompleteFileStatus completeFileInternal(String src, 
                                                 String holder) throws IOException {
     NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " + src + " for " + holder);
     if (isInSafeMode())
@@ -1376,9 +1385,9 @@
                                    ((pendingFile == null) ? "null" : 
                                      ("from " + pendingFile.getClientMachine()))
                                   );                      
-      return OPERATION_FAILED;
+      return CompleteFileStatus.OPERATION_FAILED;
     } else if (!checkFileProgress(pendingFile, true)) {
-      return STILL_WAITING;
+      return CompleteFileStatus.STILL_WAITING;
     }
 
     finalizeINodeFileUnderConstruction(src, pendingFile);
@@ -1387,7 +1396,7 @@
       NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " + src
                                   + " blocklist persisted");
     }
-    return COMPLETE_SUCCESS;
+    return CompleteFileStatus.COMPLETE_SUCCESS;
   }
 
   /** 

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java Tue Sep 23 15:32:53 2008
@@ -34,6 +34,7 @@
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
@@ -82,7 +83,7 @@
       xml.declaration();
 
       final Configuration conf = new Configuration(DataNode.getDataNode().getConf());
-      final int socketTimeout = conf.getInt("dfs.socket.timeout", FSConstants.READ_TIMEOUT);
+      final int socketTimeout = conf.getInt("dfs.socket.timeout", HdfsConstants.READ_TIMEOUT);
       final SocketFactory socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
       UnixUserGroupInformation.saveToConf(conf,
           UnixUserGroupInformation.UGI_PROPERTY_NAME, ugi);

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java Tue Sep 23 15:32:53 2008
@@ -35,9 +35,9 @@
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.fs.Path;
@@ -101,8 +101,8 @@
         
       try {
         s = new Socket();
-        s.connect(targetAddr, FSConstants.READ_TIMEOUT);
-        s.setSoTimeout(FSConstants.READ_TIMEOUT);
+        s.connect(targetAddr, HdfsConstants.READ_TIMEOUT);
+        s.setSoTimeout(HdfsConstants.READ_TIMEOUT);
       } catch (IOException e) {
         deadNodes.add(chosenNode);
         s.close();
@@ -122,8 +122,8 @@
     throws IOException {
     if (chunkSizeToView == 0) return;
     Socket s = new Socket();
-    s.connect(addr, FSConstants.READ_TIMEOUT);
-    s.setSoTimeout(FSConstants.READ_TIMEOUT);
+    s.connect(addr, HdfsConstants.READ_TIMEOUT);
+    s.setSoTimeout(HdfsConstants.READ_TIMEOUT);
       
       long amtToRead = Math.min(chunkSizeToView, blockSize - offsetIntoBlock);     
       

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java Tue Sep 23 15:32:53 2008
@@ -26,8 +26,10 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.CompleteFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
@@ -171,11 +173,11 @@
    * <p>
    * The name-node can be started with one of the following startup options:
    * <ul> 
-   * <li>{@link org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption#REGULAR REGULAR} - normal startup</li>
-   * <li>{@link org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption#FORMAT FORMAT} - format name node</li>
-   * <li>{@link org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption#UPGRADE UPGRADE} - start the cluster  
+   * <li>{@link org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption#REGULAR REGULAR} - normal startup</li>
+   * <li>{@link org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption#FORMAT FORMAT} - format name node</li>
+   * <li>{@link org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption#UPGRADE UPGRADE} - start the cluster  
    * upgrade and create a snapshot of the current file system state</li> 
-   * <li>{@link org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption#ROLLBACK ROLLBACK} - roll the  
+   * <li>{@link org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption#ROLLBACK ROLLBACK} - roll the  
    *            cluster back to the previous state</li>
    * </ul>
    * The option is passed via configuration field: 
@@ -367,10 +369,10 @@
   /** {@inheritDoc} */
   public boolean complete(String src, String clientName) throws IOException {
     stateChangeLog.debug("*DIR* NameNode.complete: " + src + " for " + clientName);
-    int returnCode = namesystem.completeFile(src, clientName);
-    if (returnCode == STILL_WAITING) {
+    CompleteFileStatus returnCode = namesystem.completeFile(src, clientName);
+    if (returnCode == CompleteFileStatus.STILL_WAITING) {
       return false;
-    } else if (returnCode == COMPLETE_SUCCESS) {
+    } else if (returnCode == CompleteFileStatus.COMPLETE_SUCCESS) {
       return true;
     } else {
       throw new IOException("Could not complete write to file " + src + " by " + clientName);

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java Tue Sep 23 15:32:53 2008
@@ -36,10 +36,10 @@
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 
@@ -400,8 +400,8 @@
       }
       try {
         s = new Socket();
-        s.connect(targetAddr, FSConstants.READ_TIMEOUT);
-        s.setSoTimeout(FSConstants.READ_TIMEOUT);
+        s.connect(targetAddr, HdfsConstants.READ_TIMEOUT);
+        s.setSoTimeout(HdfsConstants.READ_TIMEOUT);
         
         blockReader = 
           DFSClient.BlockReader.newBlockReader(s, targetAddr.toString() + ":" + 

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java Tue Sep 23 15:32:53 2008
@@ -22,6 +22,7 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
@@ -513,7 +514,7 @@
               "cannot access checkpoint directory.");
         StorageState curState;
         try {
-          curState = sd.analyzeStorage(FSConstants.StartupOption.REGULAR);
+          curState = sd.analyzeStorage(HdfsConstants.StartupOption.REGULAR);
           // sd is locked but not opened
           switch(curState) {
           case NON_EXISTENT:

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/UpgradeManagerNamenode.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/UpgradeManagerNamenode.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/UpgradeManagerNamenode.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/UpgradeManagerNamenode.java Tue Sep 23 15:32:53 2008
@@ -22,6 +22,7 @@
 
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.UpgradeManager;
 import org.apache.hadoop.hdfs.server.common.UpgradeObjectCollection;
@@ -40,8 +41,8 @@
  * and updates its status.
  */
 class UpgradeManagerNamenode extends UpgradeManager {
-  public FSConstants.NodeType getType() {
-    return FSConstants.NodeType.NAME_NODE;
+  public HdfsConstants.NodeType getType() {
+    return HdfsConstants.NodeType.NAME_NODE;
   }
 
   /**
@@ -140,7 +141,7 @@
     UpgradeManagerNamenode um = new UpgradeManagerNamenode();
     SortedSet<Upgradeable> uos;
     uos = UpgradeObjectCollection.getDistributedUpgrades(-4, 
-        FSConstants.NodeType.NAME_NODE);
+        HdfsConstants.NodeType.NAME_NODE);
     System.out.println(uos.size());
     um.startUpgrade();
   }

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/UpgradeObjectNamenode.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/UpgradeObjectNamenode.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/UpgradeObjectNamenode.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/UpgradeObjectNamenode.java Tue Sep 23 15:32:53 2008
@@ -20,6 +20,7 @@
 import java.io.IOException;
 
 import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeObject;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 
@@ -42,8 +43,8 @@
   public abstract UpgradeCommand processUpgradeCommand(UpgradeCommand command
                                                ) throws IOException;
 
-  public FSConstants.NodeType getType() {
-    return FSConstants.NodeType.NAME_NODE;
+  public HdfsConstants.NodeType getType() {
+    return HdfsConstants.NodeType.NAME_NODE;
   }
 
   /**

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/DataNodeCluster.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/DataNodeCluster.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/DataNodeCluster.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/DataNodeCluster.java Tue Sep 23 15:32:53 2008
@@ -25,7 +25,8 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/MiniDFSCluster.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/MiniDFSCluster.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/MiniDFSCluster.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/MiniDFSCluster.java Tue Sep 23 15:32:53 2008
@@ -33,7 +33,8 @@
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/NNThroughputBenchmark.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/NNThroughputBenchmark.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/NNThroughputBenchmark.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/NNThroughputBenchmark.java Tue Sep 23 15:32:53 2008
@@ -77,7 +77,7 @@
  * Then the benchmark executes the specified number of operations using 
  * the specified number of threads and outputs the resulting stats.
  */
-public class NNThroughputBenchmark implements FSConstants {
+public class NNThroughputBenchmark {
   private static final Log LOG = LogFactory.getLog(NNThroughputBenchmark.class);
   private static final int BLOCK_SIZE = 16;
 

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSFinalize.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSFinalize.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSFinalize.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSFinalize.java Tue Sep 23 15:32:53 2008
@@ -23,9 +23,9 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import static org.apache.hadoop.hdfs.protocol.FSConstants.NodeType.NAME_NODE;
-import static org.apache.hadoop.hdfs.protocol.FSConstants.NodeType.DATA_NODE;
-import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 
 /**
  * This test ensures the appropriate response from the system when 

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSRollback.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSRollback.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSRollback.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSRollback.java Tue Sep 23 15:32:53 2008
@@ -23,10 +23,13 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.FSConstants.NodeType;
-import static org.apache.hadoop.hdfs.protocol.FSConstants.NodeType.NAME_NODE;
-import static org.apache.hadoop.hdfs.protocol.FSConstants.NodeType.DATA_NODE;
-import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
+
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.fs.FileUtil;
 

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSStartupVersions.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSStartupVersions.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSStartupVersions.java Tue Sep 23 15:32:53 2008
@@ -23,10 +23,13 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
-import org.apache.hadoop.hdfs.protocol.FSConstants.NodeType;
-import static org.apache.hadoop.hdfs.protocol.FSConstants.NodeType.NAME_NODE;
-import static org.apache.hadoop.hdfs.protocol.FSConstants.NodeType.DATA_NODE;
-import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
+
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.fs.Path;

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java Tue Sep 23 15:32:53 2008
@@ -23,10 +23,11 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.FSConstants.NodeType;
-import static org.apache.hadoop.hdfs.protocol.FSConstants.NodeType.NAME_NODE;
-import static org.apache.hadoop.hdfs.protocol.FSConstants.NodeType.DATA_NODE;
-import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
 
 /**
 * This test ensures the appropriate response (successful or failure) from

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSUpgrade.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSUpgrade.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSUpgrade.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSUpgrade.java Tue Sep 23 15:32:53 2008
@@ -23,10 +23,13 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.FSConstants.NodeType;
-import static org.apache.hadoop.hdfs.protocol.FSConstants.NodeType.NAME_NODE;
-import static org.apache.hadoop.hdfs.protocol.FSConstants.NodeType.DATA_NODE;
-import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
+
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.fs.FileUtil;

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java Tue Sep 23 15:32:53 2008
@@ -31,7 +31,7 @@
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
-import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDataTransferProtocol.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDataTransferProtocol.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDataTransferProtocol.java Tue Sep 23 15:32:53 2008
@@ -34,9 +34,10 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -75,8 +76,8 @@
         LOG.info("Testing : " + testDescription);
       }
       sock = new Socket();
-      sock.connect(dnAddr, FSConstants.READ_TIMEOUT);
-      sock.setSoTimeout(FSConstants.READ_TIMEOUT);
+      sock.connect(dnAddr, HdfsConstants.READ_TIMEOUT);
+      sock.setSoTimeout(HdfsConstants.READ_TIMEOUT);
       
       OutputStream out = sock.getOutputStream();
       // Should we excuse 
@@ -156,20 +157,20 @@
     sendBuf.reset();
     
     // bad version
-    recvOut.writeShort((short)(FSConstants.DATA_TRANSFER_VERSION-1));
-    sendOut.writeShort((short)(FSConstants.DATA_TRANSFER_VERSION-1));
+    recvOut.writeShort((short)(DataTransferProtocol.DATA_TRANSFER_VERSION-1));
+    sendOut.writeShort((short)(DataTransferProtocol.DATA_TRANSFER_VERSION-1));
     sendRecvData("Wrong Version", true);
 
     // bad ops
     sendBuf.reset();
-    sendOut.writeShort((short)FSConstants.DATA_TRANSFER_VERSION);
-    sendOut.writeByte((byte)(FSConstants.OP_WRITE_BLOCK-1));
+    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
+    sendOut.writeByte((byte)(DataTransferProtocol.OP_WRITE_BLOCK-1));
     sendRecvData("Wrong Op Code", true);
     
     /* Test OP_WRITE_BLOCK */
     sendBuf.reset();
-    sendOut.writeShort((short)FSConstants.DATA_TRANSFER_VERSION);
-    sendOut.writeByte((byte)FSConstants.OP_WRITE_BLOCK);
+    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
+    sendOut.writeByte((byte)DataTransferProtocol.OP_WRITE_BLOCK);
     sendOut.writeLong(newBlockId); // block id
     sendOut.writeLong(0);          // generation stamp
     sendOut.writeInt(0);           // targets in pipeline 
@@ -182,13 +183,13 @@
     // bad bytes per checksum
     sendOut.writeInt(-1-random.nextInt(oneMil));
     recvBuf.reset();
-    recvOut.writeShort((short)FSConstants.OP_STATUS_ERROR);
+    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_ERROR);
     sendRecvData("wrong bytesPerChecksum while writing", true);
 
     sendBuf.reset();
     recvBuf.reset();
-    sendOut.writeShort((short)FSConstants.DATA_TRANSFER_VERSION);
-    sendOut.writeByte((byte)FSConstants.OP_WRITE_BLOCK);
+    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
+    sendOut.writeByte((byte)DataTransferProtocol.OP_WRITE_BLOCK);
     sendOut.writeLong(newBlockId);
     sendOut.writeLong(0);          // generation stamp
     sendOut.writeInt(0);           // targets in pipeline 
@@ -198,13 +199,13 @@
 
     // bad number of targets
     sendOut.writeInt(-1-random.nextInt(oneMil));
-    recvOut.writeShort((short)FSConstants.OP_STATUS_ERROR);
+    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_ERROR);
     sendRecvData("bad targets len while writing block " + newBlockId, true);
 
     sendBuf.reset();
     recvBuf.reset();
-    sendOut.writeShort((short)FSConstants.DATA_TRANSFER_VERSION);
-    sendOut.writeByte((byte)FSConstants.OP_WRITE_BLOCK);
+    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
+    sendOut.writeByte((byte)DataTransferProtocol.OP_WRITE_BLOCK);
     sendOut.writeLong(++newBlockId);
     sendOut.writeLong(0);          // generation stamp
     sendOut.writeInt(0);           // targets in pipeline 
@@ -223,15 +224,15 @@
     sendOut.writeInt(-1-random.nextInt(oneMil));
     Text.writeString(recvOut, ""); // first bad node
     recvOut.writeLong(100);        // sequencenumber
-    recvOut.writeShort((short)FSConstants.OP_STATUS_ERROR);
+    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_ERROR);
     sendRecvData("negative DATA_CHUNK len while writing block " + newBlockId, 
                  true);
 
     // test for writing a valid zero size block
     sendBuf.reset();
     recvBuf.reset();
-    sendOut.writeShort((short)FSConstants.DATA_TRANSFER_VERSION);
-    sendOut.writeByte((byte)FSConstants.OP_WRITE_BLOCK);
+    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
+    sendOut.writeByte((byte)DataTransferProtocol.OP_WRITE_BLOCK);
     sendOut.writeLong(++newBlockId);
     sendOut.writeLong(0);          // generation stamp
     sendOut.writeInt(0);           // targets in pipeline 
@@ -251,7 +252,7 @@
     //ok finally write a block with 0 len
     Text.writeString(recvOut, ""); // first bad node
     recvOut.writeLong(100);        // sequencenumber
-    recvOut.writeShort((short)FSConstants.OP_STATUS_SUCCESS);
+    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_SUCCESS);
     sendRecvData("Writing a zero len block blockid " + newBlockId, false);
     
     /* Test OP_READ_BLOCK */
@@ -259,21 +260,21 @@
     // bad block id
     sendBuf.reset();
     recvBuf.reset();
-    sendOut.writeShort((short)FSConstants.DATA_TRANSFER_VERSION);
-    sendOut.writeByte((byte)FSConstants.OP_READ_BLOCK);
+    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
+    sendOut.writeByte((byte)DataTransferProtocol.OP_READ_BLOCK);
     newBlockId = firstBlock.getBlockId()-1;
     sendOut.writeLong(newBlockId);
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(0L);
     sendOut.writeLong(fileLen);
-    recvOut.writeShort((short)FSConstants.OP_STATUS_ERROR);
+    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_ERROR);
     Text.writeString(sendOut, "cl");
     sendRecvData("Wrong block ID " + newBlockId + " for read", false); 
 
     // negative block start offset
     sendBuf.reset();
-    sendOut.writeShort((short)FSConstants.DATA_TRANSFER_VERSION);
-    sendOut.writeByte((byte)FSConstants.OP_READ_BLOCK);
+    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
+    sendOut.writeByte((byte)DataTransferProtocol.OP_READ_BLOCK);
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(-1L);
@@ -284,8 +285,8 @@
 
     // bad block start offset
     sendBuf.reset();
-    sendOut.writeShort((short)FSConstants.DATA_TRANSFER_VERSION);
-    sendOut.writeByte((byte)FSConstants.OP_READ_BLOCK);
+    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
+    sendOut.writeByte((byte)DataTransferProtocol.OP_READ_BLOCK);
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(fileLen);
@@ -296,10 +297,10 @@
     
     // negative length is ok. Datanode assumes we want to read the whole block.
     recvBuf.reset();
-    recvOut.writeShort((short)FSConstants.OP_STATUS_SUCCESS);    
+    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_SUCCESS);    
     sendBuf.reset();
-    sendOut.writeShort((short)FSConstants.DATA_TRANSFER_VERSION);
-    sendOut.writeByte((byte)FSConstants.OP_READ_BLOCK);
+    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
+    sendOut.writeByte((byte)DataTransferProtocol.OP_READ_BLOCK);
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(0);
@@ -310,10 +311,10 @@
     
     // length is more than size of block.
     recvBuf.reset();
-    recvOut.writeShort((short)FSConstants.OP_STATUS_ERROR);    
+    recvOut.writeShort((short)DataTransferProtocol.OP_STATUS_ERROR);    
     sendBuf.reset();
-    sendOut.writeShort((short)FSConstants.DATA_TRANSFER_VERSION);
-    sendOut.writeByte((byte)FSConstants.OP_READ_BLOCK);
+    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
+    sendOut.writeByte((byte)DataTransferProtocol.OP_READ_BLOCK);
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(0);
@@ -324,8 +325,8 @@
     
     //At the end of all this, read the file to make sure that succeeds finally.
     sendBuf.reset();
-    sendOut.writeShort((short)FSConstants.DATA_TRANSFER_VERSION);
-    sendOut.writeByte((byte)FSConstants.OP_READ_BLOCK);
+    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
+    sendOut.writeByte((byte)DataTransferProtocol.OP_READ_BLOCK);
     sendOut.writeLong(firstBlock.getBlockId());
     sendOut.writeLong(firstBlock.getGenerationStamp());
     sendOut.writeLong(0);

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/UpgradeUtilities.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/UpgradeUtilities.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/UpgradeUtilities.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/UpgradeUtilities.java Tue Sep 23 15:32:53 2008
@@ -34,10 +34,13 @@
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
-import org.apache.hadoop.hdfs.protocol.FSConstants.NodeType;
-import static org.apache.hadoop.hdfs.protocol.FSConstants.NodeType.NAME_NODE;
-import static org.apache.hadoop.hdfs.protocol.FSConstants.NodeType.DATA_NODE;
-import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
+
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java Tue Sep 23 15:32:53 2008
@@ -27,7 +27,7 @@
 
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.TestDFSUpgradeFromImage;
-import org.apache.hadoop.hdfs.protocol.FSConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.UpgradeObjectDatanode;
 import org.apache.hadoop.hdfs.server.namenode.UpgradeObjectNamenode;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java?rev=698377&r1=698376&r2=698377&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java Tue Sep 23 15:32:53 2008
@@ -38,10 +38,11 @@
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.datanode.BlockTransferThrottler;
 import org.apache.hadoop.io.Text;
@@ -220,12 +221,12 @@
       DatanodeInfo sourceProxy, DatanodeInfo destination) throws IOException {
     Socket sock = new Socket();
     sock.connect(NetUtils.createSocketAddr(
-        sourceProxy.getName()), FSConstants.READ_TIMEOUT);
-    sock.setSoTimeout(FSConstants.READ_TIMEOUT);
+        sourceProxy.getName()), HdfsConstants.READ_TIMEOUT);
+    sock.setSoTimeout(HdfsConstants.READ_TIMEOUT);
     // sendRequest
     DataOutputStream out = new DataOutputStream(sock.getOutputStream());
-    out.writeShort(FSConstants.DATA_TRANSFER_VERSION);
-    out.writeByte(FSConstants.OP_COPY_BLOCK);
+    out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
+    out.writeByte(DataTransferProtocol.OP_COPY_BLOCK);
     out.writeLong(block.getBlockId());
     out.writeLong(block.getGenerationStamp());
     Text.writeString(out, source.getStorageID());
@@ -235,7 +236,7 @@
     DataInputStream reply = new DataInputStream(sock.getInputStream());
 
     short status = reply.readShort();
-    if(status == FSConstants.OP_STATUS_SUCCESS) {
+    if(status == DataTransferProtocol.OP_STATUS_SUCCESS) {
       return true;
     }
     return false;