HBASE-14654 Reenable TestMultiParallel#testActiveThreadsCount
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/f018c371 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f018c371 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f018c371 Branch: refs/heads/hbase-12439 Commit: f018c371b574a2d51930ba99f441bd227a77ec23 Parents: 9a297ef Author: chenheng <chenh...@apache.org> Authored: Tue Dec 22 11:33:25 2015 +0800 Committer: chenheng <chenh...@apache.org> Committed: Tue Dec 22 11:33:25 2015 +0800 ---------------------------------------------------------------------- .../src/main/java/org/apache/hadoop/hbase/client/HTable.java | 5 +++-- .../java/org/apache/hadoop/hbase/client/TestMultiParallel.java | 3 ++- 2 files changed, 5 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/f018c371/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index d3f3bc4..ce5a44c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -128,14 +128,15 @@ public class HTable implements HTableInterface { if (maxThreads == 0) { maxThreads = 1; // is there a better default? } + int corePoolSize = conf.getInt("hbase.htable.threads.coresize", 1); long keepAliveTime = conf.getLong("hbase.htable.threads.keepalivetime", 60); // 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. - ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, keepAliveTime, TimeUnit.SECONDS, - new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("htable")); + ThreadPoolExecutor pool = new ThreadPoolExecutor(corePoolSize, maxThreads, keepAliveTime, + TimeUnit.SECONDS, new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("htable")); pool.allowCoreThreadTimeOut(true); return pool; } http://git-wip-us.apache.org/repos/asf/hbase/blob/f018c371/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java index 59c863e..b2c5c11 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java @@ -144,8 +144,9 @@ public class TestMultiParallel { * @throws NoSuchFieldException * @throws SecurityException */ - @Ignore ("Nice bug flakey... expected 5 but was 4..") @Test(timeout=300000) + @Test(timeout=300000) public void testActiveThreadsCount() throws Exception { + UTIL.getConfiguration().setLong("hbase.htable.threads.coresize", slaves + 1); try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration())) { ThreadPoolExecutor executor = HTable.getDefaultExecutor(UTIL.getConfiguration()); try {