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

Jonathan Ellis commented on CASSANDRA-4718:
-------------------------------------------

Granted that a new executorservice won't help i/o bound workloads, but I knew 
that when I created the ticket and "must be significantly better for all 
workloads" is an unrealistically high bar for optimization work.  This gives us 
a pretty huge benefit on at least some workloads 
([1|http://riptano.github.io/cassandra_performance/graph_v3/graph.html?stats=stats.4718.bdplab.may12.threads-810-cql3_native_prepared.json&metric=op_rate&operation=4_read&smoothing=1&xmin=0&xmax=141.13&ymin=0&ymax=238843],
 
[2|http://riptano.github.io/cassandra_performance/graph_v3/graph.html?stats=stats.4718.ec2.may12.threads-810-cql3_native_prepared.json&metric=op_rate&operation=4_read&smoothing=1&xmin=0&xmax=134.31&ymin=0&ymax=340354.3])
 and a smaller benefit on others, which I'm quite happy with.  Unless the 
longer benchmarks Ryan is running show dramatically different results, I'm +1.

I also note that the work here is almost entirely self contained, with the 
major exception being some new code in Message.Dispatcher.  So while it's not 
as simple as dropping in LTQ or BAQ or FJP, the results are absolutely good 
enough to be worth a new Executor implementation.

> 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: Benedict
>            Priority: Minor
>              Labels: performance
>             Fix For: 2.1.0
>
>         Attachments: 4718-v1.patch, PerThreadQueue.java, aws.svg, 
> aws_read.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)

Reply via email to