Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 931C3BDB2 for ; Mon, 2 Jan 2012 11:09:05 +0000 (UTC) Received: (qmail 89992 invoked by uid 500); 2 Jan 2012 11:09:05 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 89578 invoked by uid 500); 2 Jan 2012 11:08:57 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 89560 invoked by uid 99); 2 Jan 2012 11:08:55 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 02 Jan 2012 11:08:55 +0000 X-ASF-Spam-Status: No, hits=-2001.6 required=5.0 tests=ALL_TRUSTED,RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.116] (HELO hel.zones.apache.org) (140.211.11.116) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 02 Jan 2012 11:08:52 +0000 Received: from hel.zones.apache.org (hel.zones.apache.org [140.211.11.116]) by hel.zones.apache.org (Postfix) with ESMTP id D542F134335 for ; Mon, 2 Jan 2012 11:08:30 +0000 (UTC) Date: Mon, 2 Jan 2012 11:08:30 +0000 (UTC) From: "Sylvain Lebresne (Commented) (JIRA)" To: commits@cassandra.apache.org Message-ID: <1147180916.57655.1325502510874.JavaMail.tomcat@hel.zones.apache.org> In-Reply-To: <1937809468.31534.1321375372965.JavaMail.tomcat@hel.zones.apache.org> Subject: [jira] [Commented] (CASSANDRA-3494) Streaming is mono-threaded (the bulk loader too by extension) MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 X-Virus-Checked: Checked by ClamAV on apache.org [ 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(), 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