cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Benedict (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-4718) More-efficient ExecutorService for improved throughput
Date Mon, 12 May 2014 22:50:15 GMT

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

Benedict commented on CASSANDRA-4718:
-------------------------------------

I have uploaded a complete rewrite [here|https://github.com/belliottsmith/cassandra/tree/4718-sep]

This on my tests is another 10%+ faster than lse-batchnetty, making it roughly on-par with
2.1-batchnetty on our old hardware, but thoroughly outstripping it on EC2 and my laptop. I
still need to introduce some thorough tests for it, and to comment the code thoroughly, but
the basic principle is the same only all executors share the same pool of worker threads so
that scheduling is easier, and work can be passed more easily between them.

I will revisit this work again sometime in the next year to see if we can squeeze anything
more out of this, especially as we add more optimisations elsewhere - but for now we're reaching
diminishing returns.

[~enigmacurry] can you run a comparison of this and just cassandra-2.1-batchnetty on bdplab
and EC2, so we can get a final comparison? [~jasobrown] if you feel like kicking off a run
of this latest branch on your hardware so we have as many final data points to compare against
that would also be really helpful. I'll get the code commented early tomorrow so we can get
this reviewed ASAP.

> 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
>             Fix For: 2.1.0
>
>         Attachments: 4718-v1.patch, PerThreadQueue.java, aws.svg, backpressure-stress.out.txt,
baq vs trunk.png, belliotsmith_branches-stress.out.txt, jason_read.svg, jason_read_latency.svg,
jason_write.svg, op costs of various queues.ods, stress op rate with various queues.ods, v1-stress.out
>
>
> 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.2#6252)

Mime
View raw message