incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Nikolay Kоvshov <nkovs...@yandex.ru>
Subject Re: OOM on heavy write load
Date Wed, 27 Apr 2011 12:32:47 GMT
I have set quite low memory consumption (see my configuration in first message) and give Cassandra
2.7 Gb of memory. 
I cache 1M of 64-bytes keys + 64 Mb memtables. I believe overhead can't be 500% or so ?

memtable operations in millions = default 0.3 

I see now very strange behaviour

If i fill Cassandra with, say, 100 millions of 64B key + 64B data and after that I start inserting
64B key + 64 KB data, compaction queue immediately grows to hundreds and cassandra extremely
slows down (it makes aroung 30-50 operations/sec), then starts to give timeout errors. 

But if I insert 64B key + 64 KB data from the very beginning, cassandra works fine and makes
around 300 operations/sec even when the database exceeds 2-3 TB

The behaviour is quite complex and i cannot predict the effect of my actions. What consequences
I will have if I turn off compaction ? Can i read somewhere about what is compaction and why
it so heavily depends on what and in which order i write into cassandra ?

26.04.2011, 00:08, "Shu Zhang" <szhang@mediosystems.com>:
> the way I measure actual memtable row sizes is this
>
> write X rows into a cassandra node
> trigger GC
> record heap usage
> trigger compaction and GC
> record heap savings and divide by X for actual cassandra memtable row size in memory
>
> Similar process to measure per-key/per-row cache sizes for your data. To understand your
memtable row overhead size, you can do the above exercise with very different data sizes.
>
> Also, you probably know this, but when setting your memory usage ceiling or heap size,
make sure to leave a few hundred MBs for GC.
> ________________________________________
> From: Shu Zhang [szhang@mediosystems.com]
> Sent: Monday, April 25, 2011 12:55 PM
> To: user@cassandra.apache.org
> Subject: RE: OOM on heavy write load
>
> How large are your rows? binary_memtable_throughput_in_
> mb only tracks size of data, but there is an overhead associated with each row on the
order of magnitude of a few KBs. If your row data sizes are really small then the overhead
dominates the memory usage and binary_memtable_throughput_in_
> mb end up not limiting your memory usage the way you'd expect. It's a good idea to specify
memtable_operations_in_millions in that case. If you're not sure how big your data is compared
to memtable overhead, it's a good idea to specify both parameters to effectively put in a
memory ceiling no matter which dominates your actual memory usage.
>
> It could also be that your key cache is too big, you should measure your key sizes and
make sure you have enough memory to cache 1m keys (along with your memtables). Finally if
you have multiple keyspaces (for multiple applications) on your cluster, they all share the
total available heap, so you have to account for that.
>
> There's no measure in cassandra to guard against OOM, you must configure nodes such that
the max memory usage on each node, that is max size all your caches and memtables can potentially
grow to, is less than your heap size.
> ________________________________________
> From: Nikolay Kоvshov [nkovshov@yandex.ru]
> Sent: Monday, April 25, 2011 5:21 AM
> To: user@cassandra.apache.org
> Subject: Re: OOM on heavy write load
>
> I assume if I turn off swap it will just die earlier, no ? What is the mechanism of dying
?
>
> From the link you provided
>
> # Row cache is too large, or is caching large rows
> my row_cache is 0
>
> # The memtable sizes are too large for the amount of heap allocated to the JVM
> Is my memtable size too large ? I have made it less to surely fit the "magical formula"
>
> Trying to analyze heap dumps gives me the following:
>
> In one case diagram has 3 Memtables about 64 Mb each + 72 Mb "Thread" + 700 Mb "Unreachable
objects"
>
> suspected threats:
> 7 instances of "org.apache.cassandra.db.Memtable", loaded by "sun.misc.Launcher$AppClassLoader
@ 0x7f29f4992d68" occupy 456,292,912 (48.36%) bytes.
> 25,211 instances of "org.apache.cassandra.io.sstable.SSTableReader", loaded by "sun.misc.Launcher$AppClassLoader
@ 0x7f29f4992d68" occupy 294,908,984 (31.26%) byte
> 72 instances of "java.lang.Thread", loaded by "<system class loader>" occupy 143,632,624
(15.22%) bytes.
>
> In other cases memory analyzer hangs trying to parse 2Gb dump
>
> 22.04.2011, 17:26, "Jonathan Ellis" <jbellis@gmail.com>;;:
>
>>   (0) turn off swap
>>   (1) http://www.datastax.com/docs/0.7/troubleshooting/index#nodes-are-dying-with-oom-errors
>>
>>   On Fri, Apr 22, 2011 at 8:00 AM, Nikolay Kоvshov <nkovshov@yandex.ru>;;;
wrote:
>>>    I am using Cassandra 0.7.0 with following settings
>>>
>>>    binary_memtable_throughput_in_mb: 64
>>>    in_memory_compaction_limit_in_mb: 64
>>>    keys_cached 1 million
>>>    rows_cached 0
>>>
>>>    RAM for Cassandra 2 GB
>>>
>>>    I run very simple test
>>>
>>>    1 Node with 4 HDDs (1 HDD - commitlog and caches, 3 HDDs - data)
>>>    1 KS => 1 CF => 1 Column
>>>
>>>    I insert data (random key 64 bytes + value 64 bytes) at a maximum possible
speed, trying to hit disk i/o, calculate speed and make sure Cassandra stays alive. It doesn't,
unfortunately.
>>>    After several hundreds millions of inserts Cassandra always does down by
OOM. Getting it up again doesn't help - after inserting some new data it goes down again.
By this time Cassandra goes to swap and has a lot of tasks pending. I am not inserting anything
now and tasks sloooowly disappear, but it will take her weeks to do all of them.
>>>
>>>    compaction type: Minor
>>>    column family: Standard1
>>>    bytes compacted: 3661003227
>>>    bytes total in progress: 4176296448
>>>    pending tasks: 630
>>>
>>>    So, what am I (or Cassandra) doing wrong ? I dont want to get Cassandra
crashed without means of repair on heavy load circumstances.
>>   --
>>   Jonathan Ellis
>>   Project Chair, Apache Cassandra
>>   co-founder of DataStax, the source for professional Cassandra support
>>   http://www.datastax.com

Mime
View raw message