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

ASF GitHub Bot commented on BOOKKEEPER-1065:
--------------------------------------------

Github user merlimat commented on the issue:

    https://github.com/apache/bookkeeper/pull/152
  
    I got a bit scared when I saw `ScheduledThreadPoolExecutor` is pass 
`MAX_INT` as the max thread pool size.. but I think the current code is 
correct. 
    
    I just did a quick verification and indeed it doesn't create more than one 
thread :
    
    ```java
    public static void main(String[] args) {
            ScheduledThreadPoolExecutor e = new ScheduledThreadPoolExecutor(1,
                    new 
ThreadFactoryBuilder().setNameFormat("test-%d").build());
    
            for (int i = 0; i < 10; i++) {
                final int n = i;
                e.submit(() -> {
                    System.out.println( Thread.currentThread().getName() + " |  
Start task " + n);
                    try {
                        Thread.sleep(1000);
    
                        System.out.println( Thread.currentThread().getName() + 
" |  Done task " + n);
                    } catch (InterruptedException e1) {
                        e1.printStackTrace();
                    }
                });
            }
    
            e.shutdown();
        }
    ```
    
    Closing this PR as no fix is needed 


> OrderedSafeExecutor should only have 1 thread per bucket
> --------------------------------------------------------
>
>                 Key: BOOKKEEPER-1065
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-1065
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Matteo Merli
>            Assignee: Matteo Merli
>             Fix For: 4.5.0
>
>
> In a earlier commit, "BOOKKEEPER-874: Explict LAC from Writer to Bookie", 
> there was this change in the OrderedSafeExecutor implementation: 
> {noformat}
>          for (int i = 0; i < numThreads; i++) {
> -            queues[i] = new LinkedBlockingQueue<Runnable>();
> -            threads[i] =  new ThreadPoolExecutor(1, 1,
> -                    0L, TimeUnit.MILLISECONDS, queues[i],
> +            threads[i] =  new ScheduledThreadPoolExecutor(1,
>                      new ThreadFactoryBuilder()
>                          .setNameFormat(name + "-orderedsafeexecutor-" + i + 
> "-%d")
>                          .setThreadFactory(threadFactory)
>                          .build());
> +            threads[i].setMaximumPoolSize(1);
> {noformat}
> Then, as part of "BOOKKEEPER-1013: Fix findbugs errors on latest master", the 
> max pool size line has been removed.
> {noformat}
> @@ -183,7 +183,6 @@ public class OrderedSafeExecutor {
>                          .setNameFormat(name + "-orderedsafeexecutor-" + i + 
> "-%d")
>                          .setThreadFactory(threadFactory)
>                          .build());
> -            threads[i].setMaximumPoolSize(1);
>              // Save thread ids
>              final int idx = i;
> {noformat}
> Without that the thread pool would create multiple threads for the same 
> bucket, breaking the ordering guarantee of the executor.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to