xinglin opened a new pull request, #5803:
URL: https://github.com/apache/hadoop/pull/5803

   
   
   ### Description of PR
   In [HDFS-17030](https://issues.apache.org/jira/browse/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.
   
   ```
     private final ExecutorService nnProbingThreadPool =
         new ThreadPoolExecutor(1, 4, 1L, TimeUnit.MINUTES,
             new ArrayBlockingQueue<Runnable>(1024));
   ```
   
   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)).
   
   
   
   ### How was this patch tested?
   
   ```
   ~/p/h/t/h/hadoop-hdfs (HDFS-17067)> mvn test 
-Dtest="TestObserverReadProxyProvider"
   [INFO] Running 
org.apache.hadoop.hdfs.server.namenode.ha.TestObserverReadProxyProvider
   [INFO] Tests run: 16, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 
7.965 s - in 
org.apache.hadoop.hdfs.server.namenode.ha.TestObserverReadProxyProvider
   [INFO]
   [INFO] Results:
   [INFO]
   [INFO] Tests run: 16, Failures: 0, Errors: 0, Skipped: 0
   ```
   
   ### For code changes:
   
   - [ ] Does the title or this PR starts with the corresponding JIRA issue id 
(e.g. 'HADOOP-17799. Your PR title ...')?
   - [ ] Object storage: have the integration tests been executed and the 
endpoint declared according to the connector-specific documentation?
   - [ ] If adding new dependencies to the code, are these dependencies 
licensed in a way that is compatible for inclusion under [ASF 
2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the `LICENSE`, `LICENSE-binary`, 
`NOTICE-binary` files?
   
   


-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


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

Reply via email to