[ https://issues.apache.org/jira/browse/CASSANDRA-3494?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13161045#comment-13161045 ]
Peter Schuller commented on CASSANDRA-3494: ------------------------------------------- We had it set very high. Throttling is not the problem here. The problem is that each node was sending data to only one bootstrapping node at a time, and that process was bottlenecking on the destination node's write capacity (see CASSANDRA-3549 too). Another effect is that nodes weren't evenly bootstrapping across the cluster; so some nodes were at 200 gigs whiles other were at 500, because of the pseudo-random ordering restriction imposed by this limitation (multiple destinations dogpiling on a source -> some have to just wait for a long time before they even begin receiving data). In addition to the aggregate bandwidth limit, this caused even more delay to get to the state where all nodes were bootstrapped. > 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 > Priority: Minor > > 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