You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2019/09/04 15:47:52 UTC

[hbase] branch branch-2 updated: HBASE-22930 Set unique name to longCompactions/shortCompactions threads (#548)

This is an automated email from the ASF dual-hosted git repository.

stack pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 018181c  HBASE-22930 Set unique name to longCompactions/shortCompactions threads (#548)
018181c is described below

commit 018181c7a454343b0c35d062749c2ff8e3f893d8
Author: Pankaj <pa...@huawei.com>
AuthorDate: Wed Sep 4 21:17:06 2019 +0530

    HBASE-22930 Set unique name to longCompactions/shortCompactions threads (#548)
---
 .../hadoop/hbase/regionserver/CompactSplit.java    | 23 ++++++++++++----------
 1 file changed, 13 insertions(+), 10 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 bf58073..bdb90be 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
@@ -136,19 +136,22 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati
     Preconditions.checkArgument(largeThreads > 0 && smallThreads > 0);
 
     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())
-            .setDaemon(true).build());
+
+    AtomicInteger longCompactionThreadCounter = new AtomicInteger(0);
+    this.longCompactions =
+        new ThreadPoolExecutor(largeThreads, largeThreads, 60, TimeUnit.SECONDS, stealJobQueue,
+            new ThreadFactoryBuilder().setNameFormat(n + "-longCompactions-"
+                + longCompactionThreadCounter.getAndIncrement() + "-" + System.currentTimeMillis())
+                .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())
+
+    AtomicInteger shortCompactionThreadCounter = new AtomicInteger(0);
+    this.shortCompactions = new ThreadPoolExecutor(smallThreads, smallThreads, 60, TimeUnit.SECONDS,
+        stealJobQueue.getStealFromQueue(),
+        new ThreadFactoryBuilder().setNameFormat(n + "-shortCompactions-"
+            + shortCompactionThreadCounter.getAndIncrement() + "-" + System.currentTimeMillis())
             .setDaemon(true).build());
     this.shortCompactions.setRejectedExecutionHandler(new Rejection());
   }