|
@@ -45,7 +45,6 @@ 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;
|
|
@@ -124,14 +123,6 @@ 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,
|
|
@@ -153,9 +144,6 @@ 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
|
|
@@ -663,20 +651,6 @@ 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);
|
|
|
}
|
|
@@ -744,8 +718,7 @@ class BlockReceiver implements Closeable {
|
|
|
DataInputStream mirrIn, // input from next datanode
|
|
|
DataOutputStream replyOut, // output to previous datanode
|
|
|
String mirrAddr, DataTransferThrottler throttlerArg,
|
|
|
- DatanodeInfo[] downstreams,
|
|
|
- boolean isReplaceBlock) throws IOException {
|
|
|
+ DatanodeInfo[] downstreams) throws IOException {
|
|
|
|
|
|
syncOnClose = datanode.getDnConf().syncOnClose;
|
|
|
boolean responderClosed = false;
|
|
@@ -753,9 +726,6 @@ class BlockReceiver implements Closeable {
|
|
|
mirrorAddr = mirrAddr;
|
|
|
throttler = throttlerArg;
|
|
|
|
|
|
- this.replyOut = replyOut;
|
|
|
- this.isReplaceBlock = isReplaceBlock;
|
|
|
-
|
|
|
try {
|
|
|
if (isClient && !isTransfer) {
|
|
|
responder = new Daemon(datanode.threadGroup,
|