|
@@ -99,6 +99,7 @@ import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
|
|
import org.apache.hadoop.fs.Options;
|
|
import org.apache.hadoop.fs.Options;
|
|
import org.apache.hadoop.fs.Options.ChecksumOpt;
|
|
import org.apache.hadoop.fs.Options.ChecksumOpt;
|
|
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
|
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
import org.apache.hadoop.fs.VolumeId;
|
|
import org.apache.hadoop.fs.VolumeId;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
@@ -112,13 +113,13 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
-import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
|
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
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.protocol.NSQuotaExceededException;
|
|
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
|
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
|
|
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
|
|
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
|
|
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
|
|
@@ -142,6 +143,7 @@ import org.apache.hadoop.io.EnumSetWritable;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.io.MD5Hash;
|
|
import org.apache.hadoop.io.MD5Hash;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.Text;
|
|
|
|
+import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
|
|
import org.apache.hadoop.ipc.Client;
|
|
import org.apache.hadoop.ipc.Client;
|
|
import org.apache.hadoop.ipc.RPC;
|
|
import org.apache.hadoop.ipc.RPC;
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
@@ -447,7 +449,11 @@ public class DFSClient implements java.io.Closeable {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Create a new DFSClient connected to the given nameNodeUri or rpcNamenode.
|
|
* Create a new DFSClient connected to the given nameNodeUri or rpcNamenode.
|
|
- * Exactly one of nameNodeUri or rpcNamenode must be null.
|
|
|
|
|
|
+ * 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.
|
|
*/
|
|
*/
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
|
|
public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
|
|
@@ -471,7 +477,20 @@ public class DFSClient implements java.io.Closeable {
|
|
this.clientName = "DFSClient_" + dfsClientConf.taskId + "_" +
|
|
this.clientName = "DFSClient_" + dfsClientConf.taskId + "_" +
|
|
DFSUtil.getRandom().nextInt() + "_" + Thread.currentThread().getId();
|
|
DFSUtil.getRandom().nextInt() + "_" + Thread.currentThread().getId();
|
|
|
|
|
|
- if (rpcNamenode != null) {
|
|
|
|
|
|
+ int numResponseToDrop = conf.getInt(
|
|
|
|
+ DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
|
|
|
|
+ DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
|
|
|
|
+ if (numResponseToDrop > 0) {
|
|
|
|
+ // This case is used for testing.
|
|
|
|
+ LOG.warn(DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
|
|
|
|
+ + " is set to " + numResponseToDrop
|
|
|
|
+ + ", this hacked client will proactively drop responses");
|
|
|
|
+ NameNodeProxies.ProxyAndInfo<ClientProtocol> proxyInfo = NameNodeProxies
|
|
|
|
+ .createProxyWithLossyRetryHandler(conf, nameNodeUri,
|
|
|
|
+ ClientProtocol.class, numResponseToDrop);
|
|
|
|
+ this.dtService = proxyInfo.getDelegationTokenService();
|
|
|
|
+ this.namenode = proxyInfo.getProxy();
|
|
|
|
+ } else if (rpcNamenode != null) {
|
|
// This case is used for testing.
|
|
// This case is used for testing.
|
|
Preconditions.checkArgument(nameNodeUri == null);
|
|
Preconditions.checkArgument(nameNodeUri == null);
|
|
this.namenode = rpcNamenode;
|
|
this.namenode = rpcNamenode;
|
|
@@ -500,7 +519,7 @@ public class DFSClient implements java.io.Closeable {
|
|
|
|
|
|
this.peerCache = PeerCache.getInstance(dfsClientConf.socketCacheCapacity, dfsClientConf.socketCacheExpiry);
|
|
this.peerCache = PeerCache.getInstance(dfsClientConf.socketCacheCapacity, dfsClientConf.socketCacheExpiry);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Return the socket addresses to use with each configured
|
|
* Return the socket addresses to use with each configured
|
|
* local interface. Local interfaces may be specified by IP
|
|
* local interface. Local interfaces may be specified by IP
|