From 3e375547c1ebfe0ac984a130d6fefdb9d362d175 Mon Sep 17 00:00:00 2001 From: chenheng Date: Tue, 20 Oct 2015 18:07:50 +0800 Subject: [PATCH] HBASE-14654 Reenable TestMultiParallel#testActiveThreadsCount --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java | 3 ++- .../test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) 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 46f8ec0..5d2c88b 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 @@ -131,13 +131,14 @@ 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, + ThreadPoolExecutor pool = new ThreadPoolExecutor(corePoolSize, maxThreads, keepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), Threads.newDaemonThreadFactory("htable")); pool.allowCoreThreadTimeOut(true); return pool; 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 905a7db..b703570 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 { -- 1.9.3 (Apple Git-50)