You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ps...@apache.org on 2019/09/22 10:25:08 UTC
[hbase] branch master updated: HBASE-22930 Set unique name to
longCompactions/shortCompactions/split threads (#585)
This is an automated email from the ASF dual-hosted git repository.
psomogyi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/master by this push:
new 537c5f0 HBASE-22930 Set unique name to longCompactions/shortCompactions/split threads (#585)
537c5f0 is described below
commit 537c5f0660fa6ce0b59cd7bb93de95fd3975cd25
Author: Pankaj <pa...@huawei.com>
AuthorDate: Sun Sep 22 15:55:02 2019 +0530
HBASE-22930 Set unique name to longCompactions/shortCompactions/split threads (#585)
---
.../apache/hadoop/hbase/regionserver/CompactSplit.java | 17 ++++++-----------
1 file changed, 6 insertions(+), 11 deletions(-)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
index 4767284..535558a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
@@ -120,8 +120,7 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati
final String n = Thread.currentThread().getName();
int splitThreads = conf.getInt(SPLIT_THREADS, SPLIT_THREADS_DEFAULT);
this.splits = (ThreadPoolExecutor) Executors.newFixedThreadPool(splitThreads,
- new ThreadFactoryBuilder().setNameFormat(n + "-splits-" + System.currentTimeMillis())
- .setDaemon(true).build());
+ new ThreadFactoryBuilder().setNameFormat(n + "-splits-%d").setDaemon(true).build());
}
private void createCompactionExecutors() {
@@ -138,18 +137,14 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati
final String n = Thread.currentThread().getName();
StealJobQueue<Runnable> stealJobQueue = new StealJobQueue<Runnable>(COMPARATOR);
- this.longCompactions = new ThreadPoolExecutor(largeThreads, largeThreads, 60,
- TimeUnit.SECONDS, stealJobQueue,
- new ThreadFactoryBuilder()
- .setNameFormat(n + "-longCompactions-" + System.currentTimeMillis())
+ this.longCompactions = new ThreadPoolExecutor(largeThreads, largeThreads, 60, TimeUnit.SECONDS,
+ stealJobQueue, new ThreadFactoryBuilder().setNameFormat(n + "-longCompactions-%d")
.setDaemon(true).build());
this.longCompactions.setRejectedExecutionHandler(new Rejection());
this.longCompactions.prestartAllCoreThreads();
- this.shortCompactions = new ThreadPoolExecutor(smallThreads, smallThreads, 60,
- TimeUnit.SECONDS, stealJobQueue.getStealFromQueue(),
- new ThreadFactoryBuilder()
- .setNameFormat(n + "-shortCompactions-" + System.currentTimeMillis())
- .setDaemon(true).build());
+ this.shortCompactions = new ThreadPoolExecutor(smallThreads, smallThreads, 60, TimeUnit.SECONDS,
+ stealJobQueue.getStealFromQueue(), new ThreadFactoryBuilder()
+ .setNameFormat(n + "-shortCompactions-%d").setDaemon(true).build());
this.shortCompactions.setRejectedExecutionHandler(new Rejection());
}