|
@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
|
|
@@ -123,6 +124,14 @@ class BlockReceiver implements Closeable {
|
|
|
private boolean syncOnClose;
|
|
|
private long restartBudget;
|
|
|
|
|
|
+ /**
|
|
|
+ * for replaceBlock response
|
|
|
+ */
|
|
|
+ private final long responseInterval;
|
|
|
+ private long lastResponseTime = 0;
|
|
|
+ private boolean isReplaceBlock = false;
|
|
|
+ private DataOutputStream replyOut = null;
|
|
|
+
|
|
|
BlockReceiver(final ExtendedBlock block, final StorageType storageType,
|
|
|
final DataInputStream in,
|
|
|
final String inAddr, final String myAddr,
|
|
@@ -144,6 +153,9 @@ class BlockReceiver implements Closeable {
|
|
|
this.isClient = !this.isDatanode;
|
|
|
this.restartBudget = datanode.getDnConf().restartReplicaExpiry;
|
|
|
this.datanodeSlowLogThresholdMs = datanode.getDnConf().datanodeSlowIoWarningThresholdMs;
|
|
|
+ // For replaceBlock() calls response should be sent to avoid socketTimeout
|
|
|
+ // at clients. So sending with the interval of 0.5 * socketTimeout
|
|
|
+ this.responseInterval = (long) (datanode.getDnConf().socketTimeout * 0.5);
|
|
|
//for datanode, we have
|
|
|
//1: clientName.length() == 0, and
|
|
|
//2: stage == null or PIPELINE_SETUP_CREATE
|
|
@@ -651,6 +663,20 @@ class BlockReceiver implements Closeable {
|
|
|
lastPacketInBlock, offsetInBlock, Status.SUCCESS);
|
|
|
}
|
|
|
|
|
|
+ /*
|
|
|
+ * Send in-progress responses for the replaceBlock() calls back to caller to
|
|
|
+ * avoid timeouts due to balancer throttling. HDFS-6247
|
|
|
+ */
|
|
|
+ if (isReplaceBlock
|
|
|
+ && (Time.monotonicNow() - lastResponseTime > responseInterval)) {
|
|
|
+ BlockOpResponseProto.Builder response = BlockOpResponseProto.newBuilder()
|
|
|
+ .setStatus(Status.IN_PROGRESS);
|
|
|
+ response.build().writeDelimitedTo(replyOut);
|
|
|
+ replyOut.flush();
|
|
|
+
|
|
|
+ lastResponseTime = Time.monotonicNow();
|
|
|
+ }
|
|
|
+
|
|
|
if (throttler != null) { // throttle I/O
|
|
|
throttler.throttle(len);
|
|
|
}
|
|
@@ -718,7 +744,8 @@ class BlockReceiver implements Closeable {
|
|
|
DataInputStream mirrIn, // input from next datanode
|
|
|
DataOutputStream replyOut, // output to previous datanode
|
|
|
String mirrAddr, DataTransferThrottler throttlerArg,
|
|
|
- DatanodeInfo[] downstreams) throws IOException {
|
|
|
+ DatanodeInfo[] downstreams,
|
|
|
+ boolean isReplaceBlock) throws IOException {
|
|
|
|
|
|
syncOnClose = datanode.getDnConf().syncOnClose;
|
|
|
boolean responderClosed = false;
|
|
@@ -726,6 +753,9 @@ class BlockReceiver implements Closeable {
|
|
|
mirrorAddr = mirrAddr;
|
|
|
throttler = throttlerArg;
|
|
|
|
|
|
+ this.replyOut = replyOut;
|
|
|
+ this.isReplaceBlock = isReplaceBlock;
|
|
|
+
|
|
|
try {
|
|
|
if (isClient && !isTransfer) {
|
|
|
responder = new Daemon(datanode.threadGroup,
|