You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2013/10/02 22:48:50 UTC
svn commit: r1528619 -
/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
Author: eclark
Date: Wed Oct 2 20:48:50 2013
New Revision: 1528619
URL: http://svn.apache.org/r1528619
Log:
HBASE-9333 hbase.hconnection.threads.max should not be configurable else you get RejectedExecutionException
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=1528619&r1=1528618&r2=1528619&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 Wed Oct 2 20:48:50 2013
@@ -36,7 +36,7 @@ import java.util.concurrent.ConcurrentHa
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;
}