HBASE-25547 (addendum): Roll ExecutorType into ExecutorConfig (#2996)

Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
Bharath Vissapragada 2021-03-08 15:00:18 -08:00 committed by GitHub
parent 53128fe7c1
commit c1dacfd577
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 81 additions and 78 deletions

View File

@ -127,25 +127,14 @@ public class ExecutorService {
return getExecutor(type).getThreadPoolExecutor(); return getExecutor(type).getThreadPoolExecutor();
} }
public void startExecutorService(final ExecutorType type, final ExecutorConfig config) {
String name = type.getExecutorName(this.servername);
if (isExecutorServiceRunning(name)) {
LOG.debug("Executor service {} already running on {}", this,
this.servername);
return;
}
startExecutorService(config.setName(name));
}
/** /**
* Initialize the executor lazily, Note if an executor need to be initialized lazily, then all * Initialize the executor lazily, Note if an executor need to be initialized lazily, then all
* paths should use this method to get the executor, should not start executor by using * paths should use this method to get the executor, should not start executor by using
* {@link ExecutorService#startExecutorService(ExecutorConfig)} * {@link ExecutorService#startExecutorService(ExecutorConfig)}
*/ */
public ThreadPoolExecutor getExecutorLazily(ExecutorType type, ExecutorConfig config) { public ThreadPoolExecutor getExecutorLazily(ExecutorConfig config) {
String name = type.getExecutorName(this.servername); return executorMap.computeIfAbsent(config.getName(), (executorName) ->
return executorMap.computeIfAbsent(name, (executorName) -> new Executor(config)).getThreadPoolExecutor();
new Executor(config.setName(name))).getThreadPoolExecutor();
} }
public void submit(final EventHandler eh) { public void submit(final EventHandler eh) {
@ -184,7 +173,7 @@ public class ExecutorService {
/** /**
* Configuration wrapper for {@link Executor}. * Configuration wrapper for {@link Executor}.
*/ */
public static class ExecutorConfig { public class ExecutorConfig {
// Refer to ThreadPoolExecutor javadoc for details of these configuration. // Refer to ThreadPoolExecutor javadoc for details of these configuration.
// Argument validation and bound checks delegated to the underlying ThreadPoolExecutor // Argument validation and bound checks delegated to the underlying ThreadPoolExecutor
// implementation. // implementation.
@ -192,7 +181,16 @@ public class ExecutorService {
private int corePoolSize = -1; private int corePoolSize = -1;
private boolean allowCoreThreadTimeout = false; private boolean allowCoreThreadTimeout = false;
private long keepAliveTimeMillis = KEEP_ALIVE_TIME_MILLIS_DEFAULT; private long keepAliveTimeMillis = KEEP_ALIVE_TIME_MILLIS_DEFAULT;
private String name; private ExecutorType executorType;
public ExecutorConfig setExecutorType(ExecutorType type) {
this.executorType = type;
return this;
}
private ExecutorType getExecutorType() {
return Preconditions.checkNotNull(executorType, "ExecutorType not set.");
}
public int getCorePoolSize() { public int getCorePoolSize() {
return corePoolSize; return corePoolSize;
@ -217,13 +215,11 @@ public class ExecutorService {
return this; return this;
} }
/**
* @return the executor name inferred from the type and the servername on which this is running.
*/
public String getName() { public String getName() {
return Preconditions.checkNotNull(name); return getExecutorType().getExecutorName(servername);
}
public ExecutorConfig setName(String name) {
this.name = name;
return this;
} }
public long getKeepAliveTimeMillis() { public long getKeepAliveTimeMillis() {

View File

@ -1313,33 +1313,34 @@ public class HMaster extends HRegionServer implements MasterServices {
// Start the executor service pools // Start the executor service pools
final int masterOpenRegionPoolSize = conf.getInt( final int masterOpenRegionPoolSize = conf.getInt(
HConstants.MASTER_OPEN_REGION_THREADS, HConstants.MASTER_OPEN_REGION_THREADS_DEFAULT); HConstants.MASTER_OPEN_REGION_THREADS, HConstants.MASTER_OPEN_REGION_THREADS_DEFAULT);
this.executorService.startExecutorService(ExecutorType.MASTER_OPEN_REGION, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(masterOpenRegionPoolSize)); ExecutorType.MASTER_OPEN_REGION).setCorePoolSize(masterOpenRegionPoolSize));
final int masterCloseRegionPoolSize = conf.getInt( final int masterCloseRegionPoolSize = conf.getInt(
HConstants.MASTER_CLOSE_REGION_THREADS, HConstants.MASTER_CLOSE_REGION_THREADS_DEFAULT); HConstants.MASTER_CLOSE_REGION_THREADS, HConstants.MASTER_CLOSE_REGION_THREADS_DEFAULT);
this.executorService.startExecutorService(ExecutorType.MASTER_CLOSE_REGION, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(masterCloseRegionPoolSize)); ExecutorType.MASTER_CLOSE_REGION).setCorePoolSize(masterCloseRegionPoolSize));
final int masterServerOpThreads = conf.getInt(HConstants.MASTER_SERVER_OPERATIONS_THREADS, final int masterServerOpThreads = conf.getInt(HConstants.MASTER_SERVER_OPERATIONS_THREADS,
HConstants.MASTER_SERVER_OPERATIONS_THREADS_DEFAULT); HConstants.MASTER_SERVER_OPERATIONS_THREADS_DEFAULT);
this.executorService.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(masterServerOpThreads)); ExecutorType.MASTER_SERVER_OPERATIONS).setCorePoolSize(masterServerOpThreads));
final int masterServerMetaOpsThreads = conf.getInt( final int masterServerMetaOpsThreads = conf.getInt(
HConstants.MASTER_META_SERVER_OPERATIONS_THREADS, HConstants.MASTER_META_SERVER_OPERATIONS_THREADS,
HConstants.MASTER_META_SERVER_OPERATIONS_THREADS_DEFAULT); HConstants.MASTER_META_SERVER_OPERATIONS_THREADS_DEFAULT);
this.executorService.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(masterServerMetaOpsThreads)); ExecutorType.MASTER_META_SERVER_OPERATIONS).setCorePoolSize(masterServerMetaOpsThreads));
final int masterLogReplayThreads = conf.getInt( final int masterLogReplayThreads = conf.getInt(
HConstants.MASTER_LOG_REPLAY_OPS_THREADS, HConstants.MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT); HConstants.MASTER_LOG_REPLAY_OPS_THREADS, HConstants.MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT);
this.executorService.startExecutorService(ExecutorType.M_LOG_REPLAY_OPS, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(masterLogReplayThreads)); ExecutorType.M_LOG_REPLAY_OPS).setCorePoolSize(masterLogReplayThreads));
final int masterSnapshotThreads = conf.getInt( final int masterSnapshotThreads = conf.getInt(
SnapshotManager.SNAPSHOT_POOL_THREADS_KEY, SnapshotManager.SNAPSHOT_POOL_THREADS_DEFAULT); SnapshotManager.SNAPSHOT_POOL_THREADS_KEY, SnapshotManager.SNAPSHOT_POOL_THREADS_DEFAULT);
this.executorService.startExecutorService(ExecutorType.MASTER_SNAPSHOT_OPERATIONS, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(masterSnapshotThreads).setAllowCoreThreadTimeout(true)); ExecutorType.MASTER_SNAPSHOT_OPERATIONS).setCorePoolSize(masterSnapshotThreads)
.setAllowCoreThreadTimeout(true));
final int masterMergeDispatchThreads = conf.getInt(HConstants.MASTER_MERGE_DISPATCH_THREADS, final int masterMergeDispatchThreads = conf.getInt(HConstants.MASTER_MERGE_DISPATCH_THREADS,
HConstants.MASTER_MERGE_DISPATCH_THREADS_DEFAULT); HConstants.MASTER_MERGE_DISPATCH_THREADS_DEFAULT);
this.executorService.startExecutorService(ExecutorType.MASTER_MERGE_OPERATIONS, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(masterMergeDispatchThreads) ExecutorType.MASTER_MERGE_OPERATIONS).setCorePoolSize(masterMergeDispatchThreads)
.setAllowCoreThreadTimeout(true)); .setAllowCoreThreadTimeout(true));
// We depend on there being only one instance of this executor running // We depend on there being only one instance of this executor running
@ -1347,8 +1348,8 @@ public class HMaster extends HRegionServer implements MasterServices {
// tables. // tables.
// Any time changing this maxThreads to > 1, pls see the comment at // Any time changing this maxThreads to > 1, pls see the comment at
// AccessController#postCompletedCreateTableAction // AccessController#postCompletedCreateTableAction
this.executorService.startExecutorService( executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
ExecutorType.MASTER_TABLE_OPERATIONS, new ExecutorConfig().setCorePoolSize(1)); ExecutorType.MASTER_TABLE_OPERATIONS).setCorePoolSize(1));
startProcedureExecutor(); startProcedureExecutor();
// Create cleaner thread pool // Create cleaner thread pool

