HBASE-14654 Reenable TestMultiParallel#testActiveThreadsCount
This commit is contained in:
parent
9a297ef0a5
commit
f018c371b5
|
@ -128,14 +128,15 @@ public class HTable implements HTableInterface {
|
||||||
if (maxThreads == 0) {
|
if (maxThreads == 0) {
|
||||||
maxThreads = 1; // is there a better default?
|
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);
|
long keepAliveTime = conf.getLong("hbase.htable.threads.keepalivetime", 60);
|
||||||
|
|
||||||
// Using the "direct handoff" approach, new threads will only be created
|
// 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
|
// 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
|
// we only create as many Runnables as there are region servers. It means
|
||||||
// it also scales when new region servers are added.
|
// 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,
|
||||||
new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("htable"));
|
TimeUnit.SECONDS, new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("htable"));
|
||||||
pool.allowCoreThreadTimeOut(true);
|
pool.allowCoreThreadTimeOut(true);
|
||||||
return pool;
|
return pool;
|
||||||
}
|
}
|
||||||
|
|
|
@ -144,8 +144,9 @@ public class TestMultiParallel {
|
||||||
* @throws NoSuchFieldException
|
* @throws NoSuchFieldException
|
||||||
* @throws SecurityException
|
* @throws SecurityException
|
||||||
*/
|
*/
|
||||||
@Ignore ("Nice bug flakey... expected 5 but was 4..") @Test(timeout=300000)
|
@Test(timeout=300000)
|
||||||
public void testActiveThreadsCount() throws Exception {
|
public void testActiveThreadsCount() throws Exception {
|
||||||
|
UTIL.getConfiguration().setLong("hbase.htable.threads.coresize", slaves + 1);
|
||||||
try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration())) {
|
try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration())) {
|
||||||
ThreadPoolExecutor executor = HTable.getDefaultExecutor(UTIL.getConfiguration());
|
ThreadPoolExecutor executor = HTable.getDefaultExecutor(UTIL.getConfiguration());
|
||||||
try {
|
try {
|
||||||
|
|
Loading…
Reference in New Issue