|
@@ -27,12 +27,9 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_LOCA
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT;
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT;
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY;
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY;
|
|
|
|
|
|
-import java.io.BufferedOutputStream;
|
|
|
|
-import java.io.DataInputStream;
|
|
|
|
import java.io.DataOutputStream;
|
|
import java.io.DataOutputStream;
|
|
import java.io.FileNotFoundException;
|
|
import java.io.FileNotFoundException;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
-import java.io.InputStream;
|
|
|
|
import java.io.OutputStream;
|
|
import java.io.OutputStream;
|
|
import java.net.InetAddress;
|
|
import java.net.InetAddress;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
@@ -80,9 +77,7 @@ import org.apache.hadoop.fs.FsStatus;
|
|
import org.apache.hadoop.fs.FsTracer;
|
|
import org.apache.hadoop.fs.FsTracer;
|
|
import org.apache.hadoop.fs.HdfsBlockLocation;
|
|
import org.apache.hadoop.fs.HdfsBlockLocation;
|
|
import org.apache.hadoop.fs.InvalidPathException;
|
|
import org.apache.hadoop.fs.InvalidPathException;
|
|
-import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
|
|
|
|
import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
|
|
import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
|
|
-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;
|
|
@@ -138,7 +133,6 @@ import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
|
|
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
|
|
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
|
-import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
|
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
|
|
@@ -146,20 +140,16 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactor
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
|
|
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
|
|
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
|
|
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
|
|
|
|
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
|
|
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
|
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
|
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
|
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
|
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
|
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
|
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
|
|
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
|
|
import org.apache.hadoop.hdfs.util.IOUtilsClient;
|
|
import org.apache.hadoop.hdfs.util.IOUtilsClient;
|
|
-import org.apache.hadoop.io.DataOutputBuffer;
|
|
|
|
import org.apache.hadoop.io.EnumSetWritable;
|
|
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.Text;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
|
|
import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
|
|
import org.apache.hadoop.ipc.RPC;
|
|
import org.apache.hadoop.ipc.RPC;
|
|
@@ -1293,7 +1283,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|
|
|
|
|
/**
|
|
/**
|
|
* Invoke namenode append RPC.
|
|
* Invoke namenode append RPC.
|
|
- * It retries in case of {@link BlockNotYetCompleteException}.
|
|
|
|
|
|
+ * It retries in case of some {@link RetriableException}.
|
|
*/
|
|
*/
|
|
private LastBlockWithStatus callAppend(String src,
|
|
private LastBlockWithStatus callAppend(String src,
|
|
EnumSetWritable<CreateFlag> flag) throws IOException {
|
|
EnumSetWritable<CreateFlag> flag) throws IOException {
|
|
@@ -1695,7 +1685,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Get the checksum of the whole file of a range of the file. Note that the
|
|
|
|
|
|
+ * Get the checksum of the whole file or a range of the file. Note that the
|
|
* range always starts from the beginning of the file.
|
|
* range always starts from the beginning of the file.
|
|
* @param src The file path
|
|
* @param src The file path
|
|
* @param length the length of the range, i.e., the range is [0, length]
|
|
* @param length the length of the range, i.e., the range is [0, length]
|
|
@@ -1706,9 +1696,23 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|
throws IOException {
|
|
throws IOException {
|
|
checkOpen();
|
|
checkOpen();
|
|
Preconditions.checkArgument(length >= 0);
|
|
Preconditions.checkArgument(length >= 0);
|
|
|
|
+
|
|
|
|
+ LocatedBlocks blockLocations = getBlockLocations(src, length);
|
|
|
|
+
|
|
|
|
+ FileChecksumHelper.FileChecksumComputer maker =
|
|
|
|
+ new FileChecksumHelper.ReplicatedFileChecksumComputer(src, length,
|
|
|
|
+ blockLocations, namenode, this);
|
|
|
|
+
|
|
|
|
+ maker.compute();
|
|
|
|
+
|
|
|
|
+ return maker.getFileChecksum();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected LocatedBlocks getBlockLocations(String src,
|
|
|
|
+ long length) throws IOException {
|
|
//get block locations for the file range
|
|
//get block locations for the file range
|
|
- LocatedBlocks blockLocations = callGetBlockLocations(namenode, src, 0,
|
|
|
|
- length);
|
|
|
|
|
|
+ LocatedBlocks blockLocations = callGetBlockLocations(namenode,
|
|
|
|
+ src, 0, length);
|
|
if (null == blockLocations) {
|
|
if (null == blockLocations) {
|
|
throw new FileNotFoundException("File does not exist: " + src);
|
|
throw new FileNotFoundException("File does not exist: " + src);
|
|
}
|
|
}
|
|
@@ -1716,194 +1720,15 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|
throw new IOException("Fail to get checksum, since file " + src
|
|
throw new IOException("Fail to get checksum, since file " + src
|
|
+ " is under construction.");
|
|
+ " is under construction.");
|
|
}
|
|
}
|
|
- List<LocatedBlock> locatedblocks = blockLocations.getLocatedBlocks();
|
|
|
|
- final DataOutputBuffer md5out = new DataOutputBuffer();
|
|
|
|
- int bytesPerCRC = -1;
|
|
|
|
- DataChecksum.Type crcType = DataChecksum.Type.DEFAULT;
|
|
|
|
- long crcPerBlock = 0;
|
|
|
|
- boolean refetchBlocks = false;
|
|
|
|
- int lastRetriedIndex = -1;
|
|
|
|
-
|
|
|
|
- // get block checksum for each block
|
|
|
|
- long remaining = length;
|
|
|
|
- if (src.contains(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR)) {
|
|
|
|
- remaining = Math.min(length, blockLocations.getFileLength());
|
|
|
|
- }
|
|
|
|
- for(int i = 0; i < locatedblocks.size() && remaining > 0; i++) {
|
|
|
|
- if (refetchBlocks) { // refetch to get fresh tokens
|
|
|
|
- blockLocations = callGetBlockLocations(namenode, src, 0, length);
|
|
|
|
- if (null == blockLocations) {
|
|
|
|
- throw new FileNotFoundException("File does not exist: " + src);
|
|
|
|
- }
|
|
|
|
- if (blockLocations.isUnderConstruction()) {
|
|
|
|
- throw new IOException("Fail to get checksum, since file " + src
|
|
|
|
- + " is under construction.");
|
|
|
|
- }
|
|
|
|
- locatedblocks = blockLocations.getLocatedBlocks();
|
|
|
|
- refetchBlocks = false;
|
|
|
|
- }
|
|
|
|
- LocatedBlock lb = locatedblocks.get(i);
|
|
|
|
- final ExtendedBlock block = lb.getBlock();
|
|
|
|
- if (remaining < block.getNumBytes()) {
|
|
|
|
- block.setNumBytes(remaining);
|
|
|
|
- }
|
|
|
|
- remaining -= block.getNumBytes();
|
|
|
|
- final DatanodeInfo[] datanodes = lb.getLocations();
|
|
|
|
-
|
|
|
|
- //try each datanode location of the block
|
|
|
|
- final int timeout = 3000 * datanodes.length +
|
|
|
|
- dfsClientConf.getSocketTimeout();
|
|
|
|
- boolean done = false;
|
|
|
|
- for(int j = 0; !done && j < datanodes.length; j++) {
|
|
|
|
- DataOutputStream out = null;
|
|
|
|
- DataInputStream in = null;
|
|
|
|
-
|
|
|
|
- try {
|
|
|
|
- //connect to a datanode
|
|
|
|
- IOStreamPair pair = connectToDN(datanodes[j], timeout, lb);
|
|
|
|
- out = new DataOutputStream(new BufferedOutputStream(pair.out,
|
|
|
|
- smallBufferSize));
|
|
|
|
- in = new DataInputStream(pair.in);
|
|
|
|
-
|
|
|
|
- LOG.debug("write to {}: {}, block={}",
|
|
|
|
- datanodes[j], Op.BLOCK_CHECKSUM, block);
|
|
|
|
- // get block MD5
|
|
|
|
- new Sender(out).blockChecksum(block, lb.getBlockToken());
|
|
|
|
-
|
|
|
|
- final BlockOpResponseProto reply =
|
|
|
|
- BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
|
|
|
|
-
|
|
|
|
- String logInfo = "for block " + block + " from datanode " +
|
|
|
|
- datanodes[j];
|
|
|
|
- DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
|
|
|
|
-
|
|
|
|
- OpBlockChecksumResponseProto checksumData =
|
|
|
|
- reply.getChecksumResponse();
|
|
|
|
-
|
|
|
|
- //read byte-per-checksum
|
|
|
|
- final int bpc = checksumData.getBytesPerCrc();
|
|
|
|
- if (i == 0) { //first block
|
|
|
|
- bytesPerCRC = bpc;
|
|
|
|
- }
|
|
|
|
- else if (bpc != bytesPerCRC) {
|
|
|
|
- throw new IOException("Byte-per-checksum not matched: bpc=" + bpc
|
|
|
|
- + " but bytesPerCRC=" + bytesPerCRC);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- //read crc-per-block
|
|
|
|
- final long cpb = checksumData.getCrcPerBlock();
|
|
|
|
- if (locatedblocks.size() > 1 && i == 0) {
|
|
|
|
- crcPerBlock = cpb;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- //read md5
|
|
|
|
- final MD5Hash md5 = new MD5Hash(
|
|
|
|
- checksumData.getMd5().toByteArray());
|
|
|
|
- md5.write(md5out);
|
|
|
|
-
|
|
|
|
- // read crc-type
|
|
|
|
- final DataChecksum.Type ct;
|
|
|
|
- if (checksumData.hasCrcType()) {
|
|
|
|
- ct = PBHelperClient.convert(checksumData
|
|
|
|
- .getCrcType());
|
|
|
|
- } else {
|
|
|
|
- LOG.debug("Retrieving checksum from an earlier-version DataNode: " +
|
|
|
|
- "inferring checksum by reading first byte");
|
|
|
|
- ct = inferChecksumTypeByReading(lb, datanodes[j]);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (i == 0) { // first block
|
|
|
|
- crcType = ct;
|
|
|
|
- } else if (crcType != DataChecksum.Type.MIXED
|
|
|
|
- && crcType != ct) {
|
|
|
|
- // if crc types are mixed in a file
|
|
|
|
- crcType = DataChecksum.Type.MIXED;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- done = true;
|
|
|
|
-
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- if (i == 0) {
|
|
|
|
- LOG.debug("set bytesPerCRC=" + bytesPerCRC
|
|
|
|
- + ", crcPerBlock=" + crcPerBlock);
|
|
|
|
- }
|
|
|
|
- LOG.debug("got reply from " + datanodes[j] + ": md5=" + md5);
|
|
|
|
- }
|
|
|
|
- } catch (InvalidBlockTokenException ibte) {
|
|
|
|
- if (i > lastRetriedIndex) {
|
|
|
|
- LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
|
|
|
|
- + "for file {} for block {} from datanode {}. Will retry "
|
|
|
|
- + "the block once.",
|
|
|
|
- src, block, datanodes[j]);
|
|
|
|
- lastRetriedIndex = i;
|
|
|
|
- done = true; // actually it's not done; but we'll retry
|
|
|
|
- i--; // repeat at i-th block
|
|
|
|
- refetchBlocks = true;
|
|
|
|
- break;
|
|
|
|
- }
|
|
|
|
- } catch (IOException ie) {
|
|
|
|
- LOG.warn("src=" + src + ", datanodes["+j+"]=" + datanodes[j], ie);
|
|
|
|
- } finally {
|
|
|
|
- IOUtils.closeStream(in);
|
|
|
|
- IOUtils.closeStream(out);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (!done) {
|
|
|
|
- throw new IOException("Fail to get block MD5 for " + block);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
|
|
- //compute file MD5
|
|
|
|
- final MD5Hash fileMD5 = MD5Hash.digest(md5out.getData());
|
|
|
|
- switch (crcType) {
|
|
|
|
- case CRC32:
|
|
|
|
- return new MD5MD5CRC32GzipFileChecksum(bytesPerCRC,
|
|
|
|
- crcPerBlock, fileMD5);
|
|
|
|
- case CRC32C:
|
|
|
|
- return new MD5MD5CRC32CastagnoliFileChecksum(bytesPerCRC,
|
|
|
|
- crcPerBlock, fileMD5);
|
|
|
|
- default:
|
|
|
|
- // If there is no block allocated for the file,
|
|
|
|
- // return one with the magic entry that matches what previous
|
|
|
|
- // hdfs versions return.
|
|
|
|
- if (locatedblocks.size() == 0) {
|
|
|
|
- return new MD5MD5CRC32GzipFileChecksum(0, 0, fileMD5);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // we should never get here since the validity was checked
|
|
|
|
- // when getCrcType() was called above.
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
|
|
+ return blockLocations;
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Connect to the given datanode's datantrasfer port, and return
|
|
|
|
- * the resulting IOStreamPair. This includes encryption wrapping, etc.
|
|
|
|
- */
|
|
|
|
- private IOStreamPair connectToDN(DatanodeInfo dn, int timeout,
|
|
|
|
- LocatedBlock lb) throws IOException {
|
|
|
|
- boolean success = false;
|
|
|
|
- Socket sock = null;
|
|
|
|
- try {
|
|
|
|
- sock = socketFactory.createSocket();
|
|
|
|
- String dnAddr = dn.getXferAddr(getConf().isConnectToDnViaHostname());
|
|
|
|
- LOG.debug("Connecting to datanode {}", dnAddr);
|
|
|
|
- NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr), timeout);
|
|
|
|
- sock.setTcpNoDelay(dfsClientConf.getDataTransferTcpNoDelay());
|
|
|
|
- sock.setSoTimeout(timeout);
|
|
|
|
-
|
|
|
|
- OutputStream unbufOut = NetUtils.getOutputStream(sock);
|
|
|
|
- InputStream unbufIn = NetUtils.getInputStream(sock);
|
|
|
|
- IOStreamPair ret = saslClient.newSocketSend(sock, unbufOut, unbufIn, this,
|
|
|
|
- lb.getBlockToken(), dn);
|
|
|
|
- success = true;
|
|
|
|
- return ret;
|
|
|
|
- } finally {
|
|
|
|
- if (!success) {
|
|
|
|
- IOUtils.closeSocket(sock);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ protected IOStreamPair connectToDN(DatanodeInfo dn, int timeout,
|
|
|
|
+ Token<BlockTokenIdentifier> blockToken)
|
|
|
|
+ throws IOException {
|
|
|
|
+ return DFSUtilClient.connectToDN(dn, timeout, conf, saslClient,
|
|
|
|
+ socketFactory, getConf().isConnectToDnViaHostname(), this, blockToken);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1917,19 +1742,17 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|
* @return the inferred checksum type
|
|
* @return the inferred checksum type
|
|
* @throws IOException if an error occurs
|
|
* @throws IOException if an error occurs
|
|
*/
|
|
*/
|
|
- private Type inferChecksumTypeByReading(LocatedBlock lb, DatanodeInfo dn)
|
|
|
|
|
|
+ protected Type inferChecksumTypeByReading(LocatedBlock lb, DatanodeInfo dn)
|
|
throws IOException {
|
|
throws IOException {
|
|
- IOStreamPair pair = connectToDN(dn, dfsClientConf.getSocketTimeout(), lb);
|
|
|
|
|
|
+ IOStreamPair pair = connectToDN(dn, dfsClientConf.getSocketTimeout(),
|
|
|
|
+ lb.getBlockToken());
|
|
|
|
|
|
try {
|
|
try {
|
|
- DataOutputStream out = new DataOutputStream(
|
|
|
|
- new BufferedOutputStream(pair.out, smallBufferSize));
|
|
|
|
- DataInputStream in = new DataInputStream(pair.in);
|
|
|
|
-
|
|
|
|
- new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName,
|
|
|
|
|
|
+ new Sender((DataOutputStream) pair.out).readBlock(lb.getBlock(),
|
|
|
|
+ lb.getBlockToken(), clientName,
|
|
0, 1, true, CachingStrategy.newDefaultStrategy());
|
|
0, 1, true, CachingStrategy.newDefaultStrategy());
|
|
final BlockOpResponseProto reply =
|
|
final BlockOpResponseProto reply =
|
|
- BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
|
|
|
|
|
|
+ BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(pair.in));
|
|
String logInfo = "trying to read " + lb.getBlock() + " from datanode " +
|
|
String logInfo = "trying to read " + lb.getBlock() + " from datanode " +
|
|
dn;
|
|
dn;
|
|
DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
|
|
DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
|