View File

@ -2042,57 +2042,59 @@ public class HRegionServer extends Thread implements
// Start executor services // Start executor services
final int openRegionThreads = conf.getInt("hbase.regionserver.executor.openregion.threads", 3); final int openRegionThreads = conf.getInt("hbase.regionserver.executor.openregion.threads", 3);
this.executorService.startExecutorService(ExecutorType.RS_OPEN_REGION, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(openRegionThreads)); ExecutorType.RS_OPEN_REGION).setCorePoolSize(openRegionThreads));
final int openMetaThreads = conf.getInt("hbase.regionserver.executor.openmeta.threads", 1); final int openMetaThreads = conf.getInt("hbase.regionserver.executor.openmeta.threads", 1);
this.executorService.startExecutorService(ExecutorType.RS_OPEN_META, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(openMetaThreads)); ExecutorType.RS_OPEN_META).setCorePoolSize(openMetaThreads));
final int openPriorityRegionThreads = final int openPriorityRegionThreads =
conf.getInt("hbase.regionserver.executor.openpriorityregion.threads", 3); conf.getInt("hbase.regionserver.executor.openpriorityregion.threads", 3);
this.executorService.startExecutorService(ExecutorType.RS_OPEN_PRIORITY_REGION, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(openPriorityRegionThreads)); ExecutorType.RS_OPEN_PRIORITY_REGION).setCorePoolSize(openPriorityRegionThreads));
final int closeRegionThreads = final int closeRegionThreads =
conf.getInt("hbase.regionserver.executor.closeregion.threads", 3); conf.getInt("hbase.regionserver.executor.closeregion.threads", 3);
this.executorService.startExecutorService(ExecutorType.RS_CLOSE_REGION, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(closeRegionThreads)); ExecutorType.RS_CLOSE_REGION).setCorePoolSize(closeRegionThreads));
final int closeMetaThreads = conf.getInt("hbase.regionserver.executor.closemeta.threads", 1); final int closeMetaThreads = conf.getInt("hbase.regionserver.executor.closemeta.threads", 1);
this.executorService.startExecutorService(ExecutorType.RS_CLOSE_META, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(closeMetaThreads)); ExecutorType.RS_CLOSE_META).setCorePoolSize(closeMetaThreads));
if (conf.getBoolean(StoreScanner.STORESCANNER_PARALLEL_SEEK_ENABLE, false)) { if (conf.getBoolean(StoreScanner.STORESCANNER_PARALLEL_SEEK_ENABLE, false)) {
final int storeScannerParallelSeekThreads = final int storeScannerParallelSeekThreads =
conf.getInt("hbase.storescanner.parallel.seek.threads", 10); conf.getInt("hbase.storescanner.parallel.seek.threads", 10);
this.executorService.startExecutorService(ExecutorType.RS_PARALLEL_SEEK, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(storeScannerParallelSeekThreads) ExecutorType.RS_PARALLEL_SEEK).setCorePoolSize(storeScannerParallelSeekThreads)
.setAllowCoreThreadTimeout(true)); .setAllowCoreThreadTimeout(true));
} }
final int logReplayOpsThreads = conf.getInt( final int logReplayOpsThreads = conf.getInt(
HBASE_SPLIT_WAL_MAX_SPLITTER, DEFAULT_HBASE_SPLIT_WAL_MAX_SPLITTER); HBASE_SPLIT_WAL_MAX_SPLITTER, DEFAULT_HBASE_SPLIT_WAL_MAX_SPLITTER);
this.executorService.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(logReplayOpsThreads).setAllowCoreThreadTimeout(true)); ExecutorType.RS_LOG_REPLAY_OPS).setCorePoolSize(logReplayOpsThreads)
.setAllowCoreThreadTimeout(true));
// Start the threads for compacted files discharger // Start the threads for compacted files discharger
final int compactionDischargerThreads = final int compactionDischargerThreads =
conf.getInt(CompactionConfiguration.HBASE_HFILE_COMPACTION_DISCHARGER_THREAD_COUNT, 10); conf.getInt(CompactionConfiguration.HBASE_HFILE_COMPACTION_DISCHARGER_THREAD_COUNT, 10);
this.executorService.startExecutorService(ExecutorType.RS_COMPACTED_FILES_DISCHARGER, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(compactionDischargerThreads)); ExecutorType.RS_COMPACTED_FILES_DISCHARGER).setCorePoolSize(compactionDischargerThreads));
if (ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(conf)) { if (ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(conf)) {
final int regionReplicaFlushThreads = conf.getInt( final int regionReplicaFlushThreads = conf.getInt(
"hbase.regionserver.region.replica.flusher.threads", conf.getInt( "hbase.regionserver.region.replica.flusher.threads", conf.getInt(
"hbase.regionserver.executor.openregion.threads", 3)); "hbase.regionserver.executor.openregion.threads", 3));
this.executorService.startExecutorService(ExecutorType.RS_REGION_REPLICA_FLUSH_OPS, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(regionReplicaFlushThreads)); ExecutorType.RS_REGION_REPLICA_FLUSH_OPS).setCorePoolSize(regionReplicaFlushThreads));
} }
final int refreshPeerThreads = final int refreshPeerThreads =
conf.getInt("hbase.regionserver.executor.refresh.peer.threads", 2); conf.getInt("hbase.regionserver.executor.refresh.peer.threads", 2);
this.executorService.startExecutorService(ExecutorType.RS_REFRESH_PEER, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(refreshPeerThreads)); ExecutorType.RS_REFRESH_PEER).setCorePoolSize(refreshPeerThreads));
final int replaySyncReplicationWALThreads = final int replaySyncReplicationWALThreads =
conf.getInt("hbase.regionserver.executor.replay.sync.replication.wal.threads", 1); conf.getInt("hbase.regionserver.executor.replay.sync.replication.wal.threads", 1);
this.executorService.startExecutorService(ExecutorType.RS_REPLAY_SYNC_REPLICATION_WAL, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(replaySyncReplicationWALThreads)); ExecutorType.RS_REPLAY_SYNC_REPLICATION_WAL).setCorePoolSize(
replaySyncReplicationWALThreads));
final int switchRpcThrottleThreads = final int switchRpcThrottleThreads =
conf.getInt("hbase.regionserver.executor.switch.rpc.throttle.threads", 1); conf.getInt("hbase.regionserver.executor.switch.rpc.throttle.threads", 1);
this.executorService.startExecutorService(ExecutorType.RS_SWITCH_RPC_THROTTLE, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(switchRpcThrottleThreads)); ExecutorType.RS_SWITCH_RPC_THROTTLE).setCorePoolSize(switchRpcThrottleThreads));
Threads.setDaemonThreadRunning(this.walRoller, getName() + ".logRoller", Threads.setDaemonThreadRunning(this.walRoller, getName() + ".logRoller",
uncaughtExceptionHandler); uncaughtExceptionHandler);

