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 sz...@apache.org on 2009/08/18 00:46:33 UTC

svn commit: r805195 - in /hadoop/common/branches/branch-0.20: ./ src/hdfs/org/apache/hadoop/hdfs/ src/hdfs/org/apache/hadoop/hdfs/server/namenode/ src/test/org/apache/hadoop/hdfs/

Author: szetszwo
Date: Mon Aug 17 22:46:33 2009
New Revision: 805195

URL: http://svn.apache.org/viewvc?rev=805195&view=rev
Log:
HDFS-527. Remove unnecessary DFSClient constructors.

Modified:
    hadoop/common/branches/branch-0.20/CHANGES.txt
    hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
    hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
    hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestAbandonBlock.java
    hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestDFSClientRetries.java
    hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestFileStatus.java
    hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestGetBlocks.java

Modified: hadoop/common/branches/branch-0.20/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/CHANGES.txt?rev=805195&r1=805194&r2=805195&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.20/CHANGES.txt Mon Aug 17 22:46:33 2009
@@ -212,6 +212,8 @@
 
     HDFS-167. Fix a bug in DFSClient that caused infinite retries on write.
     (Bill Zeller via szetszwo)
+
+    HDFS-527. Remove unnecessary DFSClient constructors.  (szetszwo)
  
 Release 0.20.0 - 2009-04-15
 

Modified: hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java?rev=805195&r1=805194&r2=805195&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java (original)
+++ hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/DFSClient.java Mon Aug 17 22:46:33 2009
@@ -67,8 +67,8 @@
   public static final Log LOG = LogFactory.getLog(DFSClient.class);
   public static final int MAX_BLOCK_ACQUIRE_FAILURES = 3;
   private static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
-  public ClientProtocol namenode;
-  private ClientProtocol rpcNamenode;
+  public final ClientProtocol namenode;
+  private final ClientProtocol rpcNamenode;
   final UnixUserGroupInformation ugi;
   volatile boolean clientRunning = true;
   Random r = new Random();
@@ -142,44 +142,40 @@
         ClientDatanodeProtocol.versionID, addr, conf);
   }
         
-  /** 
-   * Create a new DFSClient connected to the default namenode.
+  /**
+   * Same as this(NameNode.getAddress(conf), conf);
+   * @see #DFSClient(InetSocketAddress, Configuration)
    */
   public DFSClient(Configuration conf) throws IOException {
-    this(NameNode.getAddress(conf), conf, null);
+    this(NameNode.getAddress(conf), conf);
   }
 
-  /** 
-   * Create a new DFSClient connected to the given namenode server.
+  /**
+   * Same as this(nameNodeAddr, conf, null);
+   * @see #DFSClient(InetSocketAddress, Configuration, org.apache.hadoop.fs.FileSystem.Statistics)
+   */
+  public DFSClient(InetSocketAddress nameNodeAddr, Configuration conf
+      ) throws IOException {
+    this(nameNodeAddr, conf, null);
+  }
+
+  /**
+   * Same as this(nameNodeAddr, null, conf, stats);
+   * @see #DFSClient(InetSocketAddress, ClientProtocol, Configuration, org.apache.hadoop.fs.FileSystem.Statistics) 
    */
   public DFSClient(InetSocketAddress nameNodeAddr, Configuration conf,
                    FileSystem.Statistics stats)
     throws IOException {
-    this(conf, stats);
-    this.rpcNamenode = createRPCNamenode(nameNodeAddr, conf, ugi);
-    this.namenode = createNamenode(this.rpcNamenode);
+    this(nameNodeAddr, null, conf, stats);
   }
 
   /** 
-   * 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.
+   * Create a new DFSClient connected to the given nameNodeAddr or rpcNamenode.
+   * Exactly one of nameNodeAddr or rpcNamenode must be null.
    */
-  public DFSClient(ClientProtocol namenode, ClientProtocol rpcNamenode,
-                   Configuration conf, FileSystem.Statistics stats)
+  DFSClient(InetSocketAddress nameNodeAddr, 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.stats = stats;
     this.socketTimeout = conf.getInt("dfs.socket.timeout", 
@@ -207,11 +203,18 @@
     }
     defaultBlockSize = conf.getLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
     defaultReplication = (short) conf.getInt("dfs.replication", 3);
-  }
 
