浏览代码

svn merge -c 1503720 from trunk for HDFS-4992. Make balancer's mover thread count and dispatcher thread count configurable.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.1.0-beta@1503726 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 年之前
父节点
当前提交
d48d65feb9

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -188,6 +188,9 @@ Release 2.1.0-beta - 2013-07-02
     HDFS-4645. Move from randomly generated block ID to sequentially generated
     block ID.  (Arpit Agarwal via szetszwo)
 
+    HDFS-4992. Make balancer's mover thread count and dispatcher thread count
+    configurable.  (Max Lapan via szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

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

@@ -276,6 +276,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT = false;
   public static final String  DFS_BALANCER_MOVEDWINWIDTH_KEY = "dfs.balancer.movedWinWidth";
   public static final long    DFS_BALANCER_MOVEDWINWIDTH_DEFAULT = 5400*1000L;
+  public static final String  DFS_BALANCER_MOVERTHREADS_KEY = "dfs.balancer.moverThreads";
+  public static final int     DFS_BALANCER_MOVERTHREADS_DEFAULT = 1000;
+  public static final String  DFS_BALANCER_DISPATCHERTHREADS_KEY = "dfs.balancer.dispatcherThreads";
+  public static final int     DFS_BALANCER_DISPATCHERTHREADS_DEFAULT = 200;
   public static final String  DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
   public static final int     DFS_DATANODE_DEFAULT_PORT = 50010;
   public static final String  DFS_DATANODE_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_DATANODE_DEFAULT_PORT;

+ 10 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -226,13 +226,9 @@ public class Balancer {
                  = new HashMap<String, BalancerDatanode>();
   
   private NetworkTopology cluster;
-  final static private int MOVER_THREAD_POOL_SIZE = 1000;
-  final private ExecutorService moverExecutor = 
-    Executors.newFixedThreadPool(MOVER_THREAD_POOL_SIZE);
-  final static private int DISPATCHER_THREAD_POOL_SIZE = 200;
-  final private ExecutorService dispatcherExecutor =
-    Executors.newFixedThreadPool(DISPATCHER_THREAD_POOL_SIZE);
-  
+
+  private final ExecutorService moverExecutor;
+  private final ExecutorService dispatcherExecutor;
 
   /* This class keeps track of a scheduled block move */
   private class PendingBlockMove {
@@ -830,6 +826,13 @@ public class Balancer {
     this.policy = p.policy;
     this.nnc = theblockpool;
     cluster = NetworkTopology.getInstance(conf);
+
+    this.moverExecutor = Executors.newFixedThreadPool(
+            conf.getInt(DFSConfigKeys.DFS_BALANCER_MOVERTHREADS_KEY,
+                        DFSConfigKeys.DFS_BALANCER_MOVERTHREADS_DEFAULT));
+    this.dispatcherExecutor = Executors.newFixedThreadPool(
+            conf.getInt(DFSConfigKeys.DFS_BALANCER_DISPATCHERTHREADS_KEY,
+                        DFSConfigKeys.DFS_BALANCER_DISPATCHERTHREADS_DEFAULT));
   }
   
   /* Shuffle datanode array */