View File

@ -22,6 +22,7 @@ import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorConfig; import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorConfig;
import org.apache.hadoop.hbase.executor.ExecutorType; import org.apache.hadoop.hbase.executor.ExecutorType;
import org.apache.hadoop.hbase.io.ByteBuffAllocator; import org.apache.hadoop.hbase.io.ByteBuffAllocator;
@ -97,9 +98,10 @@ public class RegionServicesForStores {
ThreadPoolExecutor getInMemoryCompactionPool() { ThreadPoolExecutor getInMemoryCompactionPool() {
if (rsServices != null) { if (rsServices != null) {
ExecutorConfig config = new ExecutorConfig().setCorePoolSize(inMemoryPoolSize); ExecutorService executorService = rsServices.getExecutorService();
return rsServices.getExecutorService().getExecutorLazily(ExecutorType.RS_IN_MEMORY_COMPACTION, ExecutorConfig config = executorService.new ExecutorConfig().setExecutorType(
config); ExecutorType.RS_IN_MEMORY_COMPACTION).setCorePoolSize(inMemoryPoolSize);
return executorService.getExecutorLazily(config);
} else { } else {
// this could only happen in tests // this could only happen in tests
return getInMemoryCompactionPoolForTest(); return getInMemoryCompactionPoolForTest();

View File

@ -59,8 +59,8 @@ public class TestExecutorStatusChore {
// Start an executor service pool with max 5 threads // Start an executor service pool with max 5 threads
ExecutorService executorService = new ExecutorService("unit_test"); ExecutorService executorService = new ExecutorService("unit_test");
executorService.startExecutorService(ExecutorType.RS_PARALLEL_SEEK, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(maxThreads)); ExecutorType.RS_PARALLEL_SEEK).setCorePoolSize(maxThreads));
MetricsRegionServerSource serverSource = CompatibilitySingletonFactory MetricsRegionServerSource serverSource = CompatibilitySingletonFactory
.getInstance(MetricsRegionServerSourceFactory.class).createServer(null); .getInstance(MetricsRegionServerSourceFactory.class).createServer(null);

View File

@ -71,8 +71,8 @@ public class TestExecutorService {
// Start an executor service pool with max 5 threads // Start an executor service pool with max 5 threads
ExecutorService executorService = new ExecutorService("unit_test"); ExecutorService executorService = new ExecutorService("unit_test");
executorService.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(maxThreads)); ExecutorType.MASTER_SERVER_OPERATIONS).setCorePoolSize(maxThreads));
Executor executor = Executor executor =
executorService.getExecutor(ExecutorType.MASTER_SERVER_OPERATIONS); executorService.getExecutor(ExecutorType.MASTER_SERVER_OPERATIONS);
@ -197,8 +197,8 @@ public class TestExecutorService {
when(server.getConfiguration()).thenReturn(conf); when(server.getConfiguration()).thenReturn(conf);
ExecutorService executorService = new ExecutorService("unit_test"); ExecutorService executorService = new ExecutorService("unit_test");
executorService.startExecutorService( executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
ExecutorType.MASTER_SERVER_OPERATIONS, new ExecutorConfig().setCorePoolSize(1)); ExecutorType.MASTER_SERVER_OPERATIONS).setCorePoolSize(1));
executorService.submit(new EventHandler(server, EventType.M_SERVER_SHUTDOWN) { executorService.submit(new EventHandler(server, EventType.M_SERVER_SHUTDOWN) {
@ -230,8 +230,8 @@ public class TestExecutorService {
when(server.getConfiguration()).thenReturn(conf); when(server.getConfiguration()).thenReturn(conf);
ExecutorService executorService = new ExecutorService("testSnapshotHandlers"); ExecutorService executorService = new ExecutorService("testSnapshotHandlers");
executorService.startExecutorService( executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
ExecutorType.MASTER_SNAPSHOT_OPERATIONS, new ExecutorConfig().setCorePoolSize(1)); ExecutorType.MASTER_SNAPSHOT_OPERATIONS).setCorePoolSize(1));
CountDownLatch latch = new CountDownLatch(1); CountDownLatch latch = new CountDownLatch(1);
CountDownLatch waitForEventToStart = new CountDownLatch(1); CountDownLatch waitForEventToStart = new CountDownLatch(1);

View File

@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorConfig; import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorConfig;
import org.apache.hadoop.hbase.executor.ExecutorType;
import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.regionserver.HRegion.FlushResultImpl; import org.apache.hadoop.hbase.regionserver.HRegion.FlushResultImpl;
@ -193,7 +194,8 @@ public class TestHRegionReplayEvents {
String string = org.apache.hadoop.hbase.executor.EventType.RS_COMPACTED_FILES_DISCHARGER String string = org.apache.hadoop.hbase.executor.EventType.RS_COMPACTED_FILES_DISCHARGER
.toString(); .toString();
ExecutorService es = new ExecutorService(string); ExecutorService es = new ExecutorService(string);
es.startExecutorService(new ExecutorConfig().setCorePoolSize(1).setName(string + "-" + string)); es.startExecutorService(es.new ExecutorConfig().setCorePoolSize(1).setExecutorType(
ExecutorType.RS_COMPACTED_FILES_DISCHARGER));
when(rss.getExecutorService()).thenReturn(es); when(rss.getExecutorService()).thenReturn(es);
primaryRegion = HRegion.createHRegion(primaryHri, rootDir, CONF, htd, walPrimary); primaryRegion = HRegion.createHRegion(primaryHri, rootDir, CONF, htd, walPrimary);
primaryRegion.close(); primaryRegion.close();

View File

@ -212,8 +212,8 @@ public class TestSplitLogWorker {
SplitLogCounters.resetCounters(); SplitLogCounters.resetCounters();
executorService = new ExecutorService("TestSplitLogWorker"); executorService = new ExecutorService("TestSplitLogWorker");
executorService.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS, executorService.startExecutorService(executorService.new ExecutorConfig().setExecutorType(
new ExecutorConfig().setCorePoolSize(10)); ExecutorType.RS_LOG_REPLAY_OPS).setCorePoolSize(10));
} }
@After @After