You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Sylvain Lebresne (Commented) (JIRA)" <ji...@apache.org> on 2012/01/02 12:08:30 UTC

[jira] [Commented] (CASSANDRA-3494) Streaming is mono-threaded (the bulk loader too by extension)

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

Sylvain Lebresne commented on CASSANDRA-3494:
---------------------------------------------

This work because of some details of ThreadPoolExecutor. Namely, in the sentence "When a new task is submitted in method execute(java.lang.Runnable), and fewer than corePoolSize threads are running, a new thread is created to handle the request, ...", the "fewer" is meant strictly. So with a 0 for corePoolSize, it will create a thread for the first task. It will also terminate that thread afterward because the doc says "If the pool currently has more than corePoolSize threads, excess threads will be terminated if they have been idle for more than the keepAliveTime".

However, this is dodgy code as for instance if maxCorePoolSize was 10, the thread pool would *not* create up to 10 threads, it would create one and then queue up all other tasks.

In fact, the documentation is actually inconsistent, as for unbounded queues it states that "Thus, no more than corePoolSize threads will ever be created. (And the value of the maximumPoolSize therefore doesn't have any effect.)". But this is not true if corePoolSize is 0.

I believe the correct way to create an executor with a max number of threads and where all threads timeout if unused with ThreadPoolExecutor is to set corePoolSize == maxPoolSize == whateverTheMaxShouldBe (1 for that patch) and to use ThreadPoolExecutor.allowCoreTheadTimeout().
                
> Streaming is mono-threaded (the bulk loader too by extension)
> -------------------------------------------------------------
>
>                 Key: CASSANDRA-3494
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3494
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Core
>    Affects Versions: 0.8.0
>            Reporter: Sylvain Lebresne
>            Assignee: Peter Schuller
>            Priority: Minor
>             Fix For: 1.1
>
>         Attachments: CASSANDRA-3494-0.8-prelim.txt, CASSANDRA-3494-1.0.txt
>
>
> The streamExecutor is define as:
> {noformat}
> streamExecutor_ = new DebuggableThreadPoolExecutor("Streaming", Thread.MIN_PRIORITY);
> {noformat}
> In the meantime, in DebuggableThreadPoolExecutor.java:
> {noformat}
> public DebuggableThreadPoolExecutor(String threadPoolName, int priority)
> {
>    this(1, Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), new NamedThreadFactory(threadPoolName, priority));
> }
> {noformat}
> In other word, since the core pool size is 1 and the queue unbounded, tasks will always queued and the executor is essentially mono-threaded.
> This is clearly not necessary since we already have stream throttling nowadays. And it could be a limiting factor in the case of the bulk loader.
> Besides, I would venture that this maybe was not the intention, because putting the max core size to MAX_VALUE would suggest that the intention was to spawn threads on demand. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira