HBASE-9333 hbase.hconnection.threads.max should not be configurable else you get RejectedExecutionException

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1528618 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
eclark 2013-10-02 20:48:33 +00:00
parent ec45fc299f
commit de49d0d1ed
1 changed files with 8 additions and 8 deletions

View File

@ -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;
@ -712,22 +712,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<Runnable> workQueue =
new LinkedBlockingQueue<Runnable>(128 *
conf.getInt("hbase.client.max.total.tasks", 200));
this.batchPool = new ThreadPoolExecutor(
Runtime.getRuntime().availableProcessors(),
maxThreads,
maxThreads,
keepAliveTime,
TimeUnit.SECONDS,
new SynchronousQueue<Runnable>(),
workQueue,
Threads.newDaemonThreadFactory("hbase-connection-shared-executor"));
((ThreadPoolExecutor) this.batchPool)
.allowCoreThreadTimeOut(true);
}
this.cleanupPool = true;
}