Skip to content

Commit 7a90c1a

Browse files
sunhellysaintstack
authored andcommitted
HBASE-22881 Fix non-daemon threads in hbase server implementation (#512) (#536)
Signed-off-by: stack <stack@apache.org>
1 parent 961f9a2 commit 7a90c1a

File tree

18 files changed

+62
-111
lines changed

18 files changed

+62
-111
lines changed

hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -807,7 +807,8 @@ public int run(String[] arg0) throws Exception {
807807

808808
// Have them all share the same connection so they all share the same instance of
809809
// ManyServersManyRegionsConnection so I can keep an eye on how many requests by server.
810-
final ExecutorService pool = Executors.newCachedThreadPool(Threads.getNamedThreadFactory("p"));
810+
final ExecutorService pool = Executors.newCachedThreadPool(
811+
Threads.newDaemonThreadFactory("p"));
811812
// Executors.newFixedThreadPool(servers * 10, Threads.getNamedThreadFactory("p"));
812813
// Share a connection so I can keep counts in the 'server' on concurrency.
813814
final Connection sharedConnection = ConnectionFactory.createConnection(getConf()/*, pool*/);

hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -204,7 +204,7 @@ public static ThreadPoolExecutor getBoundedCachedThreadPool(
204204
* @param prefix The prefix of every created Thread's name
205205
* @return a {@link java.util.concurrent.ThreadFactory} that names threads
206206
*/
207-
public static ThreadFactory getNamedThreadFactory(final String prefix) {
207+
private static ThreadFactory getNamedThreadFactory(final String prefix) {
208208
SecurityManager s = System.getSecurityManager();
209209
final ThreadGroup threadGroup = (s != null) ? s.getThreadGroup() : Thread.currentThread()
210210
.getThreadGroup();

hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -226,7 +226,9 @@ private static ThreadFactory getThreadFactory() {
226226
@Override
227227
public Thread newThread(Runnable r) {
228228
final String name = "HFileArchiver-" + threadNumber.getAndIncrement();
229-
return new Thread(r, name);
229+
Thread t = new Thread(r, name);
230+
t.setDaemon(true);
231+
return t;
230232
}
231233
};
232234
}

hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java

Lines changed: 9 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -23,21 +23,21 @@
2323
import java.util.concurrent.ExecutorService;
2424
import java.util.concurrent.RejectedExecutionException;
2525
import java.util.concurrent.SynchronousQueue;
26-
import java.util.concurrent.ThreadFactory;
2726
import java.util.concurrent.ThreadPoolExecutor;
2827
import java.util.concurrent.TimeUnit;
2928

3029
import org.apache.hadoop.conf.Configuration;
3130
import org.apache.hadoop.fs.FileSystem;
3231
import org.apache.hadoop.hbase.TableName;
33-
import org.apache.yetus.audience.InterfaceAudience;
34-
import org.slf4j.Logger;
35-
import org.slf4j.LoggerFactory;
3632
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
3733
import org.apache.hadoop.hbase.master.locking.LockManager;
3834
import org.apache.hadoop.hbase.mob.MobUtils;
3935
import org.apache.hadoop.hbase.procedure2.LockType;
4036
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
37+
import org.apache.yetus.audience.InterfaceAudience;
38+
import org.slf4j.Logger;
39+
import org.slf4j.LoggerFactory;
40+
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
4141

4242
/**
4343
* The mob compaction thread used in {@link MasterRpcServices}
@@ -55,14 +55,11 @@ public MasterMobCompactionThread(HMaster master) {
5555
this.conf = master.getConfiguration();
5656
final String n = Thread.currentThread().getName();
5757
// this pool is used to run the mob compaction
58-
this.masterMobPool = new ThreadPoolExecutor(1, 2, 60, TimeUnit.SECONDS,
59-
new SynchronousQueue<>(), new ThreadFactory() {
60-
@Override
61-
public Thread newThread(Runnable r) {
62-
String name = n + "-MasterMobCompaction-" + EnvironmentEdgeManager.currentTime();
63-
return new Thread(r, name);
64-
}
65-
});
58+
this.masterMobPool = new ThreadPoolExecutor(1, 2, 60,
59+
TimeUnit.SECONDS, new SynchronousQueue<>(),
60+
new ThreadFactoryBuilder().setDaemon(true)
61+
.setNameFormat(n + "-MasterMobCompaction-" + EnvironmentEdgeManager.currentTime())
62+
.build());
6663
((ThreadPoolExecutor) this.masterMobPool).allowCoreThreadTimeOut(true);
6764
// this pool is used in the mob compaction to compact the mob files by partitions
6865
// in parallel

hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -665,7 +665,7 @@ private Pair<Integer, Integer> splitStoreFiles(final MasterProcedureEnv env,
665665
LOG.info("pid=" + getProcId() + " splitting " + nbFiles + " storefiles, region=" +
666666
getParentRegion().getShortNameToLog() + ", threads=" + maxThreads);
667667
final ExecutorService threadPool = Executors.newFixedThreadPool(maxThreads,
668-
Threads.getNamedThreadFactory("StoreFileSplitter-%1$d"));
668+
Threads.newDaemonThreadFactory("StoreFileSplitter-%1$d"));
669669
final List<Future<Pair<Path, Path>>> futures = new ArrayList<Future<Pair<Path, Path>>>(nbFiles);
670670

671671
// Split each store file.

hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java

Lines changed: 4 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -25,11 +25,9 @@
2525
import java.util.concurrent.ExecutionException;
2626
import java.util.concurrent.ExecutorCompletionService;
2727
import java.util.concurrent.Future;
28-
import java.util.concurrent.LinkedBlockingQueue;
2928
import java.util.concurrent.ThreadPoolExecutor;
3029
import java.util.concurrent.TimeUnit;
3130

32-
import org.apache.yetus.audience.InterfaceAudience;
3331
import org.apache.hadoop.conf.Configuration;
3432
import org.apache.hadoop.hbase.Abortable;
3533
import org.apache.hadoop.hbase.DaemonThreadFactory;
@@ -47,7 +45,9 @@
4745
import org.apache.hadoop.hbase.regionserver.HRegion;
4846
import org.apache.hadoop.hbase.regionserver.HRegionServer;
4947
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
48+
import org.apache.hadoop.hbase.util.Threads;
5049
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
50+
import org.apache.yetus.audience.InterfaceAudience;
5151
import org.apache.zookeeper.KeeperException;
5252
import org.slf4j.Logger;
5353
import org.slf4j.LoggerFactory;
@@ -213,10 +213,8 @@ static class FlushTableSubprocedurePool {
213213
RegionServerFlushTableProcedureManager.FLUSH_TIMEOUT_MILLIS_DEFAULT);
214214
int threads = conf.getInt(CONCURENT_FLUSH_TASKS_KEY, DEFAULT_CONCURRENT_FLUSH_TASKS);
215215
this.name = name;
216-
executor = new ThreadPoolExecutor(threads, threads, keepAlive, TimeUnit.MILLISECONDS,
217-
new LinkedBlockingQueue<>(), new DaemonThreadFactory("rs("
218-
+ name + ")-flush-proc-pool"));
219-
executor.allowCoreThreadTimeOut(true);
216+
executor = Threads.getBoundedCachedThreadPool(threads, keepAlive, TimeUnit.MILLISECONDS,
217+
new DaemonThreadFactory("rs(" + name + ")-flush-proc-pool-"));
220218
taskPool = new ExecutorCompletionService<>(executor);
221219
}
222220

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java

Lines changed: 11 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,6 @@
3131
import java.util.concurrent.Executors;
3232
import java.util.concurrent.RejectedExecutionException;
3333
import java.util.concurrent.RejectedExecutionHandler;
34-
import java.util.concurrent.ThreadFactory;
3534
import java.util.concurrent.ThreadPoolExecutor;
3635
import java.util.concurrent.TimeUnit;
3736
import java.util.concurrent.atomic.AtomicInteger;
@@ -56,8 +55,10 @@
5655
import org.apache.yetus.audience.InterfaceAudience;
5756
import org.slf4j.Logger;
5857
import org.slf4j.LoggerFactory;
58+
5959
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
6060
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
61+
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
6162

6263
/**
6364
* Compact region on request and then run split if appropriate
@@ -118,14 +119,9 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati
118119
private void createSplitExcecutors() {
119120
final String n = Thread.currentThread().getName();
120121
int splitThreads = conf.getInt(SPLIT_THREADS, SPLIT_THREADS_DEFAULT);
121-
this.splits =
122-
(ThreadPoolExecutor) Executors.newFixedThreadPool(splitThreads, new ThreadFactory() {
123-
@Override
124-
public Thread newThread(Runnable r) {
125-
String name = n + "-splits-" + System.currentTimeMillis();
126-
return new Thread(r, name);
127-
}
128-
});
122+
this.splits = (ThreadPoolExecutor) Executors.newFixedThreadPool(splitThreads,
123+
new ThreadFactoryBuilder().setNameFormat(n + "-splits-" + System.currentTimeMillis())
124+
.setDaemon(true).build());
129125
}
130126

131127
private void createCompactionExecutors() {
@@ -144,24 +140,16 @@ private void createCompactionExecutors() {
144140
StealJobQueue<Runnable> stealJobQueue = new StealJobQueue<Runnable>(COMPARATOR);
145141
this.longCompactions = new ThreadPoolExecutor(largeThreads, largeThreads, 60,
146142
TimeUnit.SECONDS, stealJobQueue,
147-
new ThreadFactory() {
148-
@Override
149-
public Thread newThread(Runnable r) {
150-
String name = n + "-longCompactions-" + System.currentTimeMillis();
151-
return new Thread(r, name);
152-
}
153-
});
143+
new ThreadFactoryBuilder()
144+
.setNameFormat(n + "-longCompactions-" + System.currentTimeMillis())
145+
.setDaemon(true).build());
154146
this.longCompactions.setRejectedExecutionHandler(new Rejection());
155147
this.longCompactions.prestartAllCoreThreads();
156148
this.shortCompactions = new ThreadPoolExecutor(smallThreads, smallThreads, 60,
157149
TimeUnit.SECONDS, stealJobQueue.getStealFromQueue(),
158-
new ThreadFactory() {
159-
@Override
160-
public Thread newThread(Runnable r) {
161-
String name = n + "-shortCompactions-" + System.currentTimeMillis();
162-
return new Thread(r, name);
163-
}
164-
});
150+
new ThreadFactoryBuilder()
151+
.setNameFormat(n + "-shortCompactions-" + System.currentTimeMillis())
152+
.setDaemon(true).build());
165153
this.shortCompactions.setRejectedExecutionHandler(new Rejection());
166154
}
167155

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java

Lines changed: 3 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,6 @@
2626
import java.util.concurrent.ExecutionException;
2727
import java.util.concurrent.ExecutorCompletionService;
2828
import java.util.concurrent.Future;
29-
import java.util.concurrent.LinkedBlockingQueue;
3029
import java.util.concurrent.ThreadPoolExecutor;
3130
import java.util.concurrent.TimeUnit;
3231

@@ -36,6 +35,7 @@
3635
import org.apache.hadoop.hbase.DroppedSnapshotException;
3736
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
3837
import org.apache.hadoop.hbase.TableName;
38+
import org.apache.hadoop.hbase.util.Threads;
3939
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
4040
import org.apache.yetus.audience.InterfaceAudience;
4141
import org.apache.yetus.audience.InterfaceStability;
@@ -283,10 +283,8 @@ static class SnapshotSubprocedurePool {
283283
RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
284284
int threads = conf.getInt(CONCURENT_SNAPSHOT_TASKS_KEY, DEFAULT_CONCURRENT_SNAPSHOT_TASKS);
285285
this.name = name;
286-
executor = new ThreadPoolExecutor(threads, threads, keepAlive, TimeUnit.MILLISECONDS,
287-
new LinkedBlockingQueue<>(), new DaemonThreadFactory("rs("
288-
+ name + ")-snapshot-pool"));
289-
executor.allowCoreThreadTimeOut(true);
286+
executor = Threads.getBoundedCachedThreadPool(threads, keepAlive, TimeUnit.MILLISECONDS,
287+
new DaemonThreadFactory("rs(" + name + ")-snapshot-pool-"));
290288
taskPool = new ExecutorCompletionService<>(executor);
291289
}
292290

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -222,7 +222,8 @@ public FSHLog(final FileSystem fs, final Path rootDir, final String logDir,
222222
// Using BlockingWaitStrategy. Stuff that is going on here takes so long it makes no sense
223223
// spinning as other strategies do.
224224
this.disruptor = new Disruptor<>(RingBufferTruck::new,
225-
getPreallocatedEventCount(), Threads.getNamedThreadFactory(hostingThreadName + ".append"),
225+
getPreallocatedEventCount(),
226+
Threads.newDaemonThreadFactory(hostingThreadName + ".append"),
226227
ProducerType.MULTI, new BlockingWaitStrategy());
227228
// Advance the ring buffer sequence so that it starts from 1 instead of 0,
228229
// because SyncFuture.NOT_DONE = 0.

hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,6 @@
3232
import java.util.concurrent.ExecutionException;
3333
import java.util.concurrent.ExecutorCompletionService;
3434
import java.util.concurrent.Future;
35-
import java.util.concurrent.LinkedBlockingQueue;
3635
import java.util.concurrent.ThreadPoolExecutor;
3736
import java.util.concurrent.TimeUnit;
3837
import java.util.regex.Matcher;
@@ -57,13 +56,15 @@
5756
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
5857
import org.apache.hadoop.hbase.util.Bytes;
5958
import org.apache.hadoop.hbase.util.FSUtils;
59+
import org.apache.hadoop.hbase.util.Threads;
6060
import org.apache.hadoop.hbase.wal.WAL.Entry;
6161
import org.apache.hadoop.ipc.RemoteException;
6262
import org.apache.yetus.audience.InterfaceAudience;
6363
import org.slf4j.Logger;
6464
import org.slf4j.LoggerFactory;
6565

6666
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
67+
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
6768

6869
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
6970

@@ -141,9 +142,8 @@ public void init(Context context) throws IOException {
141142
// per sink thread pool
142143
this.maxThreads = this.conf.getInt(HConstants.REPLICATION_SOURCE_MAXTHREADS_KEY,
143144
HConstants.REPLICATION_SOURCE_MAXTHREADS_DEFAULT);
144-
this.exec = new ThreadPoolExecutor(maxThreads, maxThreads, 60, TimeUnit.SECONDS,
145-
new LinkedBlockingQueue<>());
146-
this.exec.allowCoreThreadTimeOut(true);
145+
this.exec = Threads.getBoundedCachedThreadPool(maxThreads, 60, TimeUnit.SECONDS,
146+
new ThreadFactoryBuilder().setDaemon(true).setNameFormat("SinkThread-%d").build());
147147
this.abortable = ctx.getAbortable();
148148
// Set the size limit for replication RPCs to 95% of the max request size.
149149
// We could do with less slop if we have an accurate estimate of encoded size. Being

hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java

Lines changed: 9 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -10,8 +10,6 @@
1010
*/
1111
package org.apache.hadoop.hbase.replication.regionserver;
1212

13-
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
14-
1513
import java.io.FileNotFoundException;
1614
import java.io.IOException;
1715
import java.io.InterruptedIOException;
@@ -27,7 +25,6 @@
2725
import java.util.concurrent.Callable;
2826
import java.util.concurrent.ExecutionException;
2927
import java.util.concurrent.Future;
30-
import java.util.concurrent.LinkedBlockingQueue;
3128
import java.util.concurrent.ThreadPoolExecutor;
3229
import java.util.concurrent.TimeUnit;
3330

@@ -39,9 +36,6 @@
3936
import org.apache.hadoop.hbase.HBaseConfiguration;
4037
import org.apache.hadoop.hbase.HConstants;
4138
import org.apache.hadoop.hbase.TableName;
42-
import org.apache.yetus.audience.InterfaceAudience;
43-
import org.slf4j.Logger;
44-
import org.slf4j.LoggerFactory;
4539
import org.apache.hadoop.hbase.client.Connection;
4640
import org.apache.hadoop.hbase.client.RegionLocator;
4741
import org.apache.hadoop.hbase.client.Table;
@@ -53,6 +47,12 @@
5347
import org.apache.hadoop.hbase.util.Bytes;
5448
import org.apache.hadoop.hbase.util.FSUtils;
5549
import org.apache.hadoop.hbase.util.Pair;
50+
import org.apache.hadoop.hbase.util.Threads;
51+
import org.apache.yetus.audience.InterfaceAudience;
52+
import org.slf4j.Logger;
53+
import org.slf4j.LoggerFactory;
54+
55+
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
5656

5757
/**
5858
* It is used for replicating HFile entries. It will first copy parallely all the hfiles to a local
@@ -105,12 +105,9 @@ public HFileReplicator(Configuration sourceClusterConf,
105105
this.maxCopyThreads =
106106
this.conf.getInt(REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY,
107107
REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT);
108-
ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
109-
builder.setNameFormat("HFileReplicationCallable-%1$d");
110-
this.exec =
111-
new ThreadPoolExecutor(maxCopyThreads, maxCopyThreads, 60, TimeUnit.SECONDS,
112-
new LinkedBlockingQueue<>(), builder.build());
113-
this.exec.allowCoreThreadTimeOut(true);
108+
this.exec = Threads.getBoundedCachedThreadPool(maxCopyThreads, 60, TimeUnit.SECONDS,
109+
new ThreadFactoryBuilder().setDaemon(true)
110+
.setNameFormat("HFileReplicationCallable-%1$d").build());
114111
this.copiesPerThread =
115112
conf.getInt(REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_KEY,
116113
REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_DEFAULT);

hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -196,8 +196,8 @@ public ReplicationSourceManager(ReplicationQueueStorage queueStorage,
196196
int nbWorkers = conf.getInt("replication.executor.workers", 1);
197197
// use a short 100ms sleep since this could be done inline with a RS startup
198198
// even if we fail, other region servers can take care of it
199-
this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers, 100, TimeUnit.MILLISECONDS,
200-
new LinkedBlockingQueue<>());
199+
this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers, 100,
200+
TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>());
201201
ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
202202
tfb.setNameFormat("ReplicationExecutor-%d");
203203
tfb.setDaemon(true);

hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -571,7 +571,7 @@ private ThreadPoolExecutor createExecutor(final String name) {
571571
public static ThreadPoolExecutor createExecutor(final Configuration conf, final String name) {
572572
int maxThreads = conf.getInt("hbase.snapshot.thread.pool.max", 8);
573573
return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
574-
Threads.getNamedThreadFactory(name));
574+
Threads.newDaemonThreadFactory(name));
575575
}
576576

577577
/**

hbase-server/src/main/java/org/apache/hadoop/hbase/tool/HFileContentValidator.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -108,7 +108,7 @@ private ExecutorService createThreadPool(Configuration conf) {
108108
int availableProcessors = Runtime.getRuntime().availableProcessors();
109109
int numThreads = conf.getInt("hfilevalidator.numthreads", availableProcessors);
110110
return Executors.newFixedThreadPool(numThreads,
111-
Threads.getNamedThreadFactory("hfile-validator"));
111+
Threads.newDaemonThreadFactory("hfile-validator"));
112112
}
113113

114114
@Override

hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1623,7 +1623,8 @@ public boolean accept(Path path) {
16231623
// run in multiple threads
16241624
ThreadPoolExecutor tpe = new ThreadPoolExecutor(threadPoolSize,
16251625
threadPoolSize, 60, TimeUnit.SECONDS,
1626-
new ArrayBlockingQueue<>(statusList.length));
1626+
new ArrayBlockingQueue<>(statusList.length),
1627+
Threads.newDaemonThreadFactory("FSRegionQuery"));
16271628
try {
16281629
// ignore all file status items that are not of interest
16291630
for (FileStatus regionStatus : statusList) {

hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java

Lines changed: 1 addition & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,6 @@
2828
import java.util.concurrent.CompletionService;
2929
import java.util.concurrent.ExecutionException;
3030
import java.util.concurrent.ExecutorCompletionService;
31-
import java.util.concurrent.ThreadFactory;
3231
import java.util.concurrent.ThreadPoolExecutor;
3332
import java.util.concurrent.TimeUnit;
3433

@@ -235,14 +234,7 @@ static ThreadPoolExecutor getRegionOpenAndInitThreadPool(final Configuration con
235234
"hbase.hregion.open.and.init.threads.max", 16));
236235
ThreadPoolExecutor regionOpenAndInitThreadPool = Threads
237236
.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
238-
new ThreadFactory() {
239-
private int count = 1;
240-
241-
@Override
242-
public Thread newThread(Runnable r) {
243-
return new Thread(r, threadNamePrefix + "-" + count++);
244-
}
245-
});
237+
Threads.newDaemonThreadFactory(threadNamePrefix));
246238
return regionOpenAndInitThreadPool;
247239
}
248240
}

hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -124,9 +124,9 @@ public class SimpleSubprocedurePool implements Closeable, Abortable {
124124

125125
public SimpleSubprocedurePool(String name, Configuration conf) {
126126
this.name = name;
127-
executor = new ThreadPoolExecutor(1, 1, 500, TimeUnit.SECONDS,
128-
new LinkedBlockingQueue<>(),
129-
new DaemonThreadFactory("rs(" + name + ")-procedure-pool"));
127+
executor = new ThreadPoolExecutor(1, 1, 500,
128+
TimeUnit.SECONDS, new LinkedBlockingQueue<>(),
129+
new DaemonThreadFactory("rs(" + name + ")-procedure-pool-"));
130130
taskPool = new ExecutorCompletionService<>(executor);
131131
}
132132

0 commit comments

Comments
 (0)