Explorar o código

HDFS-12904. Add DataTransferThrottler to the Datanode transfers. Contributed by Lisheng Sun.

Inigo Goiri %!s(int64=5) %!d(string=hai) anos
pai
achega
72d8b92ba5

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -115,6 +115,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       = "dfs.datanode.balance.max.concurrent.moves";
   public static final int
       DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT = 100;
+  public static final String DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY =
+      "dfs.datanode.data.transfer.bandwidthPerSec";
+  public static final long DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_DEFAULT =
+      0; // A value of zero indicates no limit
   @Deprecated
   public static final String  DFS_DATANODE_READAHEAD_BYTES_KEY =
       HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_KEY;

+ 14 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -112,6 +112,7 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.datanode.checker.DatasetVolumeChecker;
 import org.apache.hadoop.hdfs.server.datanode.checker.StorageLocationChecker;
+import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.client.BlockReportOptions;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -2499,6 +2500,9 @@ public class DataNode extends ReconfigurableBase
     final String clientname;
     final CachingStrategy cachingStrategy;
 
+    /** Throttle to block replication when data transfers. */
+    private DataTransferThrottler transferThrottler;
+
     /**
      * Connect to the first item in the target list.  Pass along the 
      * entire target list, the block, and the data.
@@ -2525,6 +2529,15 @@ 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();
+      }
     }
 
     /**
@@ -2583,7 +2596,7 @@ public class DataNode extends ReconfigurableBase
             targetStorageIds);
 
         // send data & checksum
-        blockSender.sendBlock(out, unbufOut, null);
+        blockSender.sendBlock(out, unbufOut, transferThrottler);
 
         // no response necessary
         LOG.info("{}, at {}: Transmitted {} (numBytes={}) to {}",

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java

@@ -169,6 +169,8 @@ class DataXceiverServer implements Runnable {
 
   final BlockBalanceThrottler balanceThrottler;
 
+  private final DataTransferThrottler transferThrottler;
+
   /**
    * Stores an estimate for block size to check if the disk partition has enough
    * space. Newer clients pass the expected block size to the DataNode. For
@@ -194,6 +196,15 @@ class DataXceiverServer implements Runnable {
             DFSConfigKeys.DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_DEFAULT),
         conf.getInt(DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY,
             DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT));
+
+    long bandwidthPerSec = conf.getLongBytes(
+        DFSConfigKeys.DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_KEY,
+        DFSConfigKeys.DFS_DATANODE_DATA_TRANSFER_BANDWIDTHPERSEC_DEFAULT);
+    if (bandwidthPerSec > 0) {
+      this.transferThrottler = new DataTransferThrottler(bandwidthPerSec);
+    } else {
+      this.transferThrottler = null;
+    }
   }
 
   @Override
@@ -443,6 +454,10 @@ class DataXceiverServer implements Runnable {
     return peerServer;
   }
 
+  public DataTransferThrottler getTransferThrottler() {
+    return transferThrottler;
+  }
+
   /**
    * Release a peer.
    *

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -4144,6 +4144,16 @@
   </description>
 </property>
 
+  <property>
+    <name>dfs.datanode.data.transfer.bandwidthPerSec</name>
+    <value>0</value>
+    <description>
+      Specifies the maximum amount of bandwidth that each datanode can utilize for the data transfering purpose in term
+      of the number of bytes per second.
+      when the bandwidth value is zero, there is no limit.
+    </description>
+  </property>
+
 <property>
   <name>dfs.datanode.fsdataset.factory</name>
   <value></value>