You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/08/04 18:18:28 UTC

[GitHub] [hbase] virajjasani opened a new pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

virajjasani opened a new pull request #2196:
URL: https://github.com/apache/hbase/pull/2196


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#discussion_r467785338



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
##########
@@ -197,83 +198,14 @@ public static ThreadPoolExecutor getBoundedCachedThreadPool(
     return boundedCachedThreadPool;
   }
 
-  public static ThreadPoolExecutor getBoundedCachedThreadPool(int maxCachedThread, long timeout,
-      TimeUnit unit, String prefix) {
-    return getBoundedCachedThreadPool(maxCachedThread, timeout, unit,
-        newDaemonThreadFactory(prefix));
-  }
-
-  /**
-   * Returns a {@link java.util.concurrent.ThreadFactory} that names each created thread uniquely,
-   * with a common prefix.
-   * @param prefix The prefix of every created Thread's name
-   * @return a {@link java.util.concurrent.ThreadFactory} that names threads
-   */
-  public static ThreadFactory getNamedThreadFactory(final String prefix) {
-    SecurityManager s = System.getSecurityManager();
-    final ThreadGroup threadGroup = (s != null) ? s.getThreadGroup() : Thread.currentThread()
-        .getThreadGroup();
-
-    return new ThreadFactory() {
-      final AtomicInteger threadNumber = new AtomicInteger(1);
-      private final int poolNumber = Threads.poolNumber.getAndIncrement();
-      final ThreadGroup group = threadGroup;
-
-      @Override
-      public Thread newThread(Runnable r) {
-        final String name = prefix + "-pool" + poolNumber + "-t" + threadNumber.getAndIncrement();
-        return new Thread(group, r, name);
-      }
-    };
-  }
-
-  /**
-   * Same as {#newDaemonThreadFactory(String, UncaughtExceptionHandler)},
-   * without setting the exception handler.
-   */
-  public static ThreadFactory newDaemonThreadFactory(final String prefix) {
-    return newDaemonThreadFactory(prefix, null);
-  }
-
-  /**
-   * Get a named {@link ThreadFactory} that just builds daemon threads.
-   * @param prefix name prefix for all threads created from the factory
-   * @param handler unhandles exception handler to set for all threads
-   * @return a thread factory that creates named, daemon threads with
-   *         the supplied exception handler and normal priority
-   */
-  public static ThreadFactory newDaemonThreadFactory(final String prefix,
-      final UncaughtExceptionHandler handler) {
-    final ThreadFactory namedFactory = getNamedThreadFactory(prefix);
-    return new ThreadFactory() {
-      @Override
-      public Thread newThread(Runnable r) {
-        Thread t = namedFactory.newThread(r);
-        if (handler != null) {
-          t.setUncaughtExceptionHandler(handler);
-        } else {

Review comment:
       I see, you are correct. We should not see issue from end result viewpoint if we are just executing some tasks and some exception interrupts executor thread specifically when `ThreadGroup` takes care of it:
   
   ```
       public void uncaughtException(Thread t, Throwable e) {
           if (parent != null) {
               parent.uncaughtException(t, e);
           } else {
               Thread.UncaughtExceptionHandler ueh =
                   Thread.getDefaultUncaughtExceptionHandler();
               if (ueh != null) {
                   ueh.uncaughtException(t, e);
               } else if (!(e instanceof ThreadDeath)) {
                   System.err.print("Exception in thread \""
                                    + t.getName() + "\" ");
                   e.printStackTrace(System.err);
               }
           }
       }
   ```
   
   However, if we want to maintain the same log present in `LOGGING_EXCEPTION_HANDLER` as default (which is what used to happen before this patch) behaviour, we should update the corresponding usages with uncaught exception handler as `LOGGING_EXCEPTION_HANDLER` in all places.
   Let me raise a PR for this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#discussion_r466317865



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
##########
@@ -200,70 +201,7 @@ public static ThreadPoolExecutor getBoundedCachedThreadPool(
   public static ThreadPoolExecutor getBoundedCachedThreadPool(int maxCachedThread, long timeout,

Review comment:
       Agree, at least for now, let me make callers use `getBoundedCachedThreadPool` with `ThreadFactoryBuilder` input and only keep that version of `getBoundedCachedThreadPool` in Threads class.

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
##########
@@ -125,9 +125,8 @@ public Subprocedure buildSubprocedure(String name, byte[] data) {
 
     public SimpleSubprocedurePool(String name, Configuration conf) {
       this.name = name;
-      executor = new ThreadPoolExecutor(1, 1, 500,
-          TimeUnit.SECONDS, new LinkedBlockingQueue<>(),
-          Threads.newDaemonThreadFactory("rs(" + name + ")-procedure"));
+      executor = new ThreadPoolExecutor(1, 1, 500, TimeUnit.SECONDS, new LinkedBlockingQueue<>(),

Review comment:
       Wow, 500s of keep-alive, Thanks, quite a catch. This indeed should belong to SingleThreadExecutor.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#discussion_r465522519



##########
File path: hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
##########
@@ -62,10 +62,9 @@ public LogRollBackupSubprocedurePool(String name, Configuration conf) {
           LogRollRegionServerProcedureManager.BACKUP_TIMEOUT_MILLIS_DEFAULT);
     int threads = conf.getInt(CONCURENT_BACKUP_TASKS_KEY, DEFAULT_CONCURRENT_BACKUP_TASKS);
     this.name = name;
-    executor =
-        new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS,
-            new LinkedBlockingQueue<>(),
-            Threads.newDaemonThreadFactory("rs(" + name + ")-backup"));
+    executor = new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS,
+      new LinkedBlockingQueue<>(),
+      new ThreadFactoryBuilder().setNameFormat("rs(" + name + ")-backup-pool-%d").build());

Review comment:
       The reason why I appended `-pool` is to actually adhere with current format only. We add `-pool` in Threads class internally.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#issuecomment-669957599


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  6s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 18s |  master passed  |
   | +1 :green_heart: |  compile  |   4m 50s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m  1s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-backup in master failed.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 23s |  hbase-examples in master failed.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-procedure in master failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   0m 51s |  hbase-thrift in master failed.  |
   | -0 :warning: |  javadoc  |   0m 16s |  hbase-zookeeper in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  6s |  the patch passed  |
   | +1 :green_heart: |  compile  |   4m 38s |  the patch passed  |
   | +1 :green_heart: |  javac  |   4m 38s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 57s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 16s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 16s |  hbase-zookeeper in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-procedure in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 43s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 50s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 20s |  hbase-backup in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 23s |  hbase-examples in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 35s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m  8s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 45s |  hbase-zookeeper in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 43s |  hbase-procedure in the patch passed.  |
   | +1 :green_heart: |  unit  | 138m 55s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |   4m 13s |  hbase-thrift in the patch passed.  |
   | +1 :green_heart: |  unit  |   8m 50s |  hbase-backup in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m  9s |  hbase-it in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 52s |  hbase-examples in the patch passed.  |
   |  |   | 203m  7s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2196 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux aa1085abb811 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c39cad220d |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-backup.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-examples.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-procedure.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-zookeeper.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-zookeeper.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-procedure.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-backup.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-examples.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/testReport/ |
   | Max. process+thread count | 5027 (vs. ulimit of 12500) |
   | modules | C: hbase-common hbase-client hbase-zookeeper hbase-procedure hbase-server hbase-thrift hbase-backup hbase-it hbase-examples U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] tedyu commented on a change in pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
tedyu commented on a change in pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#discussion_r465341971



##########
File path: hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
##########
@@ -62,10 +62,9 @@ public LogRollBackupSubprocedurePool(String name, Configuration conf) {
           LogRollRegionServerProcedureManager.BACKUP_TIMEOUT_MILLIS_DEFAULT);
     int threads = conf.getInt(CONCURENT_BACKUP_TASKS_KEY, DEFAULT_CONCURRENT_BACKUP_TASKS);
     this.name = name;
-    executor =
-        new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS,
-            new LinkedBlockingQueue<>(),
-            Threads.newDaemonThreadFactory("rs(" + name + ")-backup"));
+    executor = new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS,
+      new LinkedBlockingQueue<>(),
+      new ThreadFactoryBuilder().setNameFormat("rs(" + name + ")-backup-pool-%d").build());

Review comment:
       Should the thread names follow existing format ?
   People may have got used to the current format during debugging.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#issuecomment-668838894


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 11s |  master passed  |
   | +1 :green_heart: |  compile  |   4m 35s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 44s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-backup in master failed.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in master failed.  |
   | -0 :warning: |  javadoc  |   0m 23s |  hbase-examples in master failed.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-procedure in master failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   0m 48s |  hbase-thrift in master failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-zookeeper in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  2s |  the patch passed  |
   | +1 :green_heart: |  compile  |   4m 33s |  the patch passed  |
   | +1 :green_heart: |  javac  |   4m 33s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 48s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-common in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-zookeeper in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-procedure in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 49s |  hbase-thrift in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 20s |  hbase-backup in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 22s |  hbase-examples in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 30s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 11s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 43s |  hbase-zookeeper in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 36s |  hbase-procedure in the patch passed.  |
   | +1 :green_heart: |  unit  | 131m  6s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |   4m 22s |  hbase-thrift in the patch passed.  |
   | +1 :green_heart: |  unit  |  10m  8s |  hbase-backup in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m  7s |  hbase-it in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 41s |  hbase-examples in the patch passed.  |
   |  |   | 194m 38s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2196 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 9efd7ae56111 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / d2f5a5f27b |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-backup.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-examples.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-procedure.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-zookeeper.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-zookeeper.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-procedure.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-thrift.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-backup.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-examples.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/testReport/ |
   | Max. process+thread count | 5411 (vs. ulimit of 12500) |
   | modules | C: hbase-common hbase-client hbase-zookeeper hbase-procedure hbase-server hbase-thrift hbase-backup hbase-it hbase-examples U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] ndimiduk commented on a change in pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#discussion_r465984755



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
##########
@@ -200,70 +201,7 @@ public static ThreadPoolExecutor getBoundedCachedThreadPool(
   public static ThreadPoolExecutor getBoundedCachedThreadPool(int maxCachedThread, long timeout,

Review comment:
       I wonder if these `getBoundedCachedThreadPool` methods could be replace to calls to methods directly on `java.util.concurrent.Executors`. I think we'd need to evaluate on a case-by-case basis how the resulting pools are being used and if they're constructed to correctly match the use-case.

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
##########
@@ -125,9 +125,8 @@ public Subprocedure buildSubprocedure(String name, byte[] data) {
 
     public SimpleSubprocedurePool(String name, Configuration conf) {
       this.name = name;
-      executor = new ThreadPoolExecutor(1, 1, 500,
-          TimeUnit.SECONDS, new LinkedBlockingQueue<>(),
-          Threads.newDaemonThreadFactory("rs(" + name + ")-procedure"));
+      executor = new ThreadPoolExecutor(1, 1, 500, TimeUnit.SECONDS, new LinkedBlockingQueue<>(),

Review comment:
       This is a bizarre instantiation, don't you think? keep-alive of 500 seconds? in a test? Maybe this can be just `Executors.newSingleThreadExecutor()`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#discussion_r467847381



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
##########
@@ -197,83 +198,14 @@ public static ThreadPoolExecutor getBoundedCachedThreadPool(
     return boundedCachedThreadPool;
   }
 
-  public static ThreadPoolExecutor getBoundedCachedThreadPool(int maxCachedThread, long timeout,
-      TimeUnit unit, String prefix) {
-    return getBoundedCachedThreadPool(maxCachedThread, timeout, unit,
-        newDaemonThreadFactory(prefix));
-  }
-
-  /**
-   * Returns a {@link java.util.concurrent.ThreadFactory} that names each created thread uniquely,
-   * with a common prefix.
-   * @param prefix The prefix of every created Thread's name
-   * @return a {@link java.util.concurrent.ThreadFactory} that names threads
-   */
-  public static ThreadFactory getNamedThreadFactory(final String prefix) {
-    SecurityManager s = System.getSecurityManager();
-    final ThreadGroup threadGroup = (s != null) ? s.getThreadGroup() : Thread.currentThread()
-        .getThreadGroup();
-
-    return new ThreadFactory() {
-      final AtomicInteger threadNumber = new AtomicInteger(1);
-      private final int poolNumber = Threads.poolNumber.getAndIncrement();
-      final ThreadGroup group = threadGroup;
-
-      @Override
-      public Thread newThread(Runnable r) {
-        final String name = prefix + "-pool" + poolNumber + "-t" + threadNumber.getAndIncrement();
-        return new Thread(group, r, name);
-      }
-    };
-  }
-
-  /**
-   * Same as {#newDaemonThreadFactory(String, UncaughtExceptionHandler)},
-   * without setting the exception handler.
-   */
-  public static ThreadFactory newDaemonThreadFactory(final String prefix) {
-    return newDaemonThreadFactory(prefix, null);
-  }
-
-  /**
-   * Get a named {@link ThreadFactory} that just builds daemon threads.
-   * @param prefix name prefix for all threads created from the factory
-   * @param handler unhandles exception handler to set for all threads
-   * @return a thread factory that creates named, daemon threads with
-   *         the supplied exception handler and normal priority
-   */
-  public static ThreadFactory newDaemonThreadFactory(final String prefix,
-      final UncaughtExceptionHandler handler) {
-    final ThreadFactory namedFactory = getNamedThreadFactory(prefix);
-    return new ThreadFactory() {
-      @Override
-      public Thread newThread(Runnable r) {
-        Thread t = namedFactory.newThread(r);
-        if (handler != null) {
-          t.setUncaughtExceptionHandler(handler);
-        } else {

Review comment:
       FYI https://github.com/apache/hbase/pull/2231




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#issuecomment-669958859


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 37s |  master passed  |
   | +1 :green_heart: |  compile  |   4m 14s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m  4s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 25s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 48s |  the patch passed  |
   | +1 :green_heart: |  compile  |   4m 23s |  the patch passed  |
   | +1 :green_heart: |  javac  |   4m 23s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 41s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 20s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 25s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m  6s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 42s |  hbase-zookeeper in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 52s |  hbase-procedure in the patch passed.  |
   | +1 :green_heart: |  unit  | 144m 11s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |   4m 28s |  hbase-thrift in the patch passed.  |
   | +1 :green_heart: |  unit  |  10m 46s |  hbase-backup in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 12s |  hbase-it in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 54s |  hbase-examples in the patch passed.  |
   |  |   | 207m  5s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2196 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 61b30c1981cf 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c39cad220d |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/testReport/ |
   | Max. process+thread count | 5361 (vs. ulimit of 12500) |
   | modules | C: hbase-common hbase-client hbase-zookeeper hbase-procedure hbase-server hbase-thrift hbase-backup hbase-it hbase-examples U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#issuecomment-668844676


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  8s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 33s |  master passed  |
   | +1 :green_heart: |  compile  |   4m  8s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 38s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 15s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 25s |  the patch passed  |
   | +1 :green_heart: |  compile  |   4m  9s |  the patch passed  |
   | +1 :green_heart: |  javac  |   4m  9s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 33s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 18s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 16s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m  1s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 42s |  hbase-zookeeper in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 51s |  hbase-procedure in the patch passed.  |
   | +1 :green_heart: |  unit  | 150m 11s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |   4m 31s |  hbase-thrift in the patch passed.  |
   | +1 :green_heart: |  unit  |   9m 50s |  hbase-backup in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 10s |  hbase-it in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  2s |  hbase-examples in the patch passed.  |
   |  |   | 210m 59s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2196 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 0378fd41ce50 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / d2f5a5f27b |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/testReport/ |
   | Max. process+thread count | 4926 (vs. ulimit of 12500) |
   | modules | C: hbase-common hbase-client hbase-zookeeper hbase-procedure hbase-server hbase-thrift hbase-backup hbase-it hbase-examples U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#issuecomment-669877792


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 26s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  1s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 23s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   4m  0s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   7m 31s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 28s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 23s |  hbase-common: The patch generated 2 new + 6 unchanged - 0 fixed = 8 total (was 6)  |
   | -0 :warning: |  checkstyle  |   0m 25s |  hbase-client: The patch generated 3 new + 1 unchanged - 0 fixed = 4 total (was 1)  |
   | -0 :warning: |  checkstyle  |   0m 12s |  hbase-zookeeper: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   0m 14s |  hbase-procedure: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   1m  6s |  hbase-server: The patch generated 27 new + 249 unchanged - 1 fixed = 276 total (was 250)  |
   | -0 :warning: |  checkstyle  |   0m 38s |  hbase-thrift: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   0m 14s |  hbase-backup: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   0m 14s |  hbase-examples: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 41s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   9m 10s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 44s |  The patch does not generate ASF License warnings.  |
   |  |   |  57m 16s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2196 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 839d711878fb 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c39cad220d |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-common.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-zookeeper.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-procedure.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-thrift.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-backup.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-examples.txt |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-common hbase-client hbase-zookeeper hbase-procedure hbase-server hbase-thrift hbase-backup hbase-it hbase-examples U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] tedyu commented on a change in pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
tedyu commented on a change in pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#discussion_r465341971



##########
File path: hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
##########
@@ -62,10 +62,9 @@ public LogRollBackupSubprocedurePool(String name, Configuration conf) {
           LogRollRegionServerProcedureManager.BACKUP_TIMEOUT_MILLIS_DEFAULT);
     int threads = conf.getInt(CONCURENT_BACKUP_TASKS_KEY, DEFAULT_CONCURRENT_BACKUP_TASKS);
     this.name = name;
-    executor =
-        new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS,
-            new LinkedBlockingQueue<>(),
-            Threads.newDaemonThreadFactory("rs(" + name + ")-backup"));
+    executor = new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS,
+      new LinkedBlockingQueue<>(),
+      new ThreadFactoryBuilder().setNameFormat("rs(" + name + ")-backup-pool-%d").build());

Review comment:
       Should the thread names follow existing format (dropping '-pool') ?
   People may have got used to the current format during debugging.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] tedyu commented on a change in pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
tedyu commented on a change in pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#discussion_r465946272



##########
File path: hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
##########
@@ -62,10 +62,9 @@ public LogRollBackupSubprocedurePool(String name, Configuration conf) {
           LogRollRegionServerProcedureManager.BACKUP_TIMEOUT_MILLIS_DEFAULT);
     int threads = conf.getInt(CONCURENT_BACKUP_TASKS_KEY, DEFAULT_CONCURRENT_BACKUP_TASKS);
     this.name = name;
-    executor =
-        new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS,
-            new LinkedBlockingQueue<>(),
-            Threads.newDaemonThreadFactory("rs(" + name + ")-backup"));
+    executor = new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS,
+      new LinkedBlockingQueue<>(),
+      new ThreadFactoryBuilder().setNameFormat("rs(" + name + ")-backup-pool-%d").build());

Review comment:
       The change is fine.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#discussion_r467847381



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
##########
@@ -197,83 +198,14 @@ public static ThreadPoolExecutor getBoundedCachedThreadPool(
     return boundedCachedThreadPool;
   }
 
-  public static ThreadPoolExecutor getBoundedCachedThreadPool(int maxCachedThread, long timeout,
-      TimeUnit unit, String prefix) {
-    return getBoundedCachedThreadPool(maxCachedThread, timeout, unit,
-        newDaemonThreadFactory(prefix));
-  }
-
-  /**
-   * Returns a {@link java.util.concurrent.ThreadFactory} that names each created thread uniquely,
-   * with a common prefix.
-   * @param prefix The prefix of every created Thread's name
-   * @return a {@link java.util.concurrent.ThreadFactory} that names threads
-   */
-  public static ThreadFactory getNamedThreadFactory(final String prefix) {
-    SecurityManager s = System.getSecurityManager();
-    final ThreadGroup threadGroup = (s != null) ? s.getThreadGroup() : Thread.currentThread()
-        .getThreadGroup();
-
-    return new ThreadFactory() {
-      final AtomicInteger threadNumber = new AtomicInteger(1);
-      private final int poolNumber = Threads.poolNumber.getAndIncrement();
-      final ThreadGroup group = threadGroup;
-
-      @Override
-      public Thread newThread(Runnable r) {
-        final String name = prefix + "-pool" + poolNumber + "-t" + threadNumber.getAndIncrement();
-        return new Thread(group, r, name);
-      }
-    };
-  }
-
-  /**
-   * Same as {#newDaemonThreadFactory(String, UncaughtExceptionHandler)},
-   * without setting the exception handler.
-   */
-  public static ThreadFactory newDaemonThreadFactory(final String prefix) {
-    return newDaemonThreadFactory(prefix, null);
-  }
-
-  /**
-   * Get a named {@link ThreadFactory} that just builds daemon threads.
-   * @param prefix name prefix for all threads created from the factory
-   * @param handler unhandles exception handler to set for all threads
-   * @return a thread factory that creates named, daemon threads with
-   *         the supplied exception handler and normal priority
-   */
-  public static ThreadFactory newDaemonThreadFactory(final String prefix,
-      final UncaughtExceptionHandler handler) {
-    final ThreadFactory namedFactory = getNamedThreadFactory(prefix);
-    return new ThreadFactory() {
-      @Override
-      public Thread newThread(Runnable r) {
-        Thread t = namedFactory.newThread(r);
-        if (handler != null) {
-          t.setUncaughtExceptionHandler(handler);
-        } else {

Review comment:
       Thanks @nyl3532016 .
   FYI https://github.com/apache/hbase/pull/2231




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#issuecomment-668782334


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 21s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  2s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   4m 19s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   9m 22s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 54s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 24s |  hbase-common: The patch generated 1 new + 6 unchanged - 0 fixed = 7 total (was 6)  |
   | -0 :warning: |  checkstyle  |   0m 30s |  hbase-client: The patch generated 3 new + 1 unchanged - 0 fixed = 4 total (was 1)  |
   | -0 :warning: |  checkstyle  |   0m 15s |  hbase-zookeeper: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   0m 19s |  hbase-procedure: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   1m 20s |  hbase-server: The patch generated 24 new + 220 unchanged - 1 fixed = 244 total (was 221)  |
   | -0 :warning: |  checkstyle  |   0m 49s |  hbase-thrift: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   0m 16s |  hbase-backup: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   0m 16s |  hbase-examples: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  13m  4s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   9m 45s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 31s |  The patch does not generate ASF License warnings.  |
   |  |   |  62m 43s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2196 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux e6d82255df19 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / d2f5a5f27b |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-common.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-zookeeper.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-procedure.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-thrift.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-backup.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-examples.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-common hbase-client hbase-zookeeper hbase-procedure hbase-server hbase-thrift hbase-backup hbase-it hbase-examples U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2196/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] virajjasani closed pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
virajjasani closed pull request #2196:
URL: https://github.com/apache/hbase/pull/2196


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#discussion_r465928335



##########
File path: hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
##########
@@ -62,10 +62,9 @@ public LogRollBackupSubprocedurePool(String name, Configuration conf) {
           LogRollRegionServerProcedureManager.BACKUP_TIMEOUT_MILLIS_DEFAULT);
     int threads = conf.getInt(CONCURENT_BACKUP_TASKS_KEY, DEFAULT_CONCURRENT_BACKUP_TASKS);
     this.name = name;
-    executor =
-        new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS,
-            new LinkedBlockingQueue<>(),
-            Threads.newDaemonThreadFactory("rs(" + name + ")-backup"));
+    executor = new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS,
+      new LinkedBlockingQueue<>(),
+      new ThreadFactoryBuilder().setNameFormat("rs(" + name + ")-backup-pool-%d").build());

Review comment:
       @tedyu could you please review?
   Thanks




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] nyl3532016 commented on a change in pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
nyl3532016 commented on a change in pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#discussion_r467815030



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
##########
@@ -197,83 +198,14 @@ public static ThreadPoolExecutor getBoundedCachedThreadPool(
     return boundedCachedThreadPool;
   }
 
-  public static ThreadPoolExecutor getBoundedCachedThreadPool(int maxCachedThread, long timeout,
-      TimeUnit unit, String prefix) {
-    return getBoundedCachedThreadPool(maxCachedThread, timeout, unit,
-        newDaemonThreadFactory(prefix));
-  }
-
-  /**
-   * Returns a {@link java.util.concurrent.ThreadFactory} that names each created thread uniquely,
-   * with a common prefix.
-   * @param prefix The prefix of every created Thread's name
-   * @return a {@link java.util.concurrent.ThreadFactory} that names threads
-   */
-  public static ThreadFactory getNamedThreadFactory(final String prefix) {
-    SecurityManager s = System.getSecurityManager();
-    final ThreadGroup threadGroup = (s != null) ? s.getThreadGroup() : Thread.currentThread()
-        .getThreadGroup();
-
-    return new ThreadFactory() {
-      final AtomicInteger threadNumber = new AtomicInteger(1);
-      private final int poolNumber = Threads.poolNumber.getAndIncrement();
-      final ThreadGroup group = threadGroup;
-
-      @Override
-      public Thread newThread(Runnable r) {
-        final String name = prefix + "-pool" + poolNumber + "-t" + threadNumber.getAndIncrement();
-        return new Thread(group, r, name);
-      }
-    };
-  }
-
-  /**
-   * Same as {#newDaemonThreadFactory(String, UncaughtExceptionHandler)},
-   * without setting the exception handler.
-   */
-  public static ThreadFactory newDaemonThreadFactory(final String prefix) {
-    return newDaemonThreadFactory(prefix, null);
-  }
-
-  /**
-   * Get a named {@link ThreadFactory} that just builds daemon threads.
-   * @param prefix name prefix for all threads created from the factory
-   * @param handler unhandles exception handler to set for all threads
-   * @return a thread factory that creates named, daemon threads with
-   *         the supplied exception handler and normal priority
-   */
-  public static ThreadFactory newDaemonThreadFactory(final String prefix,
-      final UncaughtExceptionHandler handler) {
-    final ThreadFactory namedFactory = getNamedThreadFactory(prefix);
-    return new ThreadFactory() {
-      @Override
-      public Thread newThread(Runnable r) {
-        Thread t = namedFactory.newThread(r);
-        if (handler != null) {
-          t.setUncaughtExceptionHandler(handler);
-        } else {

Review comment:
       thank you for explaining about it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] nyl3532016 commented on a change in pull request #2196: HBASE-24750 : All ExecutorService should use guava ThreadFactoryBuilder

Posted by GitBox <gi...@apache.org>.
nyl3532016 commented on a change in pull request #2196:
URL: https://github.com/apache/hbase/pull/2196#discussion_r467535276



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
##########
@@ -197,83 +198,14 @@ public static ThreadPoolExecutor getBoundedCachedThreadPool(
     return boundedCachedThreadPool;
   }
 
-  public static ThreadPoolExecutor getBoundedCachedThreadPool(int maxCachedThread, long timeout,
-      TimeUnit unit, String prefix) {
-    return getBoundedCachedThreadPool(maxCachedThread, timeout, unit,
-        newDaemonThreadFactory(prefix));
-  }
-
-  /**
-   * Returns a {@link java.util.concurrent.ThreadFactory} that names each created thread uniquely,
-   * with a common prefix.
-   * @param prefix The prefix of every created Thread's name
-   * @return a {@link java.util.concurrent.ThreadFactory} that names threads
-   */
-  public static ThreadFactory getNamedThreadFactory(final String prefix) {
-    SecurityManager s = System.getSecurityManager();
-    final ThreadGroup threadGroup = (s != null) ? s.getThreadGroup() : Thread.currentThread()
-        .getThreadGroup();
-
-    return new ThreadFactory() {
-      final AtomicInteger threadNumber = new AtomicInteger(1);
-      private final int poolNumber = Threads.poolNumber.getAndIncrement();
-      final ThreadGroup group = threadGroup;
-
-      @Override
-      public Thread newThread(Runnable r) {
-        final String name = prefix + "-pool" + poolNumber + "-t" + threadNumber.getAndIncrement();
-        return new Thread(group, r, name);
-      }
-    };
-  }
-
-  /**
-   * Same as {#newDaemonThreadFactory(String, UncaughtExceptionHandler)},
-   * without setting the exception handler.
-   */
-  public static ThreadFactory newDaemonThreadFactory(final String prefix) {
-    return newDaemonThreadFactory(prefix, null);
-  }
-
-  /**
-   * Get a named {@link ThreadFactory} that just builds daemon threads.
-   * @param prefix name prefix for all threads created from the factory
-   * @param handler unhandles exception handler to set for all threads
-   * @return a thread factory that creates named, daemon threads with
-   *         the supplied exception handler and normal priority
-   */
-  public static ThreadFactory newDaemonThreadFactory(final String prefix,
-      final UncaughtExceptionHandler handler) {
-    final ThreadFactory namedFactory = getNamedThreadFactory(prefix);
-    return new ThreadFactory() {
-      @Override
-      public Thread newThread(Runnable r) {
-        Thread t = namedFactory.newThread(r);
-        if (handler != null) {
-          t.setUncaughtExceptionHandler(handler);
-        } else {

Review comment:
       compare with hbase-shaded-miscellaneous
   ```
     private static ThreadFactory doBuild(ThreadFactoryBuilder builder) {
       final String nameFormat = builder.nameFormat;
       final Boolean daemon = builder.daemon;
       final Integer priority = builder.priority;
       final UncaughtExceptionHandler uncaughtExceptionHandler = builder.uncaughtExceptionHandler;
       final ThreadFactory backingThreadFactory =
           (builder.backingThreadFactory != null)
               ? builder.backingThreadFactory
               : Executors.defaultThreadFactory();
       final AtomicLong count = (nameFormat != null) ? new AtomicLong(0) : null;
       return new ThreadFactory() {
         @Override
         public Thread newThread(Runnable runnable) {
           Thread thread = backingThreadFactory.newThread(runnable);
           if (nameFormat != null) {
             thread.setName(format(nameFormat, count.getAndIncrement()));
           }
           if (daemon != null) {
             thread.setDaemon(daemon);
           }
           if (priority != null) {
             thread.setPriority(priority);
           }
           if (uncaughtExceptionHandler != null) {
             thread.setUncaughtExceptionHandler(uncaughtExceptionHandler);
           }
           return thread;
         }
       };
     }
   
   ```

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
##########
@@ -197,83 +198,14 @@ public static ThreadPoolExecutor getBoundedCachedThreadPool(
     return boundedCachedThreadPool;
   }
 
-  public static ThreadPoolExecutor getBoundedCachedThreadPool(int maxCachedThread, long timeout,
-      TimeUnit unit, String prefix) {
-    return getBoundedCachedThreadPool(maxCachedThread, timeout, unit,
-        newDaemonThreadFactory(prefix));
-  }
-
-  /**
-   * Returns a {@link java.util.concurrent.ThreadFactory} that names each created thread uniquely,
-   * with a common prefix.
-   * @param prefix The prefix of every created Thread's name
-   * @return a {@link java.util.concurrent.ThreadFactory} that names threads
-   */
-  public static ThreadFactory getNamedThreadFactory(final String prefix) {
-    SecurityManager s = System.getSecurityManager();
-    final ThreadGroup threadGroup = (s != null) ? s.getThreadGroup() : Thread.currentThread()
-        .getThreadGroup();
-
-    return new ThreadFactory() {
-      final AtomicInteger threadNumber = new AtomicInteger(1);
-      private final int poolNumber = Threads.poolNumber.getAndIncrement();
-      final ThreadGroup group = threadGroup;
-
-      @Override
-      public Thread newThread(Runnable r) {
-        final String name = prefix + "-pool" + poolNumber + "-t" + threadNumber.getAndIncrement();
-        return new Thread(group, r, name);
-      }
-    };
-  }
-
-  /**
-   * Same as {#newDaemonThreadFactory(String, UncaughtExceptionHandler)},
-   * without setting the exception handler.
-   */
-  public static ThreadFactory newDaemonThreadFactory(final String prefix) {
-    return newDaemonThreadFactory(prefix, null);
-  }
-
-  /**
-   * Get a named {@link ThreadFactory} that just builds daemon threads.
-   * @param prefix name prefix for all threads created from the factory
-   * @param handler unhandles exception handler to set for all threads
-   * @return a thread factory that creates named, daemon threads with
-   *         the supplied exception handler and normal priority
-   */
-  public static ThreadFactory newDaemonThreadFactory(final String prefix,
-      final UncaughtExceptionHandler handler) {
-    final ThreadFactory namedFactory = getNamedThreadFactory(prefix);
-    return new ThreadFactory() {
-      @Override
-      public Thread newThread(Runnable r) {
-        Thread t = namedFactory.newThread(r);
-        if (handler != null) {
-          t.setUncaughtExceptionHandler(handler);
-        } else {

Review comment:
       @virajjasani Sorry,I did not explain clear, I wonder if here may cause Inconsistent?  In guava's builder class, if we do not assign UncaughtExceptionHandler, the UncaughtExceptionHandler will be null, not LOGGING_EXCEPTION_HANDLER




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org