You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ha...@apache.org on 2021/12/07 04:23:38 UTC

[hbase] branch branch-2.4 updated: HBASE-26525 Use unique thread name for group WALs (#3903)

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

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


The following commit(s) were added to refs/heads/branch-2.4 by this push:
     new 6d2faf4  HBASE-26525 Use unique thread name for group WALs (#3903)
6d2faf4 is described below

commit 6d2faf43233aa7bde346dde0f81e4d9464872850
Author: Xiaolin Ha <ha...@apache.org>
AuthorDate: Tue Dec 7 12:19:33 2021 +0800

    HBASE-26525 Use unique thread name for group WALs (#3903)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java    | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index f5619d4..1e4a2da 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -239,8 +239,9 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
       ThreadPoolExecutor threadPool =
         new ThreadPoolExecutor(1, 1, 0L,
           TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(),
-            new ThreadFactoryBuilder().setNameFormat("AsyncFSWAL-%d-" + rootDir.toString()).
-              setDaemon(true).build());
+          new ThreadFactoryBuilder().setNameFormat("AsyncFSWAL-%d-"+ rootDir.toString() +
+              "-prefix:" + (prefix == null ? "default" : prefix).replace("%", "%%"))
+            .setDaemon(true).build());
       hasConsumerTask = () -> threadPool.getQueue().peek() == consumer;
       this.consumeExecutor = threadPool;
     }