incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jake Luciani <jak...@gmail.com>
Subject Re: insert performance (1.2.8)
Date Mon, 26 Aug 2013 21:06:37 GMT
How are you inserting the data? Is it all partition at once?

We've had the experience that shuffling the inserts across rows for wide
rows gave us "normal" insert rates.  When you mutate a entire wide row at
once it hits a bottleneck.


On Mon, Aug 26, 2013 at 4:49 PM, Keith Freeman <8forty@gmail.com> wrote:

>  I can believe that I'm IO bound with the current disk configuration, but
> that doesn't explain the CPU load does it?  If I'm hitting a limit of disk
> performance, I should see a slowdown but not the jump in CPU, right?
>
>
> On 08/22/2013 11:52 AM, Nate McCall wrote:
>
> Given the backups in the flushing stages, I think you are IO bound. SSDs
> will work best for the data volume. Use rotational media for the commitlog
> as it is largely sequential.
>
>  Quick experiment: disable commit log on the keyspace and see if your
> test goes faster ("WITH DURABLE_WRITES = false" on keyspace creation).
>
>
> On Wed, Aug 21, 2013 at 5:41 PM, Keith Freeman <8forty@gmail.com> wrote:
>
>>  We have 2 partitions on the same physical disk for commit-log and data.
>> Definitely non-optimal, we're planning to install SSDs for the commit-log
>> partition but don't have them yet.
>>
>> Can this explain the high server loads?
>>
>> On 08/21/2013 04:24 PM, Nate McCall wrote:
>>
>> What's the disk setup like on these system? You have some pending tasks
>> in MemtablePostFlusher and FlushWriter which may mean there is contention
>> on flushing discarded segments from the commit log.
>>
>>
>> On Wed, Aug 21, 2013 at 5:14 PM, Keith Freeman <8forty@gmail.com> wrote:
>>
>>>  Ok, I tried batching 500 at a time, made no noticeable difference in
>>> the server loads.  I have been monitoring JMX via jconsole if that's what
>>> you mean?  I also did tpstats on all 3 nodes while it was under load (the
>>> 5000 rows/sec test).  Attached file contains a screen shot of the JMX and
>>> the output of the 3 tpstats commands.
>>>
>>>
>>> On 08/21/2013 02:16 PM, Nate McCall wrote:
>>>
>>> The only thing I can think to suggest at this point is upping that batch
>>> size - say to 500 and see what happens.
>>>
>>>  Do you have any monitoring on this cluster? If not, what do you see as
>>> the output of 'nodetool tpstats' while you run this test?
>>>
>>>
>>> On Wed, Aug 21, 2013 at 1:40 PM, Keith Freeman <8forty@gmail.com> wrote:
>>>
>>>>  Building the giant batch string wasn't as bad as I thought, and at
>>>> first I had great(!) results (using "unlogged" batches): 2500 rows/sec
>>>> (batches of 100 in 48 threads) ran very smoothly, and the load on the
>>>> cassandra server nodes averaged about 1.0 or less continuously.
>>>>
>>>> But then I upped it to 5000 rows/sec, and the load on the server nodes
>>>> jumped to a continuous load on all 3 of 8-10 with peaks over 14.  I also
>>>> tried running 2 separate clients at 2500 rows/sec with the same results.
 I
