[ 
https://issues.apache.org/jira/browse/HDFS-17067?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17741782#comment-17741782
 ] 

ASF GitHub Bot commented on HDFS-17067:
---------------------------------------

xinglin commented on code in PR #5803:
URL: https://github.com/apache/hadoop/pull/5803#discussion_r1258987581


##########
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProvider.java:
##########
@@ -648,6 +645,7 @@ public synchronized void close() throws IOException {
       }
     }
     failoverProxy.close();
+    nnProbingThreadPool.shutdown();

Review Comment:
   The issue is previously we did not set _allowCoreThreadTimeout_ to true. So, 
the core thread will be kept running, though there is no new tasks submitted. 
As long as there is a thread running, JVM won't be shut down (unless we force 
it with an explicit `System.exit()`).





> Use BlockingThreadPoolExecutorService for nnProbingThreadPool in 
> ObserverReadProxy
> ----------------------------------------------------------------------------------
>
>                 Key: HDFS-17067
>                 URL: https://issues.apache.org/jira/browse/HDFS-17067
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: hdfs
>    Affects Versions: 3.4.0
>            Reporter: Xing Lin
>            Assignee: Xing Lin
>            Priority: Major
>              Labels: pull-request-available
>
> In HDFS-17030, we introduced an ExecutorService, to submit 
> getHAServiceState() requests. We constructed the ExecutorService directly 
> from a basic ThreadPoolExecutor, without setting _allowCoreThreadTimeOut_ to 
> true. Then, the core thread will be kept up and running even when the main 
> thread exits. To fix it, one could set _allowCoreThreadTimeOut_ to true. 
> However, in this PR, we decide to directly use an existing executorService 
> implementation (_BlockingThreadPoolExecutorService_) in hadoop instead. It 
> takes care of setting _allowCoreThreadTimeOut_ and also allows setting the 
> prefix for thread names.
> {code:java}
>   private final ExecutorService nnProbingThreadPool =
>       new ThreadPoolExecutor(1, 4, 1L, TimeUnit.MINUTES,
>           new ArrayBlockingQueue<Runnable>(1024));
> {code}
> A second minor issue is we did not shutdown the executorService in close(). 
> It is a minor issue as close() will only be called when the garbage collector 
> starts to reclaim an ObserverReadProxyProvider object, not when there is no 
> reference to the ObserverReadProxyProvider object. The time between when an 
> ObserverReadProxyProvider becomes dereferenced and when the garage collector 
> actually starts to reclaim that object is out of control/under-defined 
> (unless the program is shutdown with an explicit System.exit(1)).



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to