incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Peter Schuller <peter.schul...@infidyne.com>
Subject Re: Cassandra behaviour
Date Mon, 26 Jul 2010 16:57:31 GMT
[ 1 billion inserts, failed after 120m with out-of-mem ]

> - is Cassandra's RAM use proportional to the number of values that
> it's storing?  I know that it uses bloom filters for preventing
> lookups of non-existent keys, but since bloom filters are designed to
> give an accuracy/space tradeoff, Cassandra should sacrifice accuracy
> in order to prevent crashes, if it's just bloom filters that are using
> all the RAM

Bloom filters are indeed linear in size with respect to the number of
items (assuming a constant target false positive rate). While I have
not looked at how Cassandra calculates the bloom filter sizes, I feel
pretty confident in saying that it won't dynamically replace bloom
filters with filters of smaller sizes in response to memory pressure.
The number of implementation issues involved in doing that are many,
just that I can think of off the top of my head ;)

Also keep in mind that, at least as far as I can tell, silently
sacrificing false positive rates would not necessarily be an optimal
thing to do anyway.

If you application is such that you can accept a significantly higher
false positive rate on the bloom filters, the best bet is probably to
tweak it to use different target rates. I don't know if this is
currently possible without code changes (I don't think so, but I'm not
sure).

> - When I start Cassandra again, it appears to go into an eternal
> read/write loop, using between 45% and 90% of my CPU.  It says it's
> compacting tables, but it's been doing that for hours, and it only has
> 70GB of data stored.  How can cassandra be run on huge datasets, when
> 70GB appears to take forever to compact?

One aspect of start-up is reading through indexes, which will take
some time linear in the size of the indexes. Given 120M entries this
should not take terribly long.

WIth respect to compaction: In general, compactions may take up to
whatever time it takes to read and write the entire data set (in the
case of a full compaction). In addition, if your test threw write
traffic at the node faster than it was able to do compaction, you may
have a built-up backlog of compaction activity. I'm pretty sure there
is no active feedback mechanism (yet anyway) to prevent this from
happening (though IIRC it's been discussed on the lists).

> I assume I'm doing something wrong, but I don't see a ton of tunables
> to play with.  Can anybody give me advice on how to make cassandra
> keep running under a high insert load?

For a database with many items I would start by:

* Increasing the memtable size. Increasing memtable size directly
affects the number of times a given entry will end up having to be
compacted on average; i.e., it decreases the total compaction work
that must be done for a given insertion workload. The default is
something like 64 MB; on a large system you probably want this
significantly larger, even up to several gigs (depending on heap sizes
and other concerns of course).

* Making sure enough memory is reserved for the bloom filters.

For sustaining high read traffic you may then want to tweak cache
sizes, but that should not affect insertion.

-- 
/ Peter Schuller

Mime
View raw message