>>>> don't see any compactions while at this load, so would this likely be the
>>>> result of GC thrashing?
>>>>
>>>> Seems like I'm spending a lot of effort and am still not getting very
>>>> close to being able to insert 10k rows (10M of data each) per second, which
>>>> is pretty disappointing.
>>>>
>>>>
>>>> On 08/20/2013 07:16 PM, Nate McCall wrote:
>>>>
>>>> Thrift will allow for more large, free-form batch contstruction. The
>>>> increase will be doing a lot more in the same payload message. Otherwise
>>>> CQL is more efficient.
>>>>
>>>>  If you do build those giant string, yes you should see a performance
>>>> improvement.
>>>>
>>>>
>>>> On Tue, Aug 20, 2013 at 8:03 PM, Keith Freeman <8forty@gmail.com>wrote:
>>>>
>>>>>  Thanks.  Can you tell me why would using thrift would improve
>>>>> performance?
>>>>>
>>>>> Also, if I do try to build those giant strings for a prepared batch
>>>>> statement, should I expect another performance improvement?
>>>>>
>>>>>
>>>>>
>>>>> On 08/20/2013 05:06 PM, Nate McCall wrote:
>>>>>
>>>>> Ugh - sorry, I knew Sylvain and Michaƫl had worked on this recently
>>>>> but it is only in 2.0 - I could have sworn it got marked for inclusion
>>>>> back into 1.2 but I was wrong:
>>>>> https://issues.apache.org/jira/browse/CASSANDRA-4693
>>>>>
>>>>>  This is indeed an issue if you don't know the column count before
>>>>> hand (or had a very large number of them like in your case). Again,
>>>>> apologies, I would not have recommended that route if I knew it was only
in
>>>>> 2.0.
>>>>>
>>>>>  I would be willing to bet you could hit those insert numbers pretty
>>>>> easily with thrift given the shape of your mutation.
>>>>>
>>>>>
>>>>> On Tue, Aug 20, 2013 at 5:00 PM, Keith Freeman <8forty@gmail.com>wrote:
>>>>>
>>>>>>  So I tried inserting prepared statements separately (no batch),
and
>>>>>> my server nodes load definitely dropped significantly.  Throughput
from my
>>>>>> client improved a bit, but only a few %.  I was able to *almost*
get 5000
>>>>>> rows/sec (sort of) by also reducing the rows/insert-thread to 20-50
and
>>>>>> eliminating all overhead from the timing, i.e. timing only the tight
for
>>>>>> loop of inserts.  But that's still a lot slower than I expected.
>>>>>>
>>>>>> I couldn't do batches because the driver doesn't allow prepared
>>>>>> statements in a batch (QueryBuilder API).  It appears the batch itself
>>>>>> could possibly be a prepared statement, but since I have 40+ columns
on
>>>>>> each insert that would take some ugly code to build so I haven't
tried it
>>>>>> yet.
>>>>>>
>>>>>> I'm using CL "ONE" on the inserts and RF 2 in my schema.
>>>>>>
>>>>>>
>>>>>> On 08/20/2013 08:04 AM, Nate McCall wrote:
>>>>>>
>>>>>> John makes a good point re:prepared statements (I'd increase batch
>>>>>> sizes again once you did this as well - separate, incremental runs
of
>>>>>> course so you can gauge the effect of each). That should take out
some of
>>>>>> the processing overhead of statement validation in the server (some
- that
>>>>>> load spike still seems high though).
>>>>>>
>>>>>>  I'd actually be really interested as to what your results were
>>>>>> after doing so - i've not tried any A/B testing here for prepared
>>>>>> statements on inserts.
>>>>>>
>>>>>>  Given your load is on the server, i'm not sure adding more async
>>>>>> indirection on the client would buy you too much though.
>>>>>>
>>>>>>  Also, at what RF and consistency level are you writing?
>>>>>>
>>>>>>
>>>>>> On Tue, Aug 20, 2013 at 8:56 AM, Keith Freeman <8forty@gmail.com>wrote:
>>>>>>
>>>>>>>  Ok, I'll try prepared statements.   But while sending my statements
>>>>>>> async might speed up my client, it wouldn't improve throughput
on the
>>>>>>> cassandra nodes would it?  They're running at pretty high loads
and only
>>>>>>> about 10% idle, so my concern is that they can't handle the data
any
>>>>>>> faster, so something's wrong on the server side.  I don't really
think
>>>>>>> there's anything on the client side that matters for this problem.
>>>>>>>
>>>>>>> Of course I know there are obvious h/w things I can do to improve
>>>>>>> server performance: SSDs, more RAM, more cores, etc.  But I thought
the
>>>>>>> servers I have would be able to handle more rows/sec than say
Mysql, since
>>>>>>> write speed is supposed to be one of Cassandra's strengths.
>>>>>>>
>>>>>>>
>>>>>>> On 08/19/2013 09:03 PM, John Sanda wrote:
>>>>>>>
>>>>>>> I'd suggest using prepared statements that you initialize at
>>>>>>> application start up and switching to use Session.executeAsync
coupled with
>>>>>>> Google Guava Futures API to get better throughput on the client
side.
>>>>>>>
>>>>>>>
>>>>>>> On Mon, Aug 19, 2013 at 10:14 PM, Keith Freeman <8forty@gmail.com>wrote:
>>>>>>>
>>>>>>>>  Sure, I've tried different numbers for batches and threads,
but
>>>>>>>> generally I'm running 10-30 threads at a time on the client,
each sending a
>>>>>>>> batch of 100 insert statements in every call, using the
>>>>>>>> QueryBuilder.batch() API from the latest datastax java driver,
then calling
>>>>>>>> the Session.execute() function (synchronous) on the Batch.
>>>>>>>>
>>>>>>>> I can't post my code, but my client does this on each iteration:
>>>>>>>> -- divides up the set of inserts by the number of threads
>>>>>>>> -- stores the current time
>>>>>>>> -- tells all the threads to send their inserts
>>>>>>>> -- then when they've all returned checks the elapsed time
>>>>>>>>
>>>>>>>> At about 2000 rows for each iteration, 20 threads with 100
inserts
>>>>>>>> each finish in about 1 second.  For 4000 rows, 40 threads
with 100 inserts
>>>>>>>> each finish in about 1.5 - 2 seconds, and as I said all 3
cassandra nodes
>>>>>>>> have a heavy CPU load while the client is hardly loaded.
 I've tried with
>>>>>>>> 10 threads and more inserts per batch, or up to 60 threads
with fewer,
>>>>>>>> doesn't seem to make a lot of difference.
>>>>>>>>
>>>>>>>>
>>>>>>>> On 08/19/2013 05:00 PM, Nate McCall wrote:
>>>>>>>>
>>>>>>>>  How big are the batch sizes? In other words, how many rows
are
>>>>>>>> you sending per insert operation?
>>>>>>>>
>>>>>>>>  Other than the above, not much else to suggest without seeing
>>>>>>>> some example code (on pastebin, gist or similar, ideally).
>>>>>>>>
>>>>>>>> On Mon, Aug 19, 2013 at 5:49 PM, Keith Freeman <8forty@gmail.com>wrote:
>>>>>>>>
>>>>>>>>> I've got a 3-node cassandra cluster (16G/4-core VMs ESXi
v5 on
>>>>>>>>> 2.5Ghz machines not shared with any other VMs).  I'm
inserting time-series
>>>>>>>>> data into a single column-family using "wide rows" (timeuuids)
and have a
>>>>>>>>> 3-part partition key so my primary key is something like
((a, b, day),
>>>>>>>>> in-time-uuid), x, y, z).
>>>>>>>>>
>>>>>>>>> My java client is feeding rows (about 1k of raw data
size each) in
>>>>>>>>> batches using multiple threads, and the fastest I can
get it run reliably
>>>>>>>>> is about 2000 rows/second.  Even at that speed, all 3
cassandra nodes are
>>>>>>>>> very CPU bound, with loads of 6-9 each (and the client
machine is hardly
>>>>>>>>> breaking a sweat).  I've tried turning off compression
in my table which
>>>>>>>>> reduced the loads slightly but not much.  There are no
other updates or
>>>>>>>>> reads occurring, except the datastax opscenter.
>>>>>>>>>
>>>>>>>>> I was expecting to be able to insert at least 10k rows/second
with
>>>>>>>>> this configuration, and after a lot of reading of docs,
blogs, and google,
>>>>>>>>> can't really figure out what's slowing my client down.
 When I increase the
>>>>>>>>> insert speed of my client beyond 2000/second, the server
responses are just
>>>>>>>>> too slow and the client falls behind.  I had a single-node
Mysql database
>>>>>>>>> that can handle 10k of these data rows/second, so I really
feel like I'm
>>>>>>>>> missing something in Cassandra.  Any ideas?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>  --
>>>>>>>
>>>>>>> - John
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>
>>>>
>>>
>>>
>>
>>
>
>


-- 
http://twitter.com/tjake

Mime
View raw message