You question was
>>>> Could you please throw light on the what conditions does
>>>> MeteredFlusher use to trigger memtable flushes.

The answer is estimates of the ratio between the live size and the serialised 
size of memtables are kept. The MeteredFlusher periodically checks the 
serialised size of all memtables and uses the ratio to determine if 
memtable_total_space_in_mb has been reached. If there is a variation between 
nodes it may be that some are getting more traffic than others. 

>> So it implies that for flushing, Cassandra copies the memtables content.

> 
No

>> So does this imply that writes to column families are not stopped even
when it is being flushed?
Yes. 
In a worst case scenario writes will block if the memtable flushing cannot keep 
up. 

> Also, Could someone please explain how the factor of 7 comes in the
picture in this sentence

In the example (see previous para) 7 is the number of memtables the CF could 
have in memory at once (forgetting about the other cf's).  

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

On 7/06/2012, at 1:08 AM, rohit bhatia wrote:

> Also, Could someone please explain how the factor of 7 comes in the
> picture in this sentence
> 
> "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. "
> 
> On Wed, Jun 6, 2012 at 6:22 PM, rohit bhatia <rohit2...@gmail.com> wrote:
>> Hi..
>> 
>> the link http://thelastpickle.com/2011/05/04/How-are-Memtables-measured/
>> mentions that "From version 0.7 onwards the worse case scenario is up
>> to 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.".
>> 
>> So it implies that for flushing, Cassandra copies the memtables content.
>> So does this imply that writes to column families are not stopped even
>> when it is being flushed?
>> 
>> Thanks
>> Rohit
>> 
>> On Wed, Jun 6, 2012 at 9:42 AM, rohit bhatia <rohit2...@gmail.com> wrote:
>>> Hi Aaron
>>> 
>>> Thanks for the link, I have gone through it. But this doesn't justify
>>> nodes of exactly same config/specs differing in their flushing
>>> frequency.
>>> The traffic on all node is same as we are using RandomPartitioner
>>> 
>>> Thanks
>>> Rohit
>>> 
>>> On Wed, Jun 6, 2012 at 12:24 AM, aaron morton <aa...@thelastpickle.com> 
>>> wrote:
>>>> See the section on memtable_total_space_in_mb here
>>>>  http://thelastpickle.com/2011/05/04/How-are-Memtables-measured/
>>>> 
>>>> Cheers
>>>> -----------------
>>>> Aaron Morton
>>>> Freelance Developer
>>>> @aaronmorton
>>>> http://www.thelastpickle.com
>>>> 
>>>> On 6/06/2012, at 2:27 AM, rohit bhatia wrote:
>>>> 
>>>> I am trying to understand the variance in flushes frequency in a 8
>>>> node Cassandra cluster.
>>>> All the flushes are of the same type and initiated by MeteredFlusher.java 
>>>> =>
>>>> 
>>>> "INFO [OptionalTasks:1] 2012-06-05 06:32:05,873 MeteredFlusher.java
>>>> (line 62) flushing high-traffic column family CFS(Keyspace='Stats',
>>>> ColumnFamily='Minutewise_Channel_Stats') (estimated 501695882 bytes)"
>>>> [taken from system.log]
>>>> 
>>>> Number of flushes for 1 column family vary from 6 flushes per day to
>>>> 24 flushes per day among nodes of same configuration and same
>>>> hardware.
>>>> Could you please throw light on the what conditions does
>>>> MeteredFlusher use to trigger memtable flushes.
>>>> Also how accurate is the estimated size in the above logfile entry.
>>>> 
>>>> Regards
>>>> Rohit Bhatia
>>>> Software Engineer, Media.net
>>>> 
>>>> 

Reply via email to