blob: 7eb6e16c55aecb24cb879548bced84cef7ba5725 [file] [log] [blame]
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.datanode;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.SocketException;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.net.SocketOutputStream;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.StringUtils;
/**
* Reads a block from the disk and sends it to a recipient.
*/
public class RaidBlockSender implements java.io.Closeable {
public static final Log LOG = DataNode.LOG;
static final Log ClientTraceLog = DataNode.ClientTraceLog;
private ExtendedBlock block; // the block to read from
/** The visible length of a replica. */
private final long replicaVisibleLength;
private InputStream blockIn; // data stream
private long blockInPosition = -1; // updated while using transferTo().
private DataInputStream checksumIn; // checksum datastream
private DataChecksum checksum; // checksum stream
private long offset; // starting position to read
/** Initial position to read */
private long initialOffset;
private long endOffset; // ending position
private int chunkSize; // chunk size
private int checksumSize; // checksum size
private boolean corruptChecksumOk; // if need to verify checksum
private boolean chunkOffsetOK; // if need to send chunk offset
private long seqno; // sequence number of packet
private boolean transferToAllowed = true;
private boolean blockReadFully; //set when the whole block is read
private boolean verifyChecksum; //if true, check is verified while reading
private final String clientTraceFmt; // format of client trace log message
/**
* Minimum buffer used while sending data to clients. Used only if
* transferTo() is enabled. 64KB is not that large. It could be larger, but
* not sure if there will be much more improvement.
*/
private static final int MIN_BUFFER_WITH_TRANSFERTO = 64*1024;
private static final int TRANSFERTO_BUFFER_SIZE = Math.max(
HdfsConstants.IO_FILE_BUFFER_SIZE, MIN_BUFFER_WITH_TRANSFERTO);
private volatile ChunkChecksum lastChunkChecksum = null;
public RaidBlockSender(ExtendedBlock block, long blockLength, long startOffset, long length,
boolean corruptChecksumOk, boolean chunkOffsetOK,
boolean verifyChecksum, boolean transferToAllowed,
DataInputStream metadataIn, InputStreamFactory streamFactory
) throws IOException {
this(block, blockLength, startOffset, length,
corruptChecksumOk, chunkOffsetOK,
verifyChecksum, transferToAllowed,
metadataIn, streamFactory, null);
}
public RaidBlockSender(ExtendedBlock block, long blockLength, long startOffset, long length,
boolean corruptChecksumOk, boolean chunkOffsetOK,
boolean verifyChecksum, boolean transferToAllowed,
DataInputStream metadataIn, InputStreamFactory streamFactory,
String clientTraceFmt) throws IOException {
try {
this.block = block;
this.chunkOffsetOK = chunkOffsetOK;
this.corruptChecksumOk = corruptChecksumOk;
this.verifyChecksum = verifyChecksum;
this.replicaVisibleLength = blockLength;
this.transferToAllowed = transferToAllowed;
this.clientTraceFmt = clientTraceFmt;
if ( !corruptChecksumOk || metadataIn != null) {
this.checksumIn = metadataIn;
// 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 ...");
}
checksum = header.getChecksum();
} else {
LOG.warn("Could not find metadata file for " + block);
// This only decides the buffer size. Use BUFFER_SIZE?
checksum = DataChecksum.newDataChecksum(DataChecksum.CHECKSUM_NULL,
16 * 1024);
}
/* If bytesPerChecksum is very large, then the metadata file
* is mostly corrupted. For now just truncate bytesPerchecksum to
* blockLength.
*/
int size = checksum.getBytesPerChecksum();
if (size > 10*1024*1024 && size > replicaVisibleLength) {
checksum = DataChecksum.newDataChecksum(checksum.getChecksumType(),
Math.max((int)replicaVisibleLength, 10*1024*1024));
size = checksum.getBytesPerChecksum();
}
chunkSize = size;
checksumSize = checksum.getChecksumSize();
if (length < 0) {
length = replicaVisibleLength;
}
endOffset = blockLength;
if (startOffset < 0 || startOffset > endOffset
|| (length + startOffset) > endOffset) {
String msg = " Offset " + startOffset + " and length " + length
+ " don't match block " + block + " ( blockLen " + endOffset + " )";
LOG.warn("sendBlock() : " + msg);
throw new IOException(msg);
}
offset = (startOffset - (startOffset % chunkSize));
if (length >= 0) {
// Make sure endOffset points to end of a checksumed chunk.
long tmpLen = startOffset + length;
if (tmpLen % chunkSize != 0) {
tmpLen += (chunkSize - tmpLen % chunkSize);
}
if (tmpLen < endOffset) {
// will use on-disk checksum here since the end is a stable chunk
endOffset = tmpLen;
}
}
// seek to the right offsets
if (offset > 0) {
long checksumSkip = (offset / chunkSize) * checksumSize;
// note blockInStream is seeked when created below
if (checksumSkip > 0) {
// Should we use seek() for checksum file as well?
IOUtils.skipFully(checksumIn, checksumSkip);
}
}
seqno = 0;
blockIn = streamFactory.createStream(offset);
} catch (IOException ioe) {
IOUtils.closeStream(this);
IOUtils.closeStream(blockIn);
throw ioe;
}
}
/**
* close opened files.
*/
public void close() throws IOException {
IOException ioe = null;
// close checksum file
if(checksumIn!=null) {
try {
checksumIn.close();
} catch (IOException e) {
ioe = e;
}
checksumIn = null;
}
// close data file
if(blockIn!=null) {
try {
blockIn.close();
} catch (IOException e) {
ioe = e;
}
blockIn = null;
}
// throw IOException if there is any
if(ioe!= null) {
throw ioe;
}
}
/**
* Converts an IOExcpetion (not subclasses) to SocketException.
* This is typically done to indicate to upper layers that the error
* was a socket error rather than often more serious exceptions like
* disk errors.
*/
private static IOException ioeToSocketException(IOException ioe) {
if (ioe.getClass().equals(IOException.class)) {
// "se" could be a new class in stead of SocketException.
IOException se = new SocketException("Original Exception : " + ioe);
se.initCause(ioe);
/* Change the stacktrace so that original trace is not truncated
* when printed.*/
se.setStackTrace(ioe.getStackTrace());
return se;
}
// otherwise just return the same exception.
return ioe;
}
/**
* @param datalen Length of data
* @return number of chunks for data of given size
*/
private int numberOfChunks(long datalen) {
return (int) ((datalen + chunkSize - 1)/chunkSize);
}
/**
* Write packet header into {@code pkt}
*/
private void writePacketHeader(ByteBuffer pkt, int dataLen, int packetLen) {
pkt.clear();
PacketHeader header = new PacketHeader(packetLen, offset, seqno,
(dataLen == 0), dataLen, false);
header.putInBuffer(pkt);
}
/**
* Read checksum into given buffer
* @param buf buffer to read the checksum into
* @param checksumOffset offset at which to write the checksum into buf
* @param checksumLen length of checksum to write
* @throws IOException on error
*/
private void readChecksum(byte[] buf, final int checksumOffset,
final int checksumLen) throws IOException {
if (checksumSize <= 0 && checksumIn == null) {
return;
}
try {
checksumIn.readFully(buf, checksumOffset, checksumLen);
} catch (IOException e) {
LOG.warn(" Could not read or failed to veirfy checksum for data"
+ " at offset " + offset + " for block " + block, e);
IOUtils.closeStream(checksumIn);
checksumIn = null;
if (corruptChecksumOk) {
if (checksumOffset < checksumLen) {
// Just fill the array with zeros.
Arrays.fill(buf, checksumOffset, checksumLen, (byte) 0);
}
} else {
throw e;
}
}
}
/**
* Sends a packet with up to maxChunks chunks of data.
*
* @param pkt buffer used for writing packet data
* @param maxChunks maximum number of chunks to send
* @param out stream to send data to
* @param transferTo use transferTo to send data
* @param throttler used for throttling data transfer bandwidth
*/
private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out,
boolean transferTo, DataTransferThrottler throttler) throws IOException {
int dataLen = (int) Math.min(endOffset - offset,
(chunkSize * (long) maxChunks));
int numChunks = numberOfChunks(dataLen); // Number of chunks be sent in the packet
int checksumDataLen = numChunks * checksumSize;
int packetLen = dataLen + checksumDataLen + 4;
boolean lastDataPacket = offset + dataLen == endOffset && dataLen > 0;
writePacketHeader(pkt, dataLen, packetLen);
int checksumOff = pkt.position();
byte[] buf = pkt.array();
if (checksumSize > 0 && checksumIn != null) {
readChecksum(buf, checksumOff, checksumDataLen);
// write in progress that we need to use to get last checksum
if (lastDataPacket && lastChunkChecksum != null) {
int start = checksumOff + checksumDataLen - checksumSize;
byte[] updatedChecksum = lastChunkChecksum.getChecksum();
if (updatedChecksum != null) {
System.arraycopy(updatedChecksum, 0, buf, start, checksumSize);
}
}
}
int dataOff = checksumOff + checksumDataLen;
if (!transferTo) { // normal transfer
IOUtils.readFully(blockIn, buf, dataOff, dataLen);
if (verifyChecksum) {
verifyChecksum(buf, dataOff, dataLen, numChunks, checksumOff);
}
}
try {
if (transferTo) {
SocketOutputStream sockOut = (SocketOutputStream)out;
sockOut.write(buf, 0, dataOff); // First write checksum
// no need to flush. since we know out is not a buffered stream.
sockOut.transferToFully(((FileInputStream)blockIn).getChannel(),
blockInPosition, dataLen);
blockInPosition += dataLen;
} else {
// normal transfer
out.write(buf, 0, dataOff + dataLen);
}
} catch (IOException e) {
/* Exception while writing to the client. Connection closure from
* the other end is mostly the case and we do not care much about
* it. But other things can go wrong, especially in transferTo(),
* which we do not want to ignore.
*
* The message parsing below should not be considered as a good
* coding example. NEVER do it to drive a program logic. NEVER.
* It was done here because the NIO throws an IOException for EPIPE.
*/
String ioem = e.getMessage();
if (!ioem.startsWith("Broken pipe") && !ioem.startsWith("Connection reset")) {
LOG.error("BlockSender.sendChunks() exception: ", e);
}
throw ioeToSocketException(e);
}
if (throttler != null) { // rebalancing so throttle
throttler.throttle(packetLen);
}
return dataLen;
}
/**
* Compute checksum for chunks and verify the checksum that is read from
* the metadata file is correct.
*
* @param buf buffer that has checksum and data
* @param dataOffset position where data is written in the buf
* @param datalen length of data
* @param numChunks number of chunks corresponding to data
* @param checksumOffset offset where checksum is written in the buf
* @throws ChecksumException on failed checksum verification
*/
public void verifyChecksum(final byte[] buf, final int dataOffset,
final int datalen, final int numChunks, final int checksumOffset)
throws ChecksumException {
int dOff = dataOffset;
int cOff = checksumOffset;
int dLeft = datalen;
for (int i = 0; i < numChunks; i++) {
checksum.reset();
int dLen = Math.min(dLeft, chunkSize);
checksum.update(buf, dOff, dLen);
if (!checksum.compare(buf, cOff)) {
long failedPos = offset + datalen - dLeft;
throw new ChecksumException("Checksum failed at " + failedPos,
failedPos);
}
dLeft -= dLen;
dOff += dLen;
cOff += checksumSize;
}
}
/**
* sendBlock() is used to read block and its metadata and stream the data to
* either a client or to another datanode.
*
* @param out stream to which the block is written to
* @param baseStream optional. if non-null, <code>out</code> is assumed to
* be a wrapper over this stream. This enables optimizations for
* sending the data, e.g.
* {@link SocketOutputStream#transferToFully(FileChannel,
* long, int)}.
* @return total bytes reads, including crc.
*/
public long sendBlock(DataOutputStream out, OutputStream baseStream)
throws IOException {
if (out == null) {
throw new IOException( "out stream is null" );
}
initialOffset = offset;
long totalRead = 0;
OutputStream streamForSendChunks = out;
final long startTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
try {
int maxChunksPerPacket;
int pktSize = PacketHeader.PKT_HEADER_LEN;
boolean transferTo = transferToAllowed && !verifyChecksum
&& baseStream instanceof SocketOutputStream
&& blockIn instanceof FileInputStream;
if (transferTo) {
FileChannel fileChannel = ((FileInputStream)blockIn).getChannel();
blockInPosition = fileChannel.position();
streamForSendChunks = baseStream;
maxChunksPerPacket = numberOfChunks(TRANSFERTO_BUFFER_SIZE);
// Smaller packet size to only hold checksum when doing transferTo
pktSize += checksumSize * maxChunksPerPacket;
} else {
maxChunksPerPacket = Math.max(1,
numberOfChunks(HdfsConstants.IO_FILE_BUFFER_SIZE));
// Packet size includes both checksum and data
pktSize += (chunkSize + checksumSize) * maxChunksPerPacket;
}
ByteBuffer pktBuf = ByteBuffer.allocate(pktSize);
while (endOffset > offset) {
long len = sendPacket(pktBuf, maxChunksPerPacket, streamForSendChunks,
transferTo, null);
offset += len;
totalRead += len + (numberOfChunks(len) * checksumSize);
seqno++;
}
try {
// send an empty packet to mark the end of the block
sendPacket(pktBuf, maxChunksPerPacket, streamForSendChunks, transferTo,
null);
out.flush();
} catch (IOException e) { //socket error
throw ioeToSocketException(e);
}
blockReadFully = true;
} finally {
if (clientTraceFmt != null) {
final long endTime = System.nanoTime();
ClientTraceLog.info(String.format(clientTraceFmt, totalRead,
initialOffset, endTime - startTime));
}
close();
}
return totalRead;
}
boolean isBlockReadFully() {
return blockReadFully;
}
public static interface InputStreamFactory {
public InputStream createStream(long offset) throws IOException;
}
/**
* @return the checksum type that will be used with this block transfer.
*/
public DataChecksum getChecksum() {
return checksum;
}
private static class BlockInputStreamFactory implements InputStreamFactory {
private final ExtendedBlock block;
private final FsDatasetSpi<?> data;
private BlockInputStreamFactory(ExtendedBlock block, FsDatasetSpi<?> data) {
this.block = block;
this.data = data;
}
@Override
public InputStream createStream(long offset) throws IOException {
return data.getBlockInputStream(block, offset);
}
}
}