You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sz...@apache.org on 2013/07/16 16:14:43 UTC

svn commit: r1503726 - in /hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs: ./ CHANGES.txt src/main/java/ src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

Author: szetszwo
Date: Tue Jul 16 14:14:43 2013
New Revision: 1503726

URL: http://svn.apache.org/r1503726
Log:
svn merge -c 1503720 from trunk for HDFS-4992. Make balancer's mover thread count and dispatcher thread count configurable.

Modified:
    hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/   (props changed)
    hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/src/main/java/   (props changed)
    hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
    hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

Propchange: hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs:r1503720

Modified: hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1503726&r1=1503725&r2=1503726&view=diff
==============================================================================
--- hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Tue Jul 16 14:14:43 2013
@@ -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

Propchange: hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/src/main/java/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java:r1503720

Modified: hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java?rev=1503726&r1=1503725&r2=1503726&view=diff
==============================================================================
--- hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java (original)
+++ hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java Tue Jul 16 14:14:43 2013
@@ -276,6 +276,10 @@ public class DFSConfigKeys extends Commo
   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;

Modified: hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java?rev=1503726&r1=1503725&r2=1503726&view=diff
==============================================================================
--- hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java (original)
+++ hadoop/common/branches/branch-2.1.0-beta/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java Tue Jul 16 14:14:43 2013
@@ -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 */