From bec844006e885d07b6d1bf180540098946c37ba5 Mon Sep 17 00:00:00 2001 From: Elliott Clark Date: Thu, 19 Sep 2013 15:14:20 -0700 Subject: [PATCH] HBASE-9333 hbase.hconnection.threads.max should not be configurable else you get RejectedExecutionException --- .../apache/hadoop/hbase/client/HConnectionManager.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java index 5cd3742..df10ef7 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java @@ -36,7 +36,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CopyOnWriteArraySet; import java.util.concurrent.ExecutorService; -import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -753,22 +753,22 @@ public class HConnectionManager { // shared HTable thread executor not yet initialized synchronized (this) { if (batchPool == null) { - int maxThreads = conf.getInt("hbase.hconnection.threads.max", - Integer.MAX_VALUE); + int maxThreads = conf.getInt("hbase.hconnection.threads.max", 256); if (maxThreads == 0) { - maxThreads = Runtime.getRuntime().availableProcessors(); + maxThreads = Runtime.getRuntime().availableProcessors() * 8; } long keepAliveTime = conf.getLong( "hbase.hconnection.threads.keepalivetime", 60); + LinkedBlockingQueue workQueue = + new LinkedBlockingQueue(128 * + conf.getInt("hbase.client.max.total.tasks", 200)); this.batchPool = new ThreadPoolExecutor( - Runtime.getRuntime().availableProcessors(), + maxThreads, maxThreads, keepAliveTime, TimeUnit.SECONDS, - new SynchronousQueue(), + workQueue, Threads.newDaemonThreadFactory("hbase-connection-shared-executor")); - ((ThreadPoolExecutor) this.batchPool) - .allowCoreThreadTimeOut(true); } this.cleanupPool = true; } -- 1.7.10.2 (Apple Git-33)