|
@@ -46,6 +46,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_DEF
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_METRICS_LOGGER_PERIOD_SECONDS_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_METRICS_LOGGER_PERIOD_SECONDS_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage.PIPELINE_SETUP_APPEND_RECOVERY;
|
|
|
+import static org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage.PIPELINE_SETUP_CREATE;
|
|
|
+import static org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage.PIPELINE_SETUP_STREAMING_RECOVERY;
|
|
|
import static org.apache.hadoop.util.ExitUtil.terminate;
|
|
|
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
@@ -2500,8 +2503,8 @@ public class DataNode extends ReconfigurableBase
|
|
|
final String clientname;
|
|
|
final CachingStrategy cachingStrategy;
|
|
|
|
|
|
- /** Throttle to block replication when data transfers. */
|
|
|
- private DataTransferThrottler transferThrottler;
|
|
|
+ /** Throttle to block replication when data transfers or writes. */
|
|
|
+ private DataTransferThrottler throttler;
|
|
|
|
|
|
/**
|
|
|
* Connect to the first item in the target list. Pass along the
|
|
@@ -2529,14 +2532,10 @@ public class DataNode extends ReconfigurableBase
|
|
|
this.clientname = clientname;
|
|
|
this.cachingStrategy =
|
|
|
new CachingStrategy(true, getDnConf().readaheadLength);
|
|
|
- // 1. the stage is PIPELINE_SETUP_CREATE,that is moving blocks, set
|
|
|
- // throttler.
|
|
|
- // 2. the stage is PIPELINE_SETUP_APPEND_RECOVERY or
|
|
|
- // PIPELINE_SETUP_STREAMING_RECOVERY,
|
|
|
- // that is writing and recovering pipeline, don't set throttle.
|
|
|
- if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE
|
|
|
- && clientname.isEmpty()) {
|
|
|
- this.transferThrottler = xserver.getTransferThrottler();
|
|
|
+ if (isTransfer(stage, clientname)) {
|
|
|
+ this.throttler = xserver.getTransferThrottler();
|
|
|
+ } else if(isWrite(stage)) {
|
|
|
+ this.throttler = xserver.getWriteThrottler();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -2596,7 +2595,7 @@ public class DataNode extends ReconfigurableBase
|
|
|
targetStorageIds);
|
|
|
|
|
|
// send data & checksum
|
|
|
- blockSender.sendBlock(out, unbufOut, transferThrottler);
|
|
|
+ blockSender.sendBlock(out, unbufOut, throttler);
|
|
|
|
|
|
// no response necessary
|
|
|
LOG.info("{}, at {}: Transmitted {} (numBytes={}) to {}",
|
|
@@ -3739,4 +3738,32 @@ public class DataNode extends ReconfigurableBase
|
|
|
}
|
|
|
return this.diskBalancer;
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Construct DataTransfer in {@link DataNode#transferBlock}, the
|
|
|
+ * BlockConstructionStage is PIPELINE_SETUP_CREATE and clientName is "".
|
|
|
+ */
|
|
|
+ private static boolean isTransfer(BlockConstructionStage stage,
|
|
|
+ String clientName) {
|
|
|
+ if (stage == PIPELINE_SETUP_CREATE && clientName.isEmpty()) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Construct DataTransfer in
|
|
|
+ * {@link DataNode#transferReplicaForPipelineRecovery}.
|
|
|
+ *
|
|
|
+ * When recover pipeline, BlockConstructionStage is
|
|
|
+ * PIPELINE_SETUP_APPEND_RECOVERY,
|
|
|
+ * PIPELINE_SETUP_STREAMING_RECOVERY,PIPELINE_CLOSE_RECOVERY. If
|
|
|
+ * BlockConstructionStage is PIPELINE_CLOSE_RECOVERY, don't need transfer
|
|
|
+ * replica. So BlockConstructionStage is PIPELINE_SETUP_APPEND_RECOVERY,
|
|
|
+ * PIPELINE_SETUP_STREAMING_RECOVERY.
|
|
|
+ */
|
|
|
+ private static boolean isWrite(BlockConstructionStage stage) {
|
|
|
+ return (stage == PIPELINE_SETUP_STREAMING_RECOVERY
|
|
|
+ || stage == PIPELINE_SETUP_APPEND_RECOVERY);
|
|
|
+ }
|
|
|
}
|