You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Dan Hendry <da...@gmail.com> on 2011/08/18 21:43:37 UTC

Memtable flush thresholds - what am I missing?

I am in the process of trying to tune the memtable flush thresholds for a
particular column family (super column family to be specific) in my
Cassandra 0.8.1 cluster. This CF is reasonably heavily used and getting
flushed roughly every 5-8 minutes which is hardly optimal, particularly
given I have JVM memory to spare at the moment. I am trying to understand
the Cassandra logs but the numbers I am seeing are not making any sense.

 

The initial memtable settings for this CF were throughput = 70 MB and
operations = 0.7  million. The flush messages I was seeing in the logs
(after a "flushing high-traffic column family" message for this CF) looked
like:

                "Enqueuing flush of Memtable-.... (17203504/600292480
serialized/live bytes, 320432 ops)"

 

So... uh... ~17 MB serialized, ~600 MB live (whatever that means), and ~320k
ops; the resulting sstables are ~34 MB. This is roughly what every flush
looks like. Two minutes before this particular flush, GC triggering the
StatusLogger shows ops and data for the CF as "122592,230094268" or 122k ops
(sensible) and 230 MB (what???). For at least 2 minutes prior to THAT
message, nothing else happened (flushes, compaction, etc) for any column
family which means that this series of events (flush to gc log entry to
flush) is reasonably isolated from any other activity. 

 

None of these numbers look even *remotely* close to 70 MB (the
memtable_throughput setting). Anyway, via JMX I went in and changed
throughput to 200 MB and operations to 0.5. This did *absolutely nothing* to
the flush behaviour: still ~17 MB serialized, ~600 MB live ~320k ops, ~34 MB
sstables, and flushes every 5-8 minutes (I waited for a few flushes in case
the change took some time to be applied). I also tried changing the
operations threshold to 0.2 million which DID work so it's not a case of the
settings not being respected.

 

WTF is going on? What is deciding that a flush is necessary and where are
all of these crazy size discrepancies coming from? Some additional info and
things to point out:

.         I am NOT seeing "the heap is X full, Cassandra will now flush the
two largest memtables warnings" or any other errors/unexpected things

.         The sum of memtable_throughput across all 10 CFs is 770 MB, well
less than the default global memtable threshold of ~4GB on a 12 GB java
heap. 

.         There are no major compactions running on this machine and no
repairs running across the cluster

.         Hinted handoff is disabled

 

Any insight would be appreciated.

 

Dan Hendry


RE: Memtable flush thresholds - what am I missing?

Posted by Dan Hendry <da...@gmail.com>.
Interesting.

Just to clarify, there are three main conditions which will trigger a flush
(based on data size):

1. The serialized size of a memtable exceeds the per CF memtable_throughput
setting.
2. For a single cf: (serialized size)*(live ratio)*(maximum possible
memtables in memory) > memtable_total_space_in_mb
3. sum_all_cf((serialized size)*(live ratio)) > memtable_total_space_in_mb

This makes a lot of sense to me, particularly in comparison to the 0.7 era
when the java overhead was not considered. 

The fact that memtable_total_space_in_mb and memtable_throughput (in MB) are
actually referring to different megabytes (live vs serialized) is pretty
confusing and should really be made more explicit in the cli and/or
cassandra.yaml.

Dan

-----Original Message-----
From: Jonathan Ellis [mailto:jbellis@gmail.com] 
Sent: August-18-11 15:51
To: user@cassandra.apache.org
Subject: Re: Memtable flush thresholds - what am I missing?

See http://thelastpickle.com/2011/05/04/How-are-Memtables-measured/,
specifically the section on memtable_total_space_in_mb

