|
@@ -95,6 +95,7 @@ import org.apache.hadoop.fs.permission.AclStatus;
|
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
|
|
|
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
|
|
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
|
|
@@ -284,19 +285,20 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|
|
|
|
|
/**
|
|
|
* Create a new DFSClient connected to the given nameNodeUri or rpcNamenode.
|
|
|
- * If HA is enabled and a positive value is set for
|
|
|
- * {@link DFSConfigKeys#DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY} in the
|
|
|
- * configuration, the DFSClient will use {@link LossyRetryInvocationHandler}
|
|
|
- * as its RetryInvocationHandler. Otherwise one of nameNodeUri or rpcNamenode
|
|
|
- * must be null.
|
|
|
+ * If HA is enabled and a positive value is set for
|
|
|
+ * {@link HdfsClientConfigKeys#DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY}
|
|
|
+ * in the configuration, the DFSClient will use
|
|
|
+ * {@link LossyRetryInvocationHandler} as its RetryInvocationHandler.
|
|
|
+ * Otherwise one of nameNodeUri or rpcNamenode must be null.
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
|
|
|
Configuration conf, FileSystem.Statistics stats)
|
|
|
throws IOException {
|
|
|
- SpanReceiverHost.get(conf, DFSConfigKeys.DFS_CLIENT_HTRACE_PREFIX);
|
|
|
+ SpanReceiverHost.get(conf, HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX);
|
|
|
traceSampler = new SamplerBuilder(TraceUtils.
|
|
|
- wrapHadoopConf(DFSConfigKeys.DFS_CLIENT_HTRACE_PREFIX, conf)).build();
|
|
|
+ wrapHadoopConf(HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX, conf))
|
|
|
+ .build();
|
|
|
// Copy only the required DFSClient configuration
|
|
|
this.dfsClientConf = new DfsClientConf(conf);
|
|
|
this.conf = conf;
|
|
@@ -312,13 +314,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|
|
ThreadLocalRandom.current().nextInt() + "_" +
|
|
|
Thread.currentThread().getId();
|
|
|
int numResponseToDrop = conf.getInt(
|
|
|
- DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
|
|
|
- DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
|
|
|
+ HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
|
|
|
+ HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
|
|
|
ProxyAndInfo<ClientProtocol> proxyInfo = null;
|
|
|
AtomicBoolean nnFallbackToSimpleAuth = new AtomicBoolean(false);
|
|
|
if (numResponseToDrop > 0) {
|
|
|
// This case is used for testing.
|
|
|
- LOG.warn(DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
|
|
|
+ LOG.warn(HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
|
|
|
+ " is set to " + numResponseToDrop
|
|
|
+ ", this hacked client will proactively drop responses");
|
|
|
proxyInfo = NameNodeProxiesClient.createProxyWithLossyRetryHandler(conf,
|
|
@@ -344,7 +346,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|
|
}
|
|
|
|
|
|
String localInterfaces[] =
|
|
|
- conf.getTrimmedStrings(DFSConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
|
|
|
+ conf.getTrimmedStrings(HdfsClientConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
|
|
|
localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
|
|
|
if (LOG.isDebugEnabled() && 0 != localInterfaces.length) {
|
|
|
LOG.debug("Using local interfaces [" +
|