[ 
https://issues.apache.org/jira/browse/HDDS-4535?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Attila Doroszlai updated HDDS-4535:
-----------------------------------
    Description: 
Number of threads for closed container replications can be adjusted by the 
settings  {{hdds.datanode.replication.streams.limit}}. But this number is 
ignored today due to the misuse of {{ThreadPoolExecutor}}:

{code}
new ThreadPoolExecutor(
        0, poolSize, 60, TimeUnit.SECONDS,
        new LinkedBlockingQueue<>(),
        new ThreadFactoryBuilder().setDaemon(true)
            .setNameFormat("ContainerReplicationThread-%d")
            .build())
{code}

Here the minimal number of threads is 0 and the maximum number of the threads 
is the configured value.  Threads in the thread pool supposed to be scaled up, 
but it doesn't.

[From the JDK 
docs|https://docs.oracle.com/javase/7/docs/api/java/util/concurrent/ThreadPoolExecutor.html#ThreadPoolExecutor(int,%20int,%20long,%20java.util.concurrent.TimeUnit,%20java.util.concurrent.BlockingQueue)]:

bq. A ThreadPoolExecutor will automatically adjust the pool size (see 
getPoolSize()) according to the bounds set by corePoolSize (see 
getCorePoolSize()) and maximumPoolSize (see getMaximumPoolSize()). When a new 
task is submitted in method execute(java.lang.Runnable), [...] [AND]  If there 
are more than corePoolSize but less than maximumPoolSize threads running, a new 
thread will be created only if the queue is full.

So if queue is not full (and {{LinkedBlockgingQueue}} is unbounded by default) 
the threads will never be created.

For a quick fix we can switch to use static thread pool instead of dynamic and 
always keep the required number of threads.

> Use fixed thread pool for closed container replication 
> -------------------------------------------------------
>
>                 Key: HDDS-4535
>                 URL: https://issues.apache.org/jira/browse/HDDS-4535
>             Project: Hadoop Distributed Data Store
>          Issue Type: Sub-task
>            Reporter: Marton Elek
>            Assignee: Marton Elek
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.1.0
>
>
> Number of threads for closed container replications can be adjusted by the 
> settings  {{hdds.datanode.replication.streams.limit}}. But this number is 
> ignored today due to the misuse of {{ThreadPoolExecutor}}:
> {code}
> new ThreadPoolExecutor(
>         0, poolSize, 60, TimeUnit.SECONDS,
>         new LinkedBlockingQueue<>(),
>         new ThreadFactoryBuilder().setDaemon(true)
>             .setNameFormat("ContainerReplicationThread-%d")
>             .build())
> {code}
> Here the minimal number of threads is 0 and the maximum number of the threads 
> is the configured value.  Threads in the thread pool supposed to be scaled 
> up, but it doesn't.
> [From the JDK 
> docs|https://docs.oracle.com/javase/7/docs/api/java/util/concurrent/ThreadPoolExecutor.html#ThreadPoolExecutor(int,%20int,%20long,%20java.util.concurrent.TimeUnit,%20java.util.concurrent.BlockingQueue)]:
> bq. A ThreadPoolExecutor will automatically adjust the pool size (see 
> getPoolSize()) according to the bounds set by corePoolSize (see 
> getCorePoolSize()) and maximumPoolSize (see getMaximumPoolSize()). When a new 
> task is submitted in method execute(java.lang.Runnable), [...] [AND]  If 
> there are more than corePoolSize but less than maximumPoolSize threads 
> running, a new thread will be created only if the queue is full.
> So if queue is not full (and {{LinkedBlockgingQueue}} is unbounded by 
> default) the threads will never be created.
> For a quick fix we can switch to use static thread pool instead of dynamic 
> and always keep the required number of threads.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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

Reply via email to