On Thu, Aug 18, 2011 at 2:43 PM, Dan Hendry <da...@gmail.com>
wrote:
> I am in the process of trying to tune the memtable flush thresholds for a
> particular column family (super column family to be specific) in my
> Cassandra 0.8.1 cluster. This CF is reasonably heavily used and getting
> flushed roughly every 5-8 minutes which is hardly optimal, particularly
> given I have JVM memory to spare at the moment. I am trying to understand
> the Cassandra logs but the numbers I am seeing are not making any sense.
>
>
>
> The initial memtable settings for this CF were throughput = 70 MB and
> operations = 0.7  million. The flush messages I was seeing in the logs
> (after a “flushing high-traffic column family” message for this CF) looked
> like:
>
>                 “Enqueuing flush of Memtable-.... (17203504/600292480
> serialized/live bytes, 320432 ops)”
>
>
>
> So... uh... ~17 MB serialized, ~600 MB live (whatever that means), and
~320k
> ops; the resulting sstables are ~34 MB. This is roughly what every flush
> looks like. Two minutes before this particular flush, GC triggering the
> StatusLogger shows ops and data for the CF as “122592,230094268” or 122k
ops
> (sensible) and 230 MB (what???). For at least 2 minutes prior to THAT
> message, nothing else happened (flushes, compaction, etc) for any column
> family which means that this series of events (flush to gc log entry to
> flush) is reasonably isolated from any other activity.
>
>
>
> None of these numbers look even *remotely* close to 70 MB (the
> memtable_throughput setting). Anyway, via JMX I went in and changed
> throughput to 200 MB and operations to 0.5. This did *absolutely nothing*
to
> the flush behaviour: still ~17 MB serialized, ~600 MB live ~320k ops, ~34
MB
> sstables, and flushes every 5-8 minutes (I waited for a few flushes in
case
> the change took some time to be applied). I also tried changing the
> operations threshold to 0.2 million which DID work so it’s not a case of
the
> settings not being respected.
>
>
>
> WTF is going on? What is deciding that a flush is necessary and where are
> all of these crazy size discrepancies coming from? Some additional info
and
> things to point out:
>
> ·         I am NOT seeing “the heap is X full, Cassandra will now flush
the
> two largest memtables warnings” or any other errors/unexpected things
>
> ·         The sum of memtable_throughput across all 10 CFs is 770 MB, well
> less than the default global memtable threshold of ~4GB on a 12 GB java
> heap.
>
> ·         There are no major compactions running on this machine and no
> repairs running across the cluster
>
> ·         Hinted handoff is disabled
>
>
>
> Any insight would be appreciated.
>
>
>
> Dan Hendry



-- 
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of DataStax, the source for professional Cassandra support
http://www.datastax.com
No virus found in this incoming message.
Checked by AVG - www.avg.com 
Version: 9.0.901 / Virus Database: 271.1.1/3842 - Release Date: 08/18/11
02:34:00


Re: Memtable flush thresholds - what am I missing?

Posted by Jonathan Ellis <jb...@gmail.com>.
See http://thelastpickle.com/2011/05/04/How-are-Memtables-measured/,
specifically the section on memtable_total_space_in_mb

On Thu, Aug 18, 2011 at 2:43 PM, Dan Hendry <da...@gmail.com> wrote:
> I am in the process of trying to tune the memtable flush thresholds for a
> particular column family (super column family to be specific) in my
> Cassandra 0.8.1 cluster. This CF is reasonably heavily used and getting
> flushed roughly every 5-8 minutes which is hardly optimal, particularly
> given I have JVM memory to spare at the moment. I am trying to understand
> the Cassandra logs but the numbers I am seeing are not making any sense.
>
>
>
> The initial memtable settings for this CF were throughput = 70 MB and
> operations = 0.7  million. The flush messages I was seeing in the logs
> (after a “flushing high-traffic column family” message for this CF) looked
> like:
>
>                 “Enqueuing flush of Memtable-.... (17203504/600292480
> serialized/live bytes, 320432 ops)”
>
>
>
> So... uh... ~17 MB serialized, ~600 MB live (whatever that means), and ~320k
> ops; the resulting sstables are ~34 MB. This is roughly what every flush
> looks like. Two minutes before this particular flush, GC triggering the
> StatusLogger shows ops and data for the CF as “122592,230094268” or 122k ops
> (sensible) and 230 MB (what???). For at least 2 minutes prior to THAT
> message, nothing else happened (flushes, compaction, etc) for any column
> family which means that this series of events (flush to gc log entry to
> flush) is reasonably isolated from any other activity.
>
>
>
> None of these numbers look even *remotely* close to 70 MB (the
> memtable_throughput setting). Anyway, via JMX I went in and changed
> throughput to 200 MB and operations to 0.5. This did *absolutely nothing* to
> the flush behaviour: still ~17 MB serialized, ~600 MB live ~320k ops, ~34 MB
> sstables, and flushes every 5-8 minutes (I waited for a few flushes in case
> the change took some time to be applied). I also tried changing the
> operations threshold to 0.2 million which DID work so it’s not a case of the
> settings not being respected.
>
>
>
> WTF is going on? What is deciding that a flush is necessary and where are
> all of these crazy size discrepancies coming from? Some additional info and
> things to point out:
>
> ·         I am NOT seeing “the heap is X full, Cassandra will now flush the
> two largest memtables warnings” or any other errors/unexpected things
>
> ·         The sum of memtable_throughput across all 10 CFs is 770 MB, well
> less than the default global memtable threshold of ~4GB on a 12 GB java
> heap.
>
> ·         There are no major compactions running on this machine and no
> repairs running across the cluster
>
> ·         Hinted handoff is disabled
>
>
>
> Any insight would be appreciated.
>
>
>
> Dan Hendry



-- 
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of DataStax, the source for professional Cassandra support
http://www.datastax.com