Skip to content
This repository has been archived by the owner on Feb 9, 2021. It is now read-only.

Commit

Permalink
HDFS-527. Remove unnecessary DFSClient constructors.
Browse files Browse the repository at this point in the history
  • Loading branch information
szetszwo committed Aug 17, 2009
1 parent b18b3ad commit 041c9a5
Show file tree
Hide file tree
Showing 7 changed files with 44 additions and 36 deletions.
2 changes: 2 additions & 0 deletions CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -212,6 +212,8 @@ Release 0.20.1 - Unreleased

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

Expand Down
65 changes: 34 additions & 31 deletions src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -67,8 +67,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
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();
Expand Down Expand Up @@ -142,44 +142,40 @@ static ClientDatanodeProtocol createClientDatanodeProtocolProxy (
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",
Expand Down Expand Up @@ -207,11 +203,18 @@ private DFSClient(Configuration conf, FileSystem.Statistics stats)
}
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 {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -317,7 +317,7 @@ private void check(FileStatus file, FsckResult res) throws IOException {

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);
Expand Down
3 changes: 2 additions & 1 deletion src/test/org/apache/hadoop/hdfs/TestAbandonBlock.java
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand All @@ -49,7 +50,7 @@ public void testAbandonBlock() throws IOException {
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 {
Expand Down
2 changes: 1 addition & 1 deletion src/test/org/apache/hadoop/hdfs/TestDFSClientRetries.java
Original file line number Diff line number Diff line change
Expand Up @@ -223,7 +223,7 @@ public void testNotYetReplicatedErrors() throws IOException
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

Expand Down
3 changes: 2 additions & 1 deletion src/test/org/apache/hadoop/hdfs/TestFileStatus.java
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -66,7 +67,7 @@ public void testFileStatus() throws IOException {
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 {

//
Expand Down
3 changes: 2 additions & 1 deletion src/test/org/apache/hadoop/hdfs/TestGetBlocks.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -75,7 +76,7 @@ public void testGetBlocks() throws Exception {
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());
Expand Down

0 comments on commit 041c9a5

Please sign in to comment.