Skip to content

Commit ee9d986

Browse files
sunhellysaintstack
authored andcommitted
HBASE-22881 Fix non-daemon threads in hbase server implementation (#512)
* address checkstyle issue * change threadPool threads to daemon in server implementation Signed-off-by: Duo Zhang <zhangduo@apache.org> Signed-off-by: stack <stack@apache.org>
1 parent a95ee63 commit ee9d986

File tree

18 files changed

+57
-110
lines changed

18 files changed

+57
-110
lines changed

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
@@ -675,7 +675,7 @@ private Pair<Integer, Integer> splitStoreFiles(final MasterProcedureEnv env,
675675
LOG.info("pid=" + getProcId() + " splitting " + nbFiles + " storefiles, region=" +
676676
getParentRegion().getShortNameToLog() + ", threads=" + maxThreads);
677677
final ExecutorService threadPool = Executors.newFixedThreadPool(maxThreads,
678-
Threads.getNamedThreadFactory("StoreFileSplitter-%1$d"));
678+
Threads.newDaemonThreadFactory("StoreFileSplitter-%1$d"));
679679
final List<Future<Pair<Path, Path>>> futures = new ArrayList<Future<Pair<Path, Path>>>(nbFiles);
680680

681681
// Split each store file.

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

Lines changed: 5 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,8 +45,11 @@
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;
5150
import org.apache.zookeeper.KeeperException;
51+
52+
import org.apache.yetus.audience.InterfaceAudience;
5253
import org.slf4j.Logger;
5354
import org.slf4j.LoggerFactory;
5455

@@ -213,10 +214,8 @@ static class FlushTableSubprocedurePool {
213214
RegionServerFlushTableProcedureManager.FLUSH_TIMEOUT_MILLIS_DEFAULT);
214215
int threads = conf.getInt(CONCURENT_FLUSH_TASKS_KEY, DEFAULT_CONCURRENT_FLUSH_TASKS);
215216
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);
217+
executor = Threads.getBoundedCachedThreadPool(threads, keepAlive, TimeUnit.MILLISECONDS,
218+
new DaemonThreadFactory("rs(" + name + ")-flush-proc-pool-"));
220219
taskPool = new ExecutorCompletionService<>(executor);
221220
}
222221

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
@@ -226,7 +226,8 @@ public FSHLog(final FileSystem fs, final Path rootDir, final String logDir,
226226
// Using BlockingWaitStrategy. Stuff that is going on here takes so long it makes no sense
227227
// spinning as other strategies do.
228228
this.disruptor = new Disruptor<>(RingBufferTruck::new,
229-
getPreallocatedEventCount(), Threads.getNamedThreadFactory(hostingThreadName + ".append"),
229+
getPreallocatedEventCount(),
230+
Threads.newDaemonThreadFactory(hostingThreadName + ".append"),
230231
ProducerType.MULTI, new BlockingWaitStrategy());
231232
// Advance the ring buffer sequence so that it starts from 1 instead of 0,
232233
// 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;
@@ -59,8 +58,10 @@
5958
import org.apache.hadoop.hbase.security.User;
6059
import org.apache.hadoop.hbase.util.Bytes;
6160
import org.apache.hadoop.hbase.util.FSUtils;
61+
import org.apache.hadoop.hbase.util.Threads;
6262
import org.apache.hadoop.hbase.wal.WAL.Entry;
6363
import org.apache.hadoop.ipc.RemoteException;
64+
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
6465
import org.apache.yetus.audience.InterfaceAudience;
6566
import org.slf4j.Logger;
6667
import org.slf4j.LoggerFactory;
@@ -140,9 +141,8 @@ public void init(Context context) throws IOException {
140141
// per sink thread pool
141142
this.maxThreads = this.conf.getInt(HConstants.REPLICATION_SOURCE_MAXTHREADS_KEY,
142143
HConstants.REPLICATION_SOURCE_MAXTHREADS_DEFAULT);
143-
this.exec = new ThreadPoolExecutor(maxThreads, maxThreads, 60, TimeUnit.SECONDS,
144-
new LinkedBlockingQueue<>());
145-
this.exec.allowCoreThreadTimeOut(true);
144+
this.exec = Threads.getBoundedCachedThreadPool(maxThreads, 60, TimeUnit.SECONDS,
145+
new ThreadFactoryBuilder().setDaemon(true).setNameFormat("SinkThread-%d").build());
146146
this.abortable = ctx.getAbortable();
147147
// Set the size limit for replication RPCs to 95% of the max request size.
148148
// 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: 4 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,6 @@
3232
import java.util.concurrent.Callable;
3333
import java.util.concurrent.ExecutionException;
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 org.apache.hadoop.conf.Configuration;
@@ -52,6 +51,7 @@
5251
import org.apache.hadoop.hbase.util.Bytes;
5352
import org.apache.hadoop.hbase.util.FSUtils;
5453
import org.apache.hadoop.hbase.util.Pair;
54+
import org.apache.hadoop.hbase.util.Threads;
5555
import org.apache.yetus.audience.InterfaceAudience;
5656
import org.slf4j.Logger;
5757
import org.slf4j.LoggerFactory;
@@ -109,12 +109,9 @@ public HFileReplicator(Configuration sourceClusterConf,
109109
this.maxCopyThreads =
110110
this.conf.getInt(REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY,
111111
REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT);
112-
ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
113-
builder.setNameFormat("HFileReplicationCallable-%1$d");
114-
this.exec =
115-
new ThreadPoolExecutor(maxCopyThreads, maxCopyThreads, 60, TimeUnit.SECONDS,
116-
new LinkedBlockingQueue<>(), builder.build());
117-
this.exec.allowCoreThreadTimeOut(true);
112+
this.exec = Threads.getBoundedCachedThreadPool(maxCopyThreads, 60, TimeUnit.SECONDS,
113+
new ThreadFactoryBuilder().setDaemon(true)
114+
.setNameFormat("HFileReplicationCallable-%1$d").build());
118115
this.copiesPerThread =
119116
conf.getInt(REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_KEY,
120117
REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_DEFAULT);

0 commit comments

Comments
 (0)