From bdea8b891f33646793238a5a136229292aa55df5 Mon Sep 17 00:00:00 2001 From: ramkrishna Date: Fri, 25 Sep 2015 23:11:53 +0530 Subject: [PATCH] HBASE-14478 - A ThreadPoolExecutor with a LinkedBlockingQueue cannot execute tasks concurrently (Jingcheng du) --- .../flush/RegionServerFlushTableProcedureManager.java | 3 ++- .../regionserver/snapshot/RegionServerSnapshotManager.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java index a441a6b1ca0..1aa959cad37 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java @@ -212,9 +212,10 @@ public class RegionServerFlushTableProcedureManager extends RegionServerProcedur RegionServerFlushTableProcedureManager.FLUSH_TIMEOUT_MILLIS_DEFAULT); int threads = conf.getInt(CONCURENT_FLUSH_TASKS_KEY, DEFAULT_CONCURRENT_FLUSH_TASKS); this.name = name; - executor = new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.MILLISECONDS, + executor = new ThreadPoolExecutor(threads, threads, keepAlive, TimeUnit.MILLISECONDS, new LinkedBlockingQueue(), new DaemonThreadFactory("rs(" + name + ")-flush-proc-pool")); + executor.allowCoreThreadTimeOut(true); taskPool = new ExecutorCompletionService(executor); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java index f04feb14d2c..d7f77fb4baa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java @@ -282,9 +282,10 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager { RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_DEFAULT); int threads = conf.getInt(CONCURENT_SNAPSHOT_TASKS_KEY, DEFAULT_CONCURRENT_SNAPSHOT_TASKS); this.name = name; - executor = new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.MILLISECONDS, + executor = new ThreadPoolExecutor(threads, threads, keepAlive, TimeUnit.MILLISECONDS, new LinkedBlockingQueue(), new DaemonThreadFactory("rs(" + name + ")-snapshot-pool")); + executor.allowCoreThreadTimeOut(true); taskPool = new ExecutorCompletionService(executor); }