Index: src/main/java/org/apache/hadoop/hbase/client/HTable.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/client/HTable.java (revision 1091966) +++ src/main/java/org/apache/hadoop/hbase/client/HTable.java (working copy) @@ -30,7 +30,7 @@ import java.util.Map; import java.util.TreeMap; import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -54,8 +54,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.zookeeper.KeeperException; /** * Used to communicate with a single HBase table. @@ -180,18 +178,18 @@ HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE); this.maxKeyValueSize = conf.getInt("hbase.client.keyvalue.maxsize", -1); - int nrThreads = conf.getInt("hbase.htable.threads.max", getCurrentNrHRS()); - if (nrThreads == 0) { - nrThreads = 1; // is there a better default? + int maxThreads = conf.getInt("hbase.htable.threads.max", Integer.MAX_VALUE); + if (maxThreads == 0) { + maxThreads = 1; // is there a better default? } - // Unfortunately Executors.newCachedThreadPool does not allow us to - // set the maximum size of the pool, so we have to do it ourselves. - // Must also set set corethreadpool size as with a LinkedBlockingQueue, - // a new thread will not be started until the queue is full - this.pool = new ThreadPoolExecutor(nrThreads, nrThreads, + // Using the "direct handoff" approach, new threads will only be created + // if it is necessary and will grow unbounded. This could be bad but in HCM + // we only create as many Runnables as there are region servers. It means + // it also scales when new region servers are added. + this.pool = new ThreadPoolExecutor(1, maxThreads, 60, TimeUnit.SECONDS, - new LinkedBlockingQueue(), + new SynchronousQueue(), new DaemonThreadFactory()); ((ThreadPoolExecutor)this.pool).allowCoreThreadTimeOut(true); } @@ -201,21 +199,6 @@ } /** - * @return the number of region servers that are currently running - * @throws IOException if a remote or network exception occurs - */ - public int getCurrentNrHRS() throws IOException { - try { - // We go to zk rather than to master to get count of regions to avoid - // HTable having a Master dependency. See HBase-2828 - return ZKUtil.getNumberOfChildren(this.connection.getZooKeeperWatcher(), - this.connection.getZooKeeperWatcher().rsZNode); - } catch (KeeperException ke) { - throw new IOException("Unexpected ZooKeeper exception", ke); - } - } - - /** * Tells whether or not a table is enabled or not. * Warning: use {@link HBaseAdmin#isTableEnabled(byte[])} instead. * @param tableName Name of table to check.