|
@@ -81,6 +81,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
private SocketFactory socketFactory;
|
|
private SocketFactory socketFactory;
|
|
private int socketTimeout;
|
|
private int socketTimeout;
|
|
private int datanodeWriteTimeout;
|
|
private int datanodeWriteTimeout;
|
|
|
|
+ private int timeoutValue; // read timeout for the socket
|
|
final int writePacketSize;
|
|
final int writePacketSize;
|
|
private final FileSystem.Statistics stats;
|
|
private final FileSystem.Statistics stats;
|
|
private int maxBlockAcquireFailures;
|
|
private int maxBlockAcquireFailures;
|
|
@@ -189,6 +190,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
HdfsConstants.READ_TIMEOUT);
|
|
HdfsConstants.READ_TIMEOUT);
|
|
this.datanodeWriteTimeout = conf.getInt("dfs.datanode.socket.write.timeout",
|
|
this.datanodeWriteTimeout = conf.getInt("dfs.datanode.socket.write.timeout",
|
|
HdfsConstants.WRITE_TIMEOUT);
|
|
HdfsConstants.WRITE_TIMEOUT);
|
|
|
|
+ this.timeoutValue = this.socketTimeout;
|
|
this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
|
|
this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
|
|
// dfs.write.packet.size is an internal config variable
|
|
// dfs.write.packet.size is an internal config variable
|
|
this.writePacketSize = conf.getInt("dfs.write.packet.size", 64*1024);
|
|
this.writePacketSize = conf.getInt("dfs.write.packet.size", 64*1024);
|
|
@@ -2205,7 +2207,28 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
int dataPos;
|
|
int dataPos;
|
|
int checksumStart;
|
|
int checksumStart;
|
|
int checksumPos;
|
|
int checksumPos;
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ private static final long HEART_BEAT_SEQNO = -1L;
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * create a heartbeat packet
|
|
|
|
+ */
|
|
|
|
+ Packet() {
|
|
|
|
+ this.lastPacketInBlock = false;
|
|
|
|
+ this.numChunks = 0;
|
|
|
|
+ this.offsetInBlock = 0;
|
|
|
|
+ this.seqno = HEART_BEAT_SEQNO;
|
|
|
|
+
|
|
|
|
+ buffer = null;
|
|
|
|
+ int packetSize = DataNode.PKT_HEADER_LEN + SIZE_OF_INTEGER;
|
|
|
|
+ buf = new byte[packetSize];
|
|
|
|
+
|
|
|
|
+ checksumStart = dataStart = packetSize;
|
|
|
|
+ checksumPos = checksumStart;
|
|
|
|
+ dataPos = dataStart;
|
|
|
|
+ maxChunks = 0;
|
|
|
|
+ }
|
|
|
|
+
|
|
// create a new packet
|
|
// create a new packet
|
|
Packet(int pktSize, int chunksPerPkt, long offsetInBlock) {
|
|
Packet(int pktSize, int chunksPerPkt, long offsetInBlock) {
|
|
this.lastPacketInBlock = false;
|
|
this.lastPacketInBlock = false;
|
|
@@ -2286,6 +2309,14 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
buffer.reset();
|
|
buffer.reset();
|
|
return buffer;
|
|
return buffer;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Check if this packet is a heart beat packet
|
|
|
|
+ * @return true if the sequence number is HEART_BEAT_SEQNO
|
|
|
|
+ */
|
|
|
|
+ private boolean isHeartbeatPacket() {
|
|
|
|
+ return seqno == HEART_BEAT_SEQNO;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
//
|
|
//
|
|
@@ -2301,6 +2332,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
private volatile boolean closed = false;
|
|
private volatile boolean closed = false;
|
|
|
|
|
|
public void run() {
|
|
public void run() {
|
|
|
|
+ long lastPacket = 0;
|
|
|
|
+
|
|
while (!closed && clientRunning) {
|
|
while (!closed && clientRunning) {
|
|
|
|
|
|
// if the Responder encountered an error, shutdown Responder
|
|
// if the Responder encountered an error, shutdown Responder
|
|
@@ -2320,23 +2353,36 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
boolean doSleep = processDatanodeError(hasError, false);
|
|
boolean doSleep = processDatanodeError(hasError, false);
|
|
|
|
|
|
// wait for a packet to be sent.
|
|
// wait for a packet to be sent.
|
|
|
|
+ long now = System.currentTimeMillis();
|
|
while ((!closed && !hasError && clientRunning
|
|
while ((!closed && !hasError && clientRunning
|
|
- && dataQueue.size() == 0) || doSleep) {
|
|
|
|
|
|
+ && dataQueue.size() == 0 &&
|
|
|
|
+ (blockStream == null || (
|
|
|
|
+ blockStream != null && now - lastPacket < timeoutValue/2)))
|
|
|
|
+ || doSleep) {
|
|
|
|
+ long timeout = timeoutValue/2 - (now-lastPacket);
|
|
|
|
+ timeout = timeout <= 0 ? 1000 : timeout;
|
|
|
|
+
|
|
try {
|
|
try {
|
|
- dataQueue.wait(1000);
|
|
|
|
|
|
+ dataQueue.wait(timeout);
|
|
|
|
+ now = System.currentTimeMillis();
|
|
} catch (InterruptedException e) {
|
|
} catch (InterruptedException e) {
|
|
}
|
|
}
|
|
doSleep = false;
|
|
doSleep = false;
|
|
}
|
|
}
|
|
- if (closed || hasError || dataQueue.size() == 0 || !clientRunning) {
|
|
|
|
|
|
+ if (closed || hasError || !clientRunning) {
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
|
|
|
|
try {
|
|
try {
|
|
// get packet to be sent.
|
|
// get packet to be sent.
|
|
- one = dataQueue.getFirst();
|
|
|
|
|
|
+ if (dataQueue.isEmpty()) {
|
|
|
|
+ one = new Packet(); // heartbeat packet
|
|
|
|
+ } else {
|
|
|
|
+ one = dataQueue.getFirst(); // regular data packet
|
|
|
|
+ }
|
|
|
|
+
|
|
long offsetInBlock = one.offsetInBlock;
|
|
long offsetInBlock = one.offsetInBlock;
|
|
-
|
|
|
|
|
|
+
|
|
// get new block from namenode.
|
|
// get new block from namenode.
|
|
if (blockStream == null) {
|
|
if (blockStream == null) {
|
|
LOG.debug("Allocating new block");
|
|
LOG.debug("Allocating new block");
|
|
@@ -2358,12 +2404,14 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
ByteBuffer buf = one.getBuffer();
|
|
ByteBuffer buf = one.getBuffer();
|
|
|
|
|
|
// move packet from dataQueue to ackQueue
|
|
// move packet from dataQueue to ackQueue
|
|
- dataQueue.removeFirst();
|
|
|
|
- dataQueue.notifyAll();
|
|
|
|
- synchronized (ackQueue) {
|
|
|
|
- ackQueue.addLast(one);
|
|
|
|
- ackQueue.notifyAll();
|
|
|
|
- }
|
|
|
|
|
|
+ if (!one.isHeartbeatPacket()) {
|
|
|
|
+ dataQueue.removeFirst();
|
|
|
|
+ dataQueue.notifyAll();
|
|
|
|
+ synchronized (ackQueue) {
|
|
|
|
+ ackQueue.addLast(one);
|
|
|
|
+ ackQueue.notifyAll();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
|
|
// write out data to remote datanode
|
|
// write out data to remote datanode
|
|
blockStream.write(buf.array(), buf.position(), buf.remaining());
|
|
blockStream.write(buf.array(), buf.position(), buf.remaining());
|
|
@@ -2372,6 +2420,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
blockStream.writeInt(0); // indicate end-of-block
|
|
blockStream.writeInt(0); // indicate end-of-block
|
|
}
|
|
}
|
|
blockStream.flush();
|
|
blockStream.flush();
|
|
|
|
+ lastPacket = System.currentTimeMillis();
|
|
|
|
+
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("DataStreamer block " + block +
|
|
LOG.debug("DataStreamer block " + block +
|
|
" wrote packet seqno:" + one.seqno +
|
|
" wrote packet seqno:" + one.seqno +
|
|
@@ -2480,38 +2530,37 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("DFSClient " + ack);
|
|
LOG.debug("DFSClient " + ack);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // processes response status from all datanodes.
|
|
|
|
+ for (int i = ack.getNumOfReplies()-1; i >=0 && clientRunning; i--) {
|
|
|
|
+ short reply = ack.getReply(i);
|
|
|
|
+ if (reply != DataTransferProtocol.OP_STATUS_SUCCESS) {
|
|
|
|
+ errorIndex = i; // first bad datanode
|
|
|
|
+ throw new IOException("Bad response " + reply +
|
|
|
|
+ " for block " + block +
|
|
|
|
+ " from datanode " +
|
|
|
|
+ targets[i].getName());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
long seqno = ack.getSeqno();
|
|
long seqno = ack.getSeqno();
|
|
- if (seqno == PipelineAck.HEART_BEAT.getSeqno()) {
|
|
|
|
|
|
+ assert seqno != PipelineAck.UNKOWN_SEQNO :
|
|
|
|
+ "Ack for unkown seqno should be a failed ack: " + ack;
|
|
|
|
+ if (seqno == Packet.HEART_BEAT_SEQNO) { // a heartbeat ack
|
|
continue;
|
|
continue;
|
|
- } else if (seqno == -2) {
|
|
|
|
- // This signifies that some pipeline node failed to read downstream
|
|
|
|
- // and therefore has no idea what sequence number the message corresponds
|
|
|
|
- // to. So, we don't try to match it up with an ack.
|
|
|
|
- assert ! ack.isSuccess();
|
|
|
|
- } else {
|
|
|
|
- Packet one = null;
|
|
|
|
- synchronized (ackQueue) {
|
|
|
|
- one = ackQueue.getFirst();
|
|
|
|
- }
|
|
|
|
- if (one.seqno != seqno) {
|
|
|
|
- throw new IOException("Responseprocessor: Expecting seqno " +
|
|
|
|
- " for block " + block +
|
|
|
|
- one.seqno + " but received " + seqno);
|
|
|
|
- }
|
|
|
|
- lastPacketInBlock = one.lastPacketInBlock;
|
|
|
|
}
|
|
}
|
|
|
|
|
|
- // processes response status from all datanodes.
|
|
|
|
- for (int i = 0; i < targets.length && clientRunning; i++) {
|
|
|
|
- short reply = ack.getReply(i);
|
|
|
|
- if (reply != DataTransferProtocol.OP_STATUS_SUCCESS) {
|
|
|
|
- errorIndex = i; // first bad datanode
|
|
|
|
- throw new IOException("Bad response " + reply +
|
|
|
|
- " for block " + block +
|
|
|
|
- " from datanode " +
|
|
|
|
- targets[i].getName());
|
|
|
|
- }
|
|
|
|
|
|
+ Packet one = null;
|
|
|
|
+ synchronized (ackQueue) {
|
|
|
|
+ one = ackQueue.getFirst();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (one.seqno != seqno) {
|
|
|
|
+ throw new IOException("Responseprocessor: Expecting seqno " +
|
|
|
|
+ " for block " + block + " " +
|
|
|
|
+ one.seqno + " but received " + seqno);
|
|
}
|
|
}
|
|
|
|
+ lastPacketInBlock = one.lastPacketInBlock;
|
|
|
|
|
|
synchronized (ackQueue) {
|
|
synchronized (ackQueue) {
|
|
ackQueue.removeFirst();
|
|
ackQueue.removeFirst();
|
|
@@ -2931,7 +2980,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
LOG.debug("Connecting to " + nodes[0].getName());
|
|
LOG.debug("Connecting to " + nodes[0].getName());
|
|
InetSocketAddress target = NetUtils.createSocketAddr(nodes[0].getName());
|
|
InetSocketAddress target = NetUtils.createSocketAddr(nodes[0].getName());
|
|
s = socketFactory.createSocket();
|
|
s = socketFactory.createSocket();
|
|
- int timeoutValue = 3000 * nodes.length + socketTimeout;
|
|
|
|
|
|
+ timeoutValue = 3000 * nodes.length + socketTimeout;
|
|
NetUtils.connect(s, target, timeoutValue);
|
|
NetUtils.connect(s, target, timeoutValue);
|
|
s.setSoTimeout(timeoutValue);
|
|
s.setSoTimeout(timeoutValue);
|
|
s.setSendBufferSize(DEFAULT_DATA_SOCKET_SIZE);
|
|
s.setSendBufferSize(DEFAULT_DATA_SOCKET_SIZE);
|