-  public DFSClient(InetSocketAddress nameNodeAddr, 
-                   Configuration conf) throws IOException {
-    this(nameNodeAddr, conf, null);
+    if (nameNodeAddr != null && rpcNamenode == null) {
+      this.rpcNamenode = createRPCNamenode(nameNodeAddr, conf, ugi);
+      this.namenode = createNamenode(this.rpcNamenode);
+    } else if (nameNodeAddr == null && rpcNamenode != null) {
+      //This case is used for testing.
+      this.namenode = this.rpcNamenode = rpcNamenode;
+    } else {
+      throw new IllegalArgumentException(
+          "Expecting exactly one of nameNodeAddr and rpcNamenode being null: "
+          + "nameNodeAddr=" + nameNodeAddr + ", rpcNamenode=" + rpcNamenode);
+    }
   }
 
   private void checkOpen() throws IOException {

Modified: hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java?rev=805195&r1=805194&r2=805195&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java (original)
+++ hadoop/common/branches/branch-0.20/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java Mon Aug 17 22:46:33 2009
@@ -317,7 +317,7 @@
   
   private void lostFoundMove(FileStatus file, LocatedBlocks blocks)
     throws IOException {
-    DFSClient dfs = new DFSClient(conf);
+    final DFSClient dfs = new DFSClient(NameNode.getAddress(conf), conf);
     try {
     if (!lfInited) {
       lostFoundInit(dfs);

Modified: hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestAbandonBlock.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestAbandonBlock.java?rev=805195&r1=805194&r2=805195&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestAbandonBlock.java (original)
+++ hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestAbandonBlock.java Mon Aug 17 22:46:33 2009
@@ -25,6 +25,7 @@
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.util.StringUtils;
 
 public class TestAbandonBlock extends junit.framework.TestCase {
@@ -49,7 +50,7 @@
       fout.sync();
   
       //try reading the block by someone
-      DFSClient dfsclient = new DFSClient(CONF);
+      final DFSClient dfsclient = new DFSClient(NameNode.getAddress(CONF), CONF);
       LocatedBlocks blocks = dfsclient.namenode.getBlockLocations(src, 0, 1);
       LocatedBlock b = blocks.get(0); 
       try {

Modified: hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestDFSClientRetries.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestDFSClientRetries.java?rev=805195&r1=805194&r2=805195&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestDFSClientRetries.java (original)
+++ hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestDFSClientRetries.java Mon Aug 17 22:46:33 2009
@@ -223,7 +223,7 @@
     conf.setInt("dfs.client.block.write.locateFollowingBlock.retries", 1);
         
     TestNameNode tnn = new TestNameNode(conf);
-    DFSClient client = new DFSClient(tnn, tnn, conf, null);
+    final DFSClient client = new DFSClient(null, tnn, conf, null);
     OutputStream os = client.create("testfile", true);
     os.write(20); // write one random byte
     

Modified: hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestFileStatus.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestFileStatus.java?rev=805195&r1=805194&r2=805195&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestFileStatus.java (original)
+++ hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestFileStatus.java Mon Aug 17 22:46:33 2009
@@ -26,6 +26,7 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 
 /**
  * This class tests the FileStatus API.
@@ -66,7 +67,7 @@
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
     FileSystem fs = cluster.getFileSystem();
-    DFSClient dfsClient = new DFSClient(conf);
+    final DFSClient dfsClient = new DFSClient(NameNode.getAddress(conf), conf);
     try {
 
       //

Modified: hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestGetBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestGetBlocks.java?rev=805195&r1=805194&r2=805195&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestGetBlocks.java (original)
+++ hadoop/common/branches/branch-0.20/src/test/org/apache/hadoop/hdfs/TestGetBlocks.java Mon Aug 17 22:46:33 2009
@@ -26,6 +26,7 @@
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -75,7 +76,7 @@
       DatanodeInfo[] dataNodes=null;
       boolean notWritten;
       do {
-        DFSClient dfsclient = new DFSClient(CONF);
+        final DFSClient dfsclient = new DFSClient(NameNode.getAddress(CONF), CONF);
         locatedBlocks = dfsclient.namenode.
           getBlockLocations("/tmp.txt", 0, fileLen).getLocatedBlocks();
         assertEquals(2, locatedBlocks.size());