diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java index ac76e5a..07decb2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java @@ -716,22 +716,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", 8); 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 workQueue = - new LinkedBlockingQueue(256 * - conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, - HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS)); + new LinkedBlockingQueue(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; }