You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Jason Brown (JIRA)" <ji...@apache.org> on 2013/12/07 01:30:38 UTC

[jira] [Commented] (CASSANDRA-4718) More-efficient ExecutorService for improved throughput

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

Jason Brown commented on CASSANDRA-4718:
----------------------------------------

OK, looks like my initial stab at switching over to FJP netted about 10-15% throughput increase, and mixed results on the latency scores (sometimes better, sometimes on par with trunk). I'm going run some more perf tests this weekend, and will decide how to proceed early next week - but the initial results do look promising. I've only tested the thrift endpoints so far, but when I retest this weekend, I'll throw in the cql3/native protocol, as well.

Here's my current working branch: https://github.com/jasobrown/cassandra/tree/4718_fjp . Note, it's very hacked up/WIP as I wanted to confirm the performance benefits before making everything happy (read: metrics pools). Also, I modified [~xedin]'s thrift-disruptor lib, for this: https://github.com/jasobrown/disruptor_thrift_server/tree/4718_fjp.

> More-efficient ExecutorService for improved throughput
> ------------------------------------------------------
>
>                 Key: CASSANDRA-4718
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-4718
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Jonathan Ellis
>            Assignee: Jason Brown
>            Priority: Minor
>              Labels: performance
>         Attachments: PerThreadQueue.java, baq vs trunk.png, op costs of various queues.ods, stress op rate with various queues.ods
>
>
> Currently all our execution stages dequeue tasks one at a time.  This can result in contention between producers and consumers (although we do our best to minimize this by using LinkedBlockingQueue).
> One approach to mitigating this would be to make consumer threads do more work in "bulk" instead of just one task per dequeue.  (Producer threads tend to be single-task oriented by nature, so I don't see an equivalent opportunity there.)
> BlockingQueue has a drainTo(collection, int) method that would be perfect for this.  However, no ExecutorService in the jdk supports using drainTo, nor could I google one.
> What I would like to do here is create just such a beast and wire it into (at least) the write and read stages.  (Other possible candidates for such an optimization, such as the CommitLog and OutboundTCPConnection, are not ExecutorService-based and will need to be one-offs.)
> AbstractExecutorService may be useful.  The implementations of ICommitLogExecutorService may also be useful. (Despite the name these are not actual ExecutorServices, although they share the most important properties of one.)



--
This message was sent by Atlassian JIRA
(v6.1#6144)