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/09 04:41:33 UTC

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

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