You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nk...@apache.org on 2014/02/03 14:20:05 UTC

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

Author: nkeywal
Date: Mon Feb  3 13:20:04 2014
New Revision: 1563879

URL: http://svn.apache.org/r1563879
Log:
HBASE-10449 Wrong execution pool configuration in HConnectionManager

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=1563879&r1=1563878&r2=1563879&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 Mon Feb  3 13:20:04 2014
@@ -718,22 +718,27 @@ 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);
+            int coreThreads = conf.getInt("hbase.hconnection.threads.core", 256);
             if (maxThreads == 0) {
               maxThreads = Runtime.getRuntime().availableProcessors() * 8;
             }
-            long keepAliveTime = conf.getLong("hbase.hconnection.threads.keepalivetime", 10);
+            if (coreThreads == 0) {
+              coreThreads = Runtime.getRuntime().availableProcessors() * 8;
+            }
+            long keepAliveTime = conf.getLong("hbase.hconnection.threads.keepalivetime", 60);
             LinkedBlockingQueue<Runnable> workQueue =
               new LinkedBlockingQueue<Runnable>(maxThreads *
                 conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
                   HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS));
-            this.batchPool = new ThreadPoolExecutor(
+            ThreadPoolExecutor tpe = new ThreadPoolExecutor(
                 coreThreads,
                 maxThreads,
                 keepAliveTime,
                 TimeUnit.SECONDS,
                 workQueue,
                 Threads.newDaemonThreadFactory(toString() + "-shared-"));
+            tpe.allowCoreThreadTimeOut(true);
+            this.batchPool = tpe;
           }
           this.cleanupPool = true;
         }