|
@@ -27,6 +27,7 @@ import org.apache.hadoop.ipc.*;
|
|
import org.apache.hadoop.conf.*;
|
|
import org.apache.hadoop.conf.*;
|
|
import org.apache.hadoop.net.DNS;
|
|
import org.apache.hadoop.net.DNS;
|
|
import org.apache.hadoop.net.NetUtils;
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
|
+import org.apache.hadoop.net.SocketOutputStream;
|
|
import org.apache.hadoop.util.*;
|
|
import org.apache.hadoop.util.*;
|
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
|
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
|
@@ -39,6 +40,7 @@ import org.apache.hadoop.dfs.datanode.metrics.DataNodeMetrics;
|
|
import java.io.*;
|
|
import java.io.*;
|
|
import java.net.*;
|
|
import java.net.*;
|
|
import java.nio.ByteBuffer;
|
|
import java.nio.ByteBuffer;
|
|
|
|
+import java.nio.channels.FileChannel;
|
|
import java.nio.channels.ServerSocketChannel;
|
|
import java.nio.channels.ServerSocketChannel;
|
|
import java.nio.channels.SocketChannel;
|
|
import java.nio.channels.SocketChannel;
|
|
import java.util.*;
|
|
import java.util.*;
|
|
@@ -89,6 +91,13 @@ public class DataNode implements FSConstants, Runnable {
|
|
return NetUtils.createSocketAddr(target);
|
|
return NetUtils.createSocketAddr(target);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * 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;
|
|
|
|
+
|
|
DatanodeProtocol namenode = null;
|
|
DatanodeProtocol namenode = null;
|
|
FSDatasetInterface data = null;
|
|
FSDatasetInterface data = null;
|
|
DatanodeRegistration dnRegistration = null;
|
|
DatanodeRegistration dnRegistration = null;
|
|
@@ -120,6 +129,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
int defaultBytesPerChecksum = 512;
|
|
int defaultBytesPerChecksum = 512;
|
|
private int socketTimeout;
|
|
private int socketTimeout;
|
|
private int socketWriteTimeout = 0;
|
|
private int socketWriteTimeout = 0;
|
|
|
|
+ private boolean transferToAllowed = true;
|
|
|
|
|
|
private DataBlockScanner blockScanner;
|
|
private DataBlockScanner blockScanner;
|
|
private Daemon blockScannerThread;
|
|
private Daemon blockScannerThread;
|
|
@@ -208,7 +218,10 @@ public class DataNode implements FSConstants, Runnable {
|
|
FSConstants.READ_TIMEOUT);
|
|
FSConstants.READ_TIMEOUT);
|
|
this.socketWriteTimeout = conf.getInt("dfs.datanode.socket.write.timeout",
|
|
this.socketWriteTimeout = conf.getInt("dfs.datanode.socket.write.timeout",
|
|
FSConstants.WRITE_TIMEOUT);
|
|
FSConstants.WRITE_TIMEOUT);
|
|
-
|
|
|
|
|
|
+ /* Based on results on different platforms, we might need set the default
|
|
|
|
+ * to false on some of them. */
|
|
|
|
+ this.transferToAllowed = conf.getBoolean("dfs.datanode.transferTo.allowed",
|
|
|
|
+ true);
|
|
String address =
|
|
String address =
|
|
NetUtils.getServerAddress(conf,
|
|
NetUtils.getServerAddress(conf,
|
|
"dfs.datanode.bindAddress",
|
|
"dfs.datanode.bindAddress",
|
|
@@ -1022,8 +1035,9 @@ public class DataNode implements FSConstants, Runnable {
|
|
long length = in.readLong();
|
|
long length = in.readLong();
|
|
|
|
|
|
// send the block
|
|
// send the block
|
|
- DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
|
|
|
- NetUtils.getOutputStream(s, socketWriteTimeout), SMALL_BUFFER_SIZE));
|
|
|
|
|
|
+ OutputStream baseStream = NetUtils.getOutputStream(s,socketWriteTimeout);
|
|
|
|
+ DataOutputStream out = new DataOutputStream(
|
|
|
|
+ new BufferedOutputStream(baseStream, SMALL_BUFFER_SIZE));
|
|
|
|
|
|
BlockSender blockSender = null;
|
|
BlockSender blockSender = null;
|
|
try {
|
|
try {
|
|
@@ -1036,7 +1050,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
}
|
|
}
|
|
|
|
|
|
out.writeShort(DataNode.OP_STATUS_SUCCESS); // send op status
|
|
out.writeShort(DataNode.OP_STATUS_SUCCESS); // send op status
|
|
- long read = blockSender.sendBlock(out, null); // send data
|
|
|
|
|
|
+ long read = blockSender.sendBlock(out, baseStream, null); // send data
|
|
|
|
|
|
if (blockSender.isBlockReadFully()) {
|
|
if (blockSender.isBlockReadFully()) {
|
|
// See if client verification succeeded.
|
|
// See if client verification succeeded.
|
|
@@ -1306,9 +1320,10 @@ public class DataNode implements FSConstants, Runnable {
|
|
targetSock.connect(targetAddr, socketTimeout);
|
|
targetSock.connect(targetAddr, socketTimeout);
|
|
targetSock.setSoTimeout(socketTimeout);
|
|
targetSock.setSoTimeout(socketTimeout);
|
|
|
|
|
|
- targetOut = new DataOutputStream(new BufferedOutputStream(
|
|
|
|
- NetUtils.getOutputStream(targetSock, socketWriteTimeout),
|
|
|
|
- SMALL_BUFFER_SIZE));
|
|
|
|
|
|
+ OutputStream baseStream = NetUtils.getOutputStream(targetSock,
|
|
|
|
+ socketWriteTimeout);
|
|
|
|
+ targetOut = new DataOutputStream(
|
|
|
|
+ new BufferedOutputStream(baseStream, SMALL_BUFFER_SIZE));
|
|
|
|
|
|
/* send request to the target */
|
|
/* send request to the target */
|
|
// fist write header info
|
|
// fist write header info
|
|
@@ -1318,7 +1333,8 @@ public class DataNode implements FSConstants, Runnable {
|
|
Text.writeString( targetOut, source); // del hint
|
|
Text.writeString( targetOut, source); // del hint
|
|
|
|
|
|
// then send data
|
|
// then send data
|
|
- long read = blockSender.sendBlock(targetOut, balancingThrottler);
|
|
|
|
|
|
+ long read = blockSender.sendBlock(targetOut, baseStream,
|
|
|
|
+ balancingThrottler);
|
|
|
|
|
|
myMetrics.bytesRead.inc((int) read);
|
|
myMetrics.bytesRead.inc((int) read);
|
|
myMetrics.blocksRead.inc();
|
|
myMetrics.blocksRead.inc();
|
|
@@ -1567,6 +1583,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
class BlockSender implements java.io.Closeable {
|
|
class BlockSender implements java.io.Closeable {
|
|
private Block block; // the block to read from
|
|
private Block block; // the block to read from
|
|
private InputStream blockIn; // data stream
|
|
private InputStream blockIn; // data stream
|
|
|
|
+ private long blockInPosition = -1; // updated while using transferTo().
|
|
private DataInputStream checksumIn; // checksum datastream
|
|
private DataInputStream checksumIn; // checksum datastream
|
|
private DataChecksum checksum; // checksum stream
|
|
private DataChecksum checksum; // checksum stream
|
|
private long offset; // starting position to read
|
|
private long offset; // starting position to read
|
|
@@ -1581,7 +1598,6 @@ public class DataNode implements FSConstants, Runnable {
|
|
private boolean blockReadFully; //set when the whole block is read
|
|
private boolean blockReadFully; //set when the whole block is read
|
|
private boolean verifyChecksum; //if true, check is verified while reading
|
|
private boolean verifyChecksum; //if true, check is verified while reading
|
|
private Throttler throttler;
|
|
private Throttler throttler;
|
|
- private OutputStream out;
|
|
|
|
|
|
|
|
static final int PKT_HEADER_LEN = ( 4 + /* PacketLen */
|
|
static final int PKT_HEADER_LEN = ( 4 + /* PacketLen */
|
|
8 + /* offset in block */
|
|
8 + /* offset in block */
|
|
@@ -1705,8 +1721,14 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Sends upto maxChunks chunks of data.
|
|
* Sends upto maxChunks chunks of data.
|
|
|
|
+ *
|
|
|
|
+ * When blockInPosition is >= 0, assumes 'out' is a
|
|
|
|
+ * {@link SocketOutputStream} and tries
|
|
|
|
+ * {@link SocketOutputStream#transferToFully(FileChannel, long, int)} to
|
|
|
|
+ * send data (and updates blockInPosition).
|
|
*/
|
|
*/
|
|
- private int sendChunks(ByteBuffer pkt, int maxChunks) throws IOException {
|
|
|
|
|
|
+ private int sendChunks(ByteBuffer pkt, int maxChunks, OutputStream out)
|
|
|
|
+ throws IOException {
|
|
// Sends multiple chunks in one packet with a single write().
|
|
// Sends multiple chunks in one packet with a single write().
|
|
|
|
|
|
int len = Math.min((int) (endOffset - offset),
|
|
int len = Math.min((int) (endOffset - offset),
|
|
@@ -1750,28 +1772,44 @@ public class DataNode implements FSConstants, Runnable {
|
|
}
|
|
}
|
|
|
|
|
|
int dataOff = checksumOff + checksumLen;
|
|
int dataOff = checksumOff + checksumLen;
|
|
- IOUtils.readFully(blockIn, buf, dataOff, len);
|
|
|
|
|
|
|
|
- if (verifyChecksum) {
|
|
|
|
- int dOff = dataOff;
|
|
|
|
- int cOff = checksumOff;
|
|
|
|
- int dLeft = len;
|
|
|
|
|
|
+ if (blockInPosition >= 0) {
|
|
|
|
+ //use transferTo(). Checks on out and blockIn are already done.
|
|
|
|
|
|
- for (int i=0; i<numChunks; i++) {
|
|
|
|
- checksum.reset();
|
|
|
|
- int dLen = Math.min(dLeft, bytesPerChecksum);
|
|
|
|
- checksum.update(buf, dOff, dLen);
|
|
|
|
- if (!checksum.compare(buf, cOff)) {
|
|
|
|
- throw new ChecksumException("Checksum failed at " +
|
|
|
|
- (offset + len - dLeft), len);
|
|
|
|
|
|
+ SocketOutputStream sockOut = (SocketOutputStream)out;
|
|
|
|
+ //first write the packet
|
|
|
|
+ sockOut.write(buf, 0, dataOff);
|
|
|
|
+ // no need to flush. since we know out is not a buffered stream.
|
|
|
|
+
|
|
|
|
+ sockOut.transferToFully(((FileInputStream)blockIn).getChannel(),
|
|
|
|
+ blockInPosition, len);
|
|
|
|
+
|
|
|
|
+ blockInPosition += len;
|
|
|
|
+ } else {
|
|
|
|
+ //normal transfer
|
|
|
|
+ IOUtils.readFully(blockIn, buf, dataOff, len);
|
|
|
|
+
|
|
|
|
+ if (verifyChecksum) {
|
|
|
|
+ int dOff = dataOff;
|
|
|
|
+ int cOff = checksumOff;
|
|
|
|
+ int dLeft = len;
|
|
|
|
+
|
|
|
|
+ for (int i=0; i<numChunks; i++) {
|
|
|
|
+ checksum.reset();
|
|
|
|
+ int dLen = Math.min(dLeft, bytesPerChecksum);
|
|
|
|
+ checksum.update(buf, dOff, dLen);
|
|
|
|
+ if (!checksum.compare(buf, cOff)) {
|
|
|
|
+ throw new ChecksumException("Checksum failed at " +
|
|
|
|
+ (offset + len - dLeft), len);
|
|
|
|
+ }
|
|
|
|
+ dLeft -= dLen;
|
|
|
|
+ dOff += dLen;
|
|
|
|
+ cOff += checksumSize;
|
|
}
|
|
}
|
|
- dLeft -= dLen;
|
|
|
|
- dOff += dLen;
|
|
|
|
- cOff += checksumSize;
|
|
|
|
}
|
|
}
|
|
- }
|
|
|
|
|
|
|
|
- out.write(buf, 0, dataOff + len);
|
|
|
|
|
|
+ out.write(buf, 0, dataOff + len);
|
|
|
|
+ }
|
|
|
|
|
|
if (throttler != null) { // rebalancing so throttle
|
|
if (throttler != null) { // rebalancing so throttle
|
|
throttler.throttle(packetLen);
|
|
throttler.throttle(packetLen);
|
|
@@ -1785,32 +1823,64 @@ public class DataNode implements FSConstants, Runnable {
|
|
* either a client or to another datanode.
|
|
* either a client or to another datanode.
|
|
*
|
|
*
|
|
* @param out stream to which the block is written to
|
|
* @param out stream to which the block is written to
|
|
- * returns total bytes reads, including crc.
|
|
|
|
|
|
+ * @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)}.
|
|
|
|
+ * @param throttler for sending data.
|
|
|
|
+ * @return total bytes reads, including crc.
|
|
*/
|
|
*/
|
|
- long sendBlock(DataOutputStream out, Throttler throttler)
|
|
|
|
- throws IOException {
|
|
|
|
|
|
+ long sendBlock(DataOutputStream out, OutputStream baseStream,
|
|
|
|
+ Throttler throttler) throws IOException {
|
|
if( out == null ) {
|
|
if( out == null ) {
|
|
throw new IOException( "out stream is null" );
|
|
throw new IOException( "out stream is null" );
|
|
}
|
|
}
|
|
- this.out = out;
|
|
|
|
this.throttler = throttler;
|
|
this.throttler = throttler;
|
|
|
|
|
|
long initialOffset = offset;
|
|
long initialOffset = offset;
|
|
long totalRead = 0;
|
|
long totalRead = 0;
|
|
|
|
+ OutputStream streamForSendChunks = out;
|
|
|
|
+
|
|
try {
|
|
try {
|
|
checksum.writeHeader(out);
|
|
checksum.writeHeader(out);
|
|
if ( chunkOffsetOK ) {
|
|
if ( chunkOffsetOK ) {
|
|
out.writeLong( offset );
|
|
out.writeLong( offset );
|
|
}
|
|
}
|
|
- //set up sendBuf:
|
|
|
|
- int maxChunksPerPacket = Math.max(1,
|
|
|
|
- (BUFFER_SIZE + bytesPerChecksum - 1)/bytesPerChecksum);
|
|
|
|
- ByteBuffer pktBuf = ByteBuffer.allocate(PKT_HEADER_LEN +
|
|
|
|
- (bytesPerChecksum + checksumSize) * maxChunksPerPacket);
|
|
|
|
|
|
+ out.flush();
|
|
|
|
+
|
|
|
|
+ int maxChunksPerPacket;
|
|
|
|
+ int pktSize;
|
|
|
|
+
|
|
|
|
+ if (transferToAllowed && !verifyChecksum &&
|
|
|
|
+ baseStream instanceof SocketOutputStream &&
|
|
|
|
+ blockIn instanceof FileInputStream) {
|
|
|
|
+
|
|
|
|
+ FileChannel fileChannel = ((FileInputStream)blockIn).getChannel();
|
|
|
|
+
|
|
|
|
+ // blockInPosition also indicates sendChunks() uses transferTo.
|
|
|
|
+ blockInPosition = fileChannel.position();
|
|
|
|
+ streamForSendChunks = baseStream;
|
|
|
|
+
|
|
|
|
+ // assure a mininum buffer size.
|
|
|
|
+ maxChunksPerPacket = (Math.max(BUFFER_SIZE,
|
|
|
|
+ MIN_BUFFER_WITH_TRANSFERTO)
|
|
|
|
+ + bytesPerChecksum - 1)/bytesPerChecksum;
|
|
|
|
+
|
|
|
|
+ // allocate smaller buffer while using transferTo().
|
|
|
|
+ pktSize = PKT_HEADER_LEN + checksumSize * maxChunksPerPacket;
|
|
|
|
+ } else {
|
|
|
|
+ maxChunksPerPacket = Math.max(1,
|
|
|
|
+ (BUFFER_SIZE + bytesPerChecksum - 1)/bytesPerChecksum);
|
|
|
|
+ pktSize = PKT_HEADER_LEN +
|
|
|
|
+ (bytesPerChecksum + checksumSize) * maxChunksPerPacket;
|
|
|
|
+ }
|
|
|
|
|
|
|
|
+ ByteBuffer pktBuf = ByteBuffer.allocate(pktSize);
|
|
|
|
|
|
while (endOffset > offset) {
|
|
while (endOffset > offset) {
|
|
- long len = sendChunks(pktBuf, maxChunksPerPacket);
|
|
|
|
|
|
+ long len = sendChunks(pktBuf, maxChunksPerPacket,
|
|
|
|
+ streamForSendChunks);
|
|
offset += len;
|
|
offset += len;
|
|
totalRead += len + ((len + bytesPerChecksum - 1)/bytesPerChecksum*
|
|
totalRead += len + ((len + bytesPerChecksum - 1)/bytesPerChecksum*
|
|
checksumSize);
|
|
checksumSize);
|
|
@@ -2606,8 +2676,9 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
|
|
|
long writeTimeout = socketWriteTimeout +
|
|
long writeTimeout = socketWriteTimeout +
|
|
WRITE_TIMEOUT_EXTENSION * (targets.length-1);
|
|
WRITE_TIMEOUT_EXTENSION * (targets.length-1);
|
|
- out = new DataOutputStream(new BufferedOutputStream(
|
|
|
|
- NetUtils.getOutputStream(sock, writeTimeout), SMALL_BUFFER_SIZE));
|
|
|
|
|
|
+ OutputStream baseStream = NetUtils.getOutputStream(sock, writeTimeout);
|
|
|
|
+ out = new DataOutputStream(new BufferedOutputStream(baseStream,
|
|
|
|
+ SMALL_BUFFER_SIZE));
|
|
|
|
|
|
blockSender = new BlockSender(b, 0, -1, false, false, false);
|
|
blockSender = new BlockSender(b, 0, -1, false, false, false);
|
|
|
|
|
|
@@ -2626,7 +2697,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
targets[i].write(out);
|
|
targets[i].write(out);
|
|
}
|
|
}
|
|
// send data & checksum
|
|
// send data & checksum
|
|
- blockSender.sendBlock(out, null);
|
|
|
|
|
|
+ blockSender.sendBlock(out, baseStream, null);
|
|
|
|
|
|
// no response necessary
|
|
// no response necessary
|
|
LOG.info(dnRegistration + ":Transmitted block " + b + " to " + curTarget);
|
|
LOG.info(dnRegistration + ":Transmitted block " + b + " to " + curTarget);
|