You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by rohit bhatia <ro...@gmail.com> on 2012/06/25 18:41:41 UTC

Interpreting system.log MeteredFlusher messages

Hi

We have 8 cassandra 1.0.5 nodes with 16 cores and 32G ram, Heap size
is 12G, memtable_total_space_in_mb is one third = 4G, There are 12 Hot
CFs (write-read ratio of 10).
memtable_flush_queue_size = 4 and memtable_flush_writers = 2..

I got this log-entry " MeteredFlusher.java (line 74) estimated
4239999318 bytes used by all memtables pre-flush", following which
cassandra flushed several of its "largest" memtables.
I understand that this message is due to the
"memtable_total_space_in_mb" setting being reached, but I do not
understand the remedy to the problem.
Is increasing this variable my only option?

Also, In standard MeteredFlusher flushes (the ones that trigger due to
"if my entire flush pipeline were full of memtables of this size, how
big could I allow them to be." logic),
I see memtables of serialized size of 100-200 MB with estimated live
size of 500 MB get flushed to produce sstables of around 10-15 MB
sizes.
Are these factors of 10-20 between serialized on disk and memory and
3-5 for liveRatio expected?

Also, this very informative article
http://thelastpickle.com/2011/05/04/How-are-Memtables-measured/ has
this to say
"For example if memtable_total_space_in_mb is 100MB, and
memtable_flush_writers is the default 1 (with one data directory), and
memtable_flush_queue_size is the default 4, and a Column Family has no
secondary indexes. The CF will not be allowed to get above one seventh
of 100MB or 14MB, as if the CF filled the flush pipeline with 7
memtables of this size it would take 98MB".
Since the formula is "CF Count + Secondary Index Count +
memtable_flush_queue_size (defaults to 4) + memtable_flush_writers
(defaults to 1 per data directory) memtables in memory the JVM at
once.", shouldn't the limit be 6 (and not 7) memtables in memory?


Thanks
Rohit

Re: Interpreting system.log MeteredFlusher messages

Posted by rohit bhatia <ro...@gmail.com>.
On Wed, Jun 27, 2012 at 2:27 PM, aaron morton <aa...@thelastpickle.com> wrote:
> , but I do not
> understand the remedy to the problem.
> Is increasing this variable my only option?
>
> There is nothing to be fixed. This is Cassandra flushing data to disk to
> free memory and checkpoint the commit log.
yes, but it induces simultaneous flushes of around 7-8 column families
which exceeds the flush queue size, I believe this can lead cassandra
to stop accepting writes.
>
> I see memtables of serialized size of 100-200 MB with estimated live
>
> size of 500 MB get flushed to produce sstables of around 10-15 MB
> sizes.
> Are these factors of 10-20 between serialized on disk and memory and
> 3-5 for liveRatio expected?
>
> Do you have some log messages for this ?
> The elevated estimated size may be due to a lot of overwrites.

Sample Log Message
" INFO [OptionalTasks:1] 2012-06-27 07:14:25,720 MeteredFlusher.java
(line 62) flushing high-traffic column family CFS(Keyspace='Stats',
ColumnFamily='Minutewise_Adtype_Customer_Stats') (estimated 529810674
bytes)
 INFO [OptionalTasks:1] 2012-06-27 07:14:25,721 ColumnFamilyStore.java
(line 688) Enqueuing flush of
Memtable-Minutewise_Adtype_Customer_Stats@1651281270(163641387/529810674
serialized/live bytes, 1633074 ops)
 INFO [FlushWriter:3808] 2012-06-27 07:14:25,727 Memtable.java (line
239) Writing Memtable-Minutewise_Adtype_Customer_Stats@1651281270(163641387/529810674
serialized/live bytes, 1633074 ops)
 INFO [FlushWriter:3808] 2012-06-27 07:14:26,131 Memtable.java (line
275) Completed flushing
/mnt/data/cassandra/data/Stats/Minutewise_Adtype_Customer_Stats-hb-70-Data.db
(6315581 bytes)"
Yes, there are overwrites. Since these are Counter Column family, it
sees a lot of increments,
Does cassandra store all the history for a column (and is there some
way to not store it)??

>
> Since the formula is "CF Count + Secondary Index Count +
> memtable_flush_queue_size (defaults to 4) + memtable_flush_writers
> (defaults to 1 per data directory) memtables in memory the JVM at
> once.", shouldn't the limit be 6 (and not 7) memtables in memory?
>
> It's 7
> because https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/MeteredFlusher.java#L51
Thanks a lot for this. I should have looked this up myself.
>
> Cheers
>
> -----------------
> Aaron Morton
> Freelance Developer
> @aaronmorton
> http://www.thelastpickle.com
>
> On 26/06/2012, at 4:41 AM, rohit bhatia wrote:
>
> Hi
>
> We have 8 cassandra 1.0.5 nodes with 16 cores and 32G ram, Heap size
> is 12G, memtable_total_space_in_mb is one third = 4G, There are 12 Hot
> CFs (write-read ratio of 10).
> memtable_flush_queue_size = 4 and memtable_flush_writers = 2..
>
> I got this log-entry " MeteredFlusher.java (line 74) estimated
> 4239999318 bytes used by all memtables pre-flush", following which
> cassandra flushed several of its "largest" memtables.
> I understand that this message is due to the
> "memtable_total_space_in_mb" setting being reached, but I do not
> understand the remedy to the problem.
> Is increasing this variable my only option?
>
> Also, In standard MeteredFlusher flushes (the ones that trigger due to
> "if my entire flush pipeline were full of memtables of this size, how
> big could I allow them to be." logic),
> I see memtables of serialized size of 100-200 MB with estimated live
> size of 500 MB get flushed to produce sstables of around 10-15 MB
> sizes.
> Are these factors of 10-20 between serialized on disk and memory and
> 3-5 for liveRatio expected?
>
> Also, this very informative article
> http://thelastpickle.com/2011/05/04/How-are-Memtables-measured/ has
> this to say
> "For example if memtable_total_space_in_mb is 100MB, and
> memtable_flush_writers is the default 1 (with one data directory), and
> memtable_flush_queue_size is the default 4, and a Column Family has no
> secondary indexes. The CF will not be allowed to get above one seventh
> of 100MB or 14MB, as if the CF filled the flush pipeline with 7
> memtables of this size it would take 98MB".
> Since the formula is "CF Count + Secondary Index Count +
> memtable_flush_queue_size (defaults to 4) + memtable_flush_writers
> (defaults to 1 per data directory) memtables in memory the JVM at
> once.", shouldn't the limit be 6 (and not 7) memtables in memory?
>
>
> Thanks
> Rohit
>
>

Re: Interpreting system.log MeteredFlusher messages

Posted by aaron morton <aa...@thelastpickle.com>.
> , but I do not
> understand the remedy to the problem.
> Is increasing this variable my only option?
There is nothing to be fixed. This is Cassandra flushing data to disk to free memory and checkpoint the commit log.

> I see memtables of serialized size of 100-200 MB with estimated live
> size of 500 MB get flushed to produce sstables of around 10-15 MB
> sizes.
> Are these factors of 10-20 between serialized on disk and memory and
> 3-5 for liveRatio expected?
Do you have some log messages for this ? 
The elevated estimated size may be due to a lot of overwrites.

> Since the formula is "CF Count + Secondary Index Count +
> memtable_flush_queue_size (defaults to 4) + memtable_flush_writers
> (defaults to 1 per data directory) memtables in memory the JVM at
> once.", shouldn't the limit be 6 (and not 7) memtables in memory?
It's 7 because https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/MeteredFlusher.java#L51

Cheers

-----------------
Aaron Morton
Freelance Developer
@aaronmorton
http://www.thelastpickle.com

On 26/06/2012, at 4:41 AM, rohit bhatia wrote:

> Hi
> 
> We have 8 cassandra 1.0.5 nodes with 16 cores and 32G ram, Heap size
> is 12G, memtable_total_space_in_mb is one third = 4G, There are 12 Hot
> CFs (write-read ratio of 10).
> memtable_flush_queue_size = 4 and memtable_flush_writers = 2..
> 
> I got this log-entry " MeteredFlusher.java (line 74) estimated
> 4239999318 bytes used by all memtables pre-flush", following which
> cassandra flushed several of its "largest" memtables.
> I understand that this message is due to the
> "memtable_total_space_in_mb" setting being reached, but I do not
> understand the remedy to the problem.
> Is increasing this variable my only option?
> 
> Also, In standard MeteredFlusher flushes (the ones that trigger due to
> "if my entire flush pipeline were full of memtables of this size, how
> big could I allow them to be." logic),
> I see memtables of serialized size of 100-200 MB with estimated live
> size of 500 MB get flushed to produce sstables of around 10-15 MB
> sizes.
> Are these factors of 10-20 between serialized on disk and memory and
> 3-5 for liveRatio expected?
> 
> Also, this very informative article
> http://thelastpickle.com/2011/05/04/How-are-Memtables-measured/ has
> this to say
> "For example if memtable_total_space_in_mb is 100MB, and
> memtable_flush_writers is the default 1 (with one data directory), and
> memtable_flush_queue_size is the default 4, and a Column Family has no
> secondary indexes. The CF will not be allowed to get above one seventh
> of 100MB or 14MB, as if the CF filled the flush pipeline with 7
> memtables of this size it would take 98MB".
> Since the formula is "CF Count + Secondary Index Count +
> memtable_flush_queue_size (defaults to 4) + memtable_flush_writers
> (defaults to 1 per data directory) memtables in memory the JVM at
> once.", shouldn't the limit be 6 (and not 7) memtables in memory?
> 
> 
> Thanks
> Rohit