HDDS-15114. Replace misconfigured ThreadPoolExecutor with Executors factory methods#10133
HDDS-15114. Replace misconfigured ThreadPoolExecutor with Executors factory methods#10133ptlrs wants to merge 2 commits intoapache:masterfrom
Conversation
There was a problem hiding this comment.
The %d placeholder will be substituted with the thread number, resulting in names like FetchOMDBTar-0somePrefix rather than the presumably intended FetchOMDBTar-somePrefix-0. This is a pre-existing issue, but since the code is being touched it's a good opportunity to fix it.
There was a problem hiding this comment.
Thanks for the review @devmadhuu. I have updated the code. I took a look at the other ThreadFactories in our code. I adopted their convention and moved the threadNamePrefix to be the initial part of the thread name.
|
Thanks @ptlrs for updating the patch. Can we add atleast one integration test testing the change because now there can be multiple concurrent worker threads running for downloading the OM DB tar for multiple sst files. May be a test that verifies tasks submitted to |
| if (executorServiceStarted.compareAndSet(false, true)) { | ||
| this.executor = | ||
| new ThreadPoolExecutor(0, threadPoolSize, 60L, TimeUnit.SECONDS, new LinkedBlockingQueue<>(), threadFactory); | ||
| this.executor = Executors.newFixedThreadPool(threadPoolSize, threadFactory); |
There was a problem hiding this comment.
Just a thought: The pool size is Math.max(64, Runtime.getRuntime().availableProcessors()) at the call site in OzoneManagerServiceProviderImpl. On high-CPU machines this could be large. With the old code it barely mattered; now it will actually create that many threads. On a 256-core host that's 256 threads all blocking on I/O at the same time. Consider documenting or probably IMO we should be capping this.
| new ThreadFactoryBuilder() | ||
| .setNameFormat("DataNodeMetricsCollector-%d") | ||
| .build()); | ||
| ThreadFactory threadFactory = new ThreadFactoryBuilder() |
There was a problem hiding this comment.
The original pool was intended (if misconfigured) to scale up to 500 threads under load (e.g., Recon polling thousands of DataNodes simultaneously). The fix correctly reflects actual behavior today, but it also avoiding efficient future scaling. If the intent was truly to allow bursting to a larger pool, can we think of using a bounded queue rather than simply removing it. At minimum, a comment explaining the corePoolSize choice and impact on removing the MAX_POOL_SIZE = 500.
Bu wait, this could make it slower in large cluster having hundreds of datanodes where it makes concurrent DataNode queries. The original MAX_POOL_SIZE = 500 captured the right intent — it was just broken by the unbounded LinkedBlockingQueue. A better fix would be to make the pool size configurable with a sensible default:
// e.g., add a config key OZONE_RECON_DN_METRICS_THREAD_COUNT, default 128 or min(nodeCount, 500)
int poolSize = config.getInt(OZONE_RECON_DN_METRICS_THREAD_COUNT, 128);
this.executorService = Executors.newFixedThreadPool(poolSize, threadFactory);
What changes were proposed in this pull request?
Some threadpools are misconfigured. Their effective number of threads is one or they never reach their maximum number of threads.
This PR:
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/HDDS-15114
How was this patch tested?
CI: https://github.com/ptlrs/ozone/actions/runs/24918410704