|
@@ -45,6 +45,8 @@ import org.apache.hadoop.io.nativeio.NativeIO;
|
|
|
import org.apache.hadoop.net.SocketOutputStream;
|
|
|
import org.apache.hadoop.util.DataChecksum;
|
|
|
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
+
|
|
|
/**
|
|
|
* Reads a block from the disk and sends it to a recipient.
|
|
|
*
|
|
@@ -158,12 +160,14 @@ class BlockSender implements java.io.Closeable {
|
|
|
* @param length length of data to read
|
|
|
* @param corruptChecksumOk
|
|
|
* @param verifyChecksum verify checksum while reading the data
|
|
|
+ * @param sendChecksum send checksum to client.
|
|
|
* @param datanode datanode from which the block is being read
|
|
|
* @param clientTraceFmt format string used to print client trace logs
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
BlockSender(ExtendedBlock block, long startOffset, long length,
|
|
|
boolean corruptChecksumOk, boolean verifyChecksum,
|
|
|
+ boolean sendChecksum,
|
|
|
DataNode datanode, String clientTraceFmt)
|
|
|
throws IOException {
|
|
|
try {
|
|
@@ -175,6 +179,13 @@ class BlockSender implements java.io.Closeable {
|
|
|
this.shouldDropCacheBehindRead = datanode.getDnConf().dropCacheBehindReads;
|
|
|
this.datanode = datanode;
|
|
|
|
|
|
+ if (verifyChecksum) {
|
|
|
+ // To simplify implementation, callers may not specify verification
|
|
|
+ // without sending.
|
|
|
+ Preconditions.checkArgument(sendChecksum,
|
|
|
+ "If verifying checksum, currently must also send it.");
|
|
|
+ }
|
|
|
+
|
|
|
final Replica replica;
|
|
|
final long replicaVisibleLength;
|
|
|
synchronized(datanode.data) {
|
|
@@ -213,29 +224,37 @@ class BlockSender implements java.io.Closeable {
|
|
|
* False, True: will verify checksum
|
|
|
* False, False: throws IOException file not found
|
|
|
*/
|
|
|
- DataChecksum csum;
|
|
|
- final InputStream metaIn = datanode.data.getMetaDataInputStream(block);
|
|
|
- if (!corruptChecksumOk || metaIn != null) {
|
|
|
- if (metaIn == null) {
|
|
|
- //need checksum but meta-data not found
|
|
|
- throw new FileNotFoundException("Meta-data not found for " + block);
|
|
|
- }
|
|
|
-
|
|
|
- checksumIn = new DataInputStream(
|
|
|
- new BufferedInputStream(metaIn, HdfsConstants.IO_FILE_BUFFER_SIZE));
|
|
|
-
|
|
|
- // read and handle the common header here. For now just a version
|
|
|
- BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
|
|
|
- short version = header.getVersion();
|
|
|
- if (version != BlockMetadataHeader.VERSION) {
|
|
|
- LOG.warn("Wrong version (" + version + ") for metadata file for "
|
|
|
- + block + " ignoring ...");
|
|
|
+ DataChecksum csum = null;
|
|
|
+ if (verifyChecksum || sendChecksum) {
|
|
|
+ final InputStream metaIn = datanode.data.getMetaDataInputStream(block);
|
|
|
+ if (!corruptChecksumOk || metaIn != null) {
|
|
|
+ if (metaIn == null) {
|
|
|
+ //need checksum but meta-data not found
|
|
|
+ throw new FileNotFoundException("Meta-data not found for " + block);
|
|
|
+ }
|
|
|
+
|
|
|
+ checksumIn = new DataInputStream(
|
|
|
+ new BufferedInputStream(metaIn, HdfsConstants.IO_FILE_BUFFER_SIZE));
|
|
|
+
|
|
|
+ // read and handle the common header here. For now just a version
|
|
|
+ BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
|
|
|
+ short version = header.getVersion();
|
|
|
+ if (version != BlockMetadataHeader.VERSION) {
|
|
|
+ LOG.warn("Wrong version (" + version + ") for metadata file for "
|
|
|
+ + block + " ignoring ...");
|
|
|
+ }
|
|
|
+ csum = header.getChecksum();
|
|
|
+ } else {
|
|
|
+ LOG.warn("Could not find metadata file for " + block);
|
|
|
}
|
|
|
- csum = header.getChecksum();
|
|
|
- } else {
|
|
|
- LOG.warn("Could not find metadata file for " + block);
|
|
|
- // This only decides the buffer size. Use BUFFER_SIZE?
|
|
|
- csum = DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 16 * 1024);
|
|
|
+ }
|
|
|
+ if (csum == null) {
|
|
|
+ // The number of bytes per checksum here determines the alignment
|
|
|
+ // of reads: we always start reading at a checksum chunk boundary,
|
|
|
+ // even if the checksum type is NULL. So, choosing too big of a value
|
|
|
+ // would risk sending too much unnecessary data. 512 (1 disk sector)
|
|
|
+ // is likely to result in minimal extra IO.
|
|
|
+ csum = DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 512);
|
|
|
}
|
|
|
|
|
|
/*
|