You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/11/12 05:55:46 UTC

svn commit: r1540946 - /hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java

Author: stack
Date: Tue Nov 12 04:55:46 2013
New Revision: 1540946

URL: http://svn.apache.org/r1540946
Log:
HBASE-9917 Fix it so Default Connection Pool does not spin up max threads even when not needed

Modified:
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1540946&r1=1540945&r2=1540946&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Tue Nov 12 04:55:46 2013
@@ -719,22 +719,22 @@ public class HConnectionManager {
         synchronized (this) {
           if (batchPool == null) {
             int maxThreads = conf.getInt("hbase.hconnection.threads.max", 256);
+            int coreThreads = conf.getInt("hbase.hconnection.threads.core", 0);
             if (maxThreads == 0) {
               maxThreads = Runtime.getRuntime().availableProcessors() * 8;
             }
-            long keepAliveTime = conf.getLong(
-                "hbase.hconnection.threads.keepalivetime", 60);
+            long keepAliveTime = conf.getLong("hbase.hconnection.threads.keepalivetime", 10);
             LinkedBlockingQueue<Runnable> workQueue =
-                new LinkedBlockingQueue<Runnable>(256 *
-                    conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
-                    HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS));
+              new LinkedBlockingQueue<Runnable>(maxThreads *
+                conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
+                  HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS));
             this.batchPool = new ThreadPoolExecutor(
-                maxThreads,
+                coreThreads,
                 maxThreads,
                 keepAliveTime,
                 TimeUnit.SECONDS,
                 workQueue,
-                Threads.newDaemonThreadFactory("hbase-connection-shared-executor"));
+                Threads.newDaemonThreadFactory(toString() + "-shared-"));
           }
           this.cleanupPool = true;
         }