Browse Source

HDFS-527. Remove/deprecate unnecessary DFSClient constructors.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@801480 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 15 years ago
parent
commit
a51727dd8a

+ 2 - 0
CHANGES.txt

@@ -81,6 +81,8 @@ Trunk (unreleased changes)
 
 
     HDFS-524. Further DataTransferProtocol code refactoring.  (szetszwo)
     HDFS-524. Further DataTransferProtocol code refactoring.  (szetszwo)
 
 
+    HDFS-527. Remove/deprecate unnecessary DFSClient constructors.  (szetszwo)
+
   BUG FIXES
   BUG FIXES
     HDFS-76. Better error message to users when commands fail because of 
     HDFS-76. Better error message to users when commands fail because of 
     lack of quota. Allow quota to be set even if the limit is lower than
     lack of quota. Allow quota to be set even if the limit is lower than

+ 36 - 29
src/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -127,8 +127,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   public static final Log LOG = LogFactory.getLog(DFSClient.class);
   public static final Log LOG = LogFactory.getLog(DFSClient.class);
   public static final int MAX_BLOCK_ACQUIRE_FAILURES = 3;
   public static final int MAX_BLOCK_ACQUIRE_FAILURES = 3;
   private static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
   private static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
-  private ClientProtocol namenode;
-  private ClientProtocol rpcNamenode;
+  private final ClientProtocol namenode;
+  private final ClientProtocol rpcNamenode;
   final UnixUserGroupInformation ugi;
   final UnixUserGroupInformation ugi;
   volatile boolean clientRunning = true;
   volatile boolean clientRunning = true;
   Random r = new Random();
   Random r = new Random();
@@ -206,41 +206,41 @@ public class DFSClient implements FSConstants, java.io.Closeable {
         ClientDatanodeProtocol.versionID, addr, conf);
         ClientDatanodeProtocol.versionID, addr, conf);
   }
   }
         
         
-  /** 
-   * Create a new DFSClient connected to the default namenode.
+  /**
+   * Same as this(NameNode.getAddress(conf), conf);
+   * @see #DFSClient(InetSocketAddress, Configuration)
+   * @deprecated Deprecated at 0.21
    */
    */
+  @Deprecated
   public DFSClient(Configuration conf) throws IOException {
   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,
   public DFSClient(InetSocketAddress nameNodeAddr, Configuration conf,
                    FileSystem.Statistics stats)
                    FileSystem.Statistics stats)
     throws IOException {
     throws IOException {
-    this(conf, stats);
-    this.rpcNamenode = createRPCNamenode(nameNodeAddr, conf, ugi);
-    this.namenode = createNamenode(this.rpcNamenode);
+    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)
-    throws IOException {
-      this(conf, stats);
-      this.namenode = namenode;
-      this.rpcNamenode = rpcNamenode;
-  }
-
-  
-  private DFSClient(Configuration conf, FileSystem.Statistics stats)
+  DFSClient(InetSocketAddress nameNodeAddr, ClientProtocol rpcNamenode,
+      Configuration conf, FileSystem.Statistics stats)
     throws IOException {
     throws IOException {
     this.conf = conf;
     this.conf = conf;
     this.stats = stats;
     this.stats = stats;
@@ -271,11 +271,18 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     }
     }
     defaultBlockSize = conf.getLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
     defaultBlockSize = conf.getLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
     defaultReplication = (short) conf.getInt("dfs.replication", 3);
     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 {
   private void checkOpen() throws IOException {

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

@@ -324,7 +324,7 @@ public class NamenodeFsck {
   
   
   private void lostFoundMove(FileStatus file, LocatedBlocks blocks)
   private void lostFoundMove(FileStatus file, LocatedBlocks blocks)
     throws IOException {
     throws IOException {
-    DFSClient dfs = new DFSClient(conf);
+    final DFSClient dfs = new DFSClient(NameNode.getAddress(conf), conf);
     try {
     try {
     if (!lfInited) {
     if (!lfInited) {
       lostFoundInit(dfs);
       lostFoundInit(dfs);

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

@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 
 
 public class TestAbandonBlock extends junit.framework.TestCase {
 public class TestAbandonBlock extends junit.framework.TestCase {
@@ -49,7 +50,7 @@ public class TestAbandonBlock extends junit.framework.TestCase {
       fout.sync();
       fout.sync();
   
   
       //try reading the block by someone
       //try reading the block by someone
-      DFSClient dfsclient = new DFSClient(CONF);
+      final DFSClient dfsclient = new DFSClient(NameNode.getAddress(CONF), CONF);
       LocatedBlocks blocks = dfsclient.getNamenode().getBlockLocations(src, 0, 1);
       LocatedBlocks blocks = dfsclient.getNamenode().getBlockLocations(src, 0, 1);
       LocatedBlock b = blocks.get(0); 
       LocatedBlock b = blocks.get(0); 
       try {
       try {

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

@@ -223,7 +223,7 @@ public class TestDFSClientRetries extends TestCase {
     conf.setInt("dfs.client.block.write.locateFollowingBlock.retries", 1);
     conf.setInt("dfs.client.block.write.locateFollowingBlock.retries", 1);
         
         
     TestNameNode tnn = new TestNameNode(conf);
     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);
     OutputStream os = client.create("testfile", true);
     os.write(20); // write one random byte
     os.write(20); // write one random byte
     
     

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

@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 
 
 /**
 /**
@@ -63,7 +64,7 @@ public class TestFileStatus extends TestCase {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
     FileSystem fs = cluster.getFileSystem();
     FileSystem fs = cluster.getFileSystem();
-    DFSClient dfsClient = new DFSClient(conf);
+    final DFSClient dfsClient = new DFSClient(NameNode.getAddress(conf), conf);
     try {
     try {
 
 
       //
       //
@@ -83,7 +84,7 @@ public class TestFileStatus extends TestCase {
       // make sure getFileInfo throws the appropriate exception for non-relative
       // make sure getFileInfo throws the appropriate exception for non-relative
       // filenames
       // filenames
       try {
       try {
-        FileStatus foo = dfsClient.getFileInfo("non-relative");
+        dfsClient.getFileInfo("non-relative");
         fail("getFileInfo for a non-relative path did not thro IOException");
         fail("getFileInfo for a non-relative path did not thro IOException");
       } catch (RemoteException re) {
       } catch (RemoteException re) {
         assertTrue(re.toString().contains("Invalid file name"));
         assertTrue(re.toString().contains("Invalid file name"));

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

@@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 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.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -75,7 +76,7 @@ public class TestGetBlocks extends TestCase {
       DatanodeInfo[] dataNodes=null;
       DatanodeInfo[] dataNodes=null;
       boolean notWritten;
       boolean notWritten;
       do {
       do {
-        DFSClient dfsclient = new DFSClient(CONF);
+        final DFSClient dfsclient = new DFSClient(NameNode.getAddress(CONF), CONF);
         locatedBlocks = dfsclient.getNamenode().
         locatedBlocks = dfsclient.getNamenode().
           getBlockLocations("/tmp.txt", 0, fileLen).getLocatedBlocks();
           getBlockLocations("/tmp.txt", 0, fileLen).getLocatedBlocks();
         assertEquals(2, locatedBlocks.size());
         assertEquals(2, locatedBlocks.size());