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 Wed, 14 May 2014 20:41:18 GMT

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

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

Thanks [~enigmacurry]!

Those graphs all look pretty good to me. Think it's time to run some of the longer tests to
see that performance is still good for other workloads. Let's drop thrift from the equation
now.

I'd suggest something like 

write n=600000000 -key populate=1..600000000 
force major compaction
for each thread count/branch:
 read n=100000000 -key dist=extr(1..600000000,2)
and warm up with one (any) read test run before the rest, so that they all are playing from
a roughly level page cache point

This should create a dataset in the region of 110Gb, but around 75% of requests will be to
~40Gb of it, which should be in the region of the amount of page cache available to the EC2
systems after bloom filters etc. are accounted for

NB: if you want to play with different distributions, cassandra-stress print lets you see
what a spec would yield

> 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)

Mime
View raw message