|
@@ -128,17 +128,85 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
static Random r = new Random();
|
|
|
final String clientName;
|
|
|
Configuration conf;
|
|
|
- long defaultBlockSize;
|
|
|
- private short defaultReplication;
|
|
|
SocketFactory socketFactory;
|
|
|
- int socketTimeout;
|
|
|
- final int writePacketSize;
|
|
|
final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
|
|
|
final FileSystem.Statistics stats;
|
|
|
final int hdfsTimeout; // timeout value for a DFS operation.
|
|
|
final LeaseRenewer leaserenewer;
|
|
|
-
|
|
|
final SocketCache socketCache;
|
|
|
+ final Conf dfsClientConf;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * DFSClient configuration
|
|
|
+ */
|
|
|
+ static class Conf {
|
|
|
+ final int maxBlockAcquireFailures;
|
|
|
+ final int confTime;
|
|
|
+ final int ioBufferSize;
|
|
|
+ final int bytesPerChecksum;
|
|
|
+ final int writePacketSize;
|
|
|
+ final int socketTimeout;
|
|
|
+ final int socketCacheCapacity;
|
|
|
+ /** Wait time window (in msec) if BlockMissingException is caught */
|
|
|
+ final int timeWindow;
|
|
|
+ final int nCachedConnRetry;
|
|
|
+ final int nBlockWriteRetry;
|
|
|
+ final int nBlockWriteLocateFollowingRetry;
|
|
|
+ final long defaultBlockSize;
|
|
|
+ final long prefetchSize;
|
|
|
+ final short defaultReplication;
|
|
|
+ final String taskId;
|
|
|
+
|
|
|
+ Conf(Configuration conf) {
|
|
|
+ maxBlockAcquireFailures = conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
|
|
|
+ DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
|
|
|
+ confTime = conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
|
|
|
+ HdfsConstants.WRITE_TIMEOUT);
|
|
|
+ ioBufferSize = conf.getInt(
|
|
|
+ CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
|
|
|
+ CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
|
|
|
+ bytesPerChecksum = conf.getInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY,
|
|
|
+ DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT);
|
|
|
+ socketTimeout = conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY,
|
|
|
+ HdfsConstants.READ_TIMEOUT);
|
|
|
+ /** dfs.write.packet.size is an internal config variable */
|
|
|
+ writePacketSize = conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
|
|
|
+ DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
|
|
|
+ defaultBlockSize = conf.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
|
|
|
+ DEFAULT_BLOCK_SIZE);
|
|
|
+ defaultReplication = (short) conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_REPLICATION_KEY,
|
|
|
+ DFSConfigKeys.DFS_REPLICATION_DEFAULT);
|
|
|
+ taskId = conf.get("mapreduce.task.attempt.id", "NONMAPREDUCE");
|
|
|
+ socketCacheCapacity = conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY,
|
|
|
+ DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT);
|
|
|
+ prefetchSize = conf.getLong(
|
|
|
+ DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY,
|
|
|
+ 10 * defaultBlockSize);
|
|
|
+ timeWindow = conf
|
|
|
+ .getInt(DFSConfigKeys.DFS_CLIENT_RETRY_WINDOW_BASE, 3000);
|
|
|
+ nCachedConnRetry = conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_KEY,
|
|
|
+ DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT);
|
|
|
+ nBlockWriteRetry = conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY,
|
|
|
+ DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT);
|
|
|
+ nBlockWriteLocateFollowingRetry = conf
|
|
|
+ .getInt(
|
|
|
+ DFSConfigKeys
|
|
|
+ .DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY,
|
|
|
+ DFSConfigKeys
|
|
|
+ .DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ Conf getConf() {
|
|
|
+ return dfsClientConf;
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* A map from file names to {@link DFSOutputStream} objects
|
|
@@ -257,16 +325,11 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
DFSClient(InetSocketAddress nameNodeAddr, ClientProtocol rpcNamenode,
|
|
|
Configuration conf, FileSystem.Statistics stats)
|
|
|
throws IOException {
|
|
|
+ // Copy only the required DFSClient configuration
|
|
|
+ this.dfsClientConf = new Conf(conf);
|
|
|
this.conf = conf;
|
|
|
this.stats = stats;
|
|
|
- this.socketTimeout =
|
|
|
- conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY,
|
|
|
- HdfsConstants.READ_TIMEOUT);
|
|
|
this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
|
|
|
- // dfs.write.packet.size is an internal config variable
|
|
|
- this.writePacketSize =
|
|
|
- conf.getInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
|
|
|
- DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
|
|
|
this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
|
|
|
|
|
|
// The hdfsTimeout is currently the same as the ipc timeout
|
|
@@ -275,19 +338,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
final String authority = nameNodeAddr == null? "null":
|
|
|
nameNodeAddr.getHostName() + ":" + nameNodeAddr.getPort();
|
|
|
this.leaserenewer = LeaseRenewer.getInstance(authority, ugi, this);
|
|
|
-
|
|
|
- String taskId = conf.get("mapreduce.task.attempt.id", "NONMAPREDUCE");
|
|
|
- this.clientName = leaserenewer.getClientName(taskId);
|
|
|
-
|
|
|
- defaultBlockSize = conf.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
|
|
|
- defaultReplication = (short)
|
|
|
- conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
|
|
|
- DFSConfigKeys.DFS_REPLICATION_DEFAULT);
|
|
|
-
|
|
|
- this.socketCache = new SocketCache(
|
|
|
- conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY,
|
|
|
- DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT));
|
|
|
-
|
|
|
+ this.clientName = leaserenewer.getClientName(dfsClientConf.taskId);
|
|
|
+ this.socketCache = new SocketCache(dfsClientConf.socketCacheCapacity);
|
|
|
if (nameNodeAddr != null && rpcNamenode == null) {
|
|
|
this.rpcNamenode = createRPCNamenode(nameNodeAddr, conf, ugi);
|
|
|
this.namenode = createNamenode(this.rpcNamenode);
|
|
@@ -306,8 +358,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
* to retrieve block locations when reading.
|
|
|
*/
|
|
|
int getMaxBlockAcquireFailures() {
|
|
|
- return conf.getInt(DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
|
|
|
- DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
|
|
|
+ return dfsClientConf.maxBlockAcquireFailures;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -315,18 +366,14 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
* @param numNodes the number of nodes in the pipeline.
|
|
|
*/
|
|
|
int getDatanodeWriteTimeout(int numNodes) {
|
|
|
- int confTime =
|
|
|
- conf.getInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
|
|
|
- HdfsConstants.WRITE_TIMEOUT);
|
|
|
-
|
|
|
- return (confTime > 0) ?
|
|
|
- (confTime + HdfsConstants.WRITE_TIMEOUT_EXTENSION * numNodes) : 0;
|
|
|
+ return (dfsClientConf.confTime > 0) ?
|
|
|
+ (dfsClientConf.confTime + HdfsConstants.WRITE_TIMEOUT_EXTENSION * numNodes) : 0;
|
|
|
}
|
|
|
|
|
|
int getDatanodeReadTimeout(int numNodes) {
|
|
|
- return socketTimeout > 0 ?
|
|
|
+ return dfsClientConf.socketTimeout > 0 ?
|
|
|
(HdfsConstants.READ_TIMEOUT_EXTENSION * numNodes +
|
|
|
- socketTimeout) : 0;
|
|
|
+ dfsClientConf.socketTimeout) : 0;
|
|
|
}
|
|
|
|
|
|
int getHdfsTimeout() {
|
|
@@ -430,7 +477,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
* @return the default block size in bytes
|
|
|
*/
|
|
|
public long getDefaultBlockSize() {
|
|
|
- return defaultBlockSize;
|
|
|
+ return dfsClientConf.defaultBlockSize;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -528,7 +575,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
}
|
|
|
|
|
|
public short getDefaultReplication() {
|
|
|
- return defaultReplication;
|
|
|
+ return dfsClientConf.defaultReplication;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -583,7 +630,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
|
|
|
public DFSInputStream open(String src)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
- return open(src, conf.getInt("io.file.buffer.size", 4096), true, null);
|
|
|
+ return open(src, dfsClientConf.ioBufferSize, true, null);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -629,7 +676,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
*/
|
|
|
public OutputStream create(String src, boolean overwrite)
|
|
|
throws IOException {
|
|
|
- return create(src, overwrite, defaultReplication, defaultBlockSize, null);
|
|
|
+ return create(src, overwrite, dfsClientConf.defaultReplication,
|
|
|
+ dfsClientConf.defaultBlockSize, null);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -639,7 +687,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
public OutputStream create(String src,
|
|
|
boolean overwrite,
|
|
|
Progressable progress) throws IOException {
|
|
|
- return create(src, overwrite, defaultReplication, defaultBlockSize, progress);
|
|
|
+ return create(src, overwrite, dfsClientConf.defaultReplication,
|
|
|
+ dfsClientConf.defaultBlockSize, progress);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -660,7 +709,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
public OutputStream create(String src, boolean overwrite, short replication,
|
|
|
long blockSize, Progressable progress) throws IOException {
|
|
|
return create(src, overwrite, replication, blockSize, progress,
|
|
|
- conf.getInt("io.file.buffer.size", 4096));
|
|
|
+ dfsClientConf.ioBufferSize);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -744,10 +793,9 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug(src + ": masked=" + masked);
|
|
|
}
|
|
|
- final DFSOutputStream result = new DFSOutputStream(this, src, masked,
|
|
|
- flag, createParent, replication, blockSize, progress, buffersize,
|
|
|
- conf.getInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY,
|
|
|
- DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT));
|
|
|
+ final DFSOutputStream result = new DFSOutputStream(this, src, masked, flag,
|
|
|
+ createParent, replication, blockSize, progress, buffersize,
|
|
|
+ dfsClientConf.bytesPerChecksum);
|
|
|
leaserenewer.put(src, result, this);
|
|
|
return result;
|
|
|
}
|
|
@@ -851,8 +899,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
UnresolvedPathException.class);
|
|
|
}
|
|
|
return new DFSOutputStream(this, src, buffersize, progress,
|
|
|
- lastBlock, stat, conf.getInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY,
|
|
|
- DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT));
|
|
|
+ lastBlock, stat, dfsClientConf.bytesPerChecksum);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1061,7 +1108,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
*/
|
|
|
public MD5MD5CRC32FileChecksum getFileChecksum(String src) throws IOException {
|
|
|
checkOpen();
|
|
|
- return getFileChecksum(src, namenode, socketFactory, socketTimeout);
|
|
|
+ return getFileChecksum(src, namenode, socketFactory, dfsClientConf.socketTimeout);
|
|
|
}
|
|
|
|
|
|
/**
|