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, 07 Oct 2013 18:43:53 GMT

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

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

Disruptors are very difficult to use as a drop in replacement for the executor service, so
I tried to knock up some queues that could provide similar performance without ripping apart
the whole application. The resulting queues I benchmarked under high load, in isolation, against
LinkedBlockingQueue, BlockingArrayQueue and the Disruptor, and plotted the average op costs
in the "op costs of various queues" attachment*. As can be seen, these queues and the Disruptor
are substantially faster under high load than LinkedBlockingQueue, however it can also be
seen that:

- The average op cost for LinkedBlockingQueue is still very low, in fact only around 300ns
at worst
- BlockingArrayQueue is considerably worse than LinkedBlockingQueue under all conditions

These suggest both that the overhead attributed to LinkedBlockingQueue for a 1Mop workload
(as run above) should be at most a few seconds of the overall cost (probably much less); and
that BlockingArrayQueue is unlikely to make any cost incurred by LinkedBlockingQueue substantially
better. This made me suspect the previous result might be attributable to random variance,
but to be sure I ran a number of ccm -stress tests with the different queues, and plotted
the results in "stress op rate with various queues.ods", which show the following:

1) No meaningful difference between BAQ, LBQ and SlowQueue (though the latter has a clear
~1% slow down)
2) UltraSlow (~10x slow down, or 2000ns spinning each op) is approximately 5% slower
3) The faster queue actually slows down the process, by about 9% - more than the queue supposedly
much slower than it!

Anyway, I've been concurrently looking at where I might be able to improve performance independent
of this, and have found the following:

A) Raw performance of local reads is ~6-7x faster than through Stress
B) Raw performance of local reads run asynchronously is ~4x faster
C) Raw performance of local reads run asynchronously using the fast queue is ~4.7x faster
D) Performance of local reads from the Thrift server-side methods is ~3x faster
E) Performance of remote (i.e. local non-optimised) reads is ~1.5x faster

In particular (C) is interesting, as it demonstrates the queue really is faster in use, but
I've yet to absolutely determine why that translates into an overall decline in throughput.
It looks as though it's possible it causes greater congestion in LockSupport.unpark(), but
this is a new piece of information, derived from YourKit. As these sorts of methods are difficult
to meter accurately I don't necessarily trust it, and haven't had a chance to figure out what
I can do with the information. If it is accurate, and I can figure out how to reduce the overhead,
we might get a modest speed boost, which will accumulate as we find other places to improve.

As to the overall problem of improving throughput, it seems to me that there are two big avenues
to explore: 

  1) the networking (software) overhead is large;
  2) possibly the cost of managing thread liveness (e.g. park/unpark/scheduler costs); though
the evidence for this is as yet inconclusive... given the op rate and other evidence it doesn't
seem to be synchronization overhead. I'm still trying to pin this down.

Once the costs here are nailed down as tight as they can go, I'm pretty confident we can get
some noticeable improvements to the actual work being done, but since that currently accounts
for only a fraction of the time spent (probably less than 20%), I'd rather wait until it was
a higher percentage so any improvement is multiplied.


* These can be replicated by running org.apache.cassandra.concurrent.test.bench.Benchmark
on any of the linked branches on github. 

https://github.com/belliottsmith/cassandra/tree/4718-lbq [using LinkedBlockingQueue]
https://github.com/belliottsmith/cassandra/tree/4718-baq [using BlockingArrayQueue]
https://github.com/belliottsmith/cassandra/tree/4718-lpbq [using a new high performance queue]
https://github.com/belliottsmith/cassandra/tree/4718-slow [using a LinkedBlockingQueue with
200ns spinning each op]
https://github.com/belliottsmith/cassandra/tree/4718-ultraslow [using a LinkedBlockingQueue
with 2000ns spinning each op]


> 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: baq vs trunk.png, op costs of various queues.ods, PerThreadQueue.java,
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)

Mime
View raw message