Thanks guys, these sound like good suggestions, will try those out.

Aaron, we have around 80 CFs.

From: aaron morton [mailto:aa...@thelastpickle.com]
Sent: Friday, June 28, 2013 10:05 PM
To: user@cassandra.apache.org
Subject: Re: about FlushWriter "All time blocked"

We do not use secondary indexes or snapshots
Out of interest how many CF's do you have ?

Cheers

-----------------
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 28/06/2013, at 7:52 AM, Nate McCall 
<zznat...@gmail.com<mailto:zznat...@gmail.com>> wrote:


Non-zero for pending tasks is too transient. Try monitoring tpstats
with a (much) higher frequency and look for sustained threshold over a
duration.

Then, using a percentage of the configuration values for the max - 75%
of memtable_flush_queue_size in this case - alert when it has been
higher than '3' for more than N time. (Start with N=60 seconds and go
from there).

Also, that is a very high 'all time blocked' to 'completed' ratio for
FlushWriter. If iostat is happy, i'd do as Aaron suggested above and
turn up the memtable_flush_queue_size and play around with turning up
memtable_flush_writers (incrementally and separately for both of
course so you can see the effect).

On Thu, Jun 27, 2013 at 2:27 AM, Arindam Barua 
<aba...@247-inc.com<mailto:aba...@247-inc.com>> wrote:

In our performance tests, we are seeing similar FlushWriter, MutationStage, 
MemtablePostFlusher pending tasks become non-zero. We collect snapshots every 5 
minutes, and they seem to clear after ~10-15 minutes though. (The flush writer 
has an 'All time blocked' count of 540 in the below example).

We do not use secondary indexes or snapshots. We do not use SSDs. We have a 
4-node cluster with around 30-40 GB data on each node. Each node has 3 1-TB 
disks with a RAID 0 setup.

Currently we monitor the tpstats every 5 minutes, and alert if FlushWriter or 
MutationStage has a non-zero Pending count. Any suggestions if this is a cause 
of concern already, or, should we alert only if that count becomes greater than 
a bigger number, say 10, or if the count remains non-zero greater than a 
specified time.

Pool Name                    Active   Pending      Completed   Blocked  All 
time blocked
ReadStage                         0         0       15685133         0          
       0
RequestResponseStage              0         0       29880863         0          
       0
MutationStage                     0         0       40457340         0          
       0
ReadRepairStage                   0         0         704322         0          
       0
ReplicateOnWriteStage             0         0              0         0          
       0
GossipStage                       0         0        2283062         0          
       0
AntiEntropyStage                  0         0              0         0          
       0
MigrationStage                    0         0             70         0          
       0
MemtablePostFlusher               1         1           1837         0          
       0
StreamStage                       0         0              0         0          
       0
FlushWriter                       1         1           1446         0          
     540
MiscStage                         0         0              0         0          
       0
commitlog_archiver                0         0              0         0          
       0
InternalResponseStage             0         0             43         0          
       0
HintedHandoff                     0         0              3         0          
       0

Thanks,
Arindam

-----Original Message-----
From: aaron morton [mailto:aa...@thelastpickle.com<http://thelastpickle.com>]
Sent: Tuesday, June 25, 2013 10:29 PM
To: user@cassandra.apache.org<mailto:user@cassandra.apache.org>
Subject: Re: about FlushWriter "All time blocked"


FlushWriter                       0         0            191         0          
      12

This means there were 12 times the code wanted to put an memtable in the queue 
to be flushed to disk but the queue was full.

The length of this queue is controlled by the memtable_flush_queue_size 
https://github.com/apache/cassandra/blob/cassandra-1.2/conf/cassandra.yaml#L299 
and memtable_flush_writers .

When this happens an internal lock around the commit log is held which prevents 
writes from being processed.

In general it means the IO system cannot keep up. It can sometimes happen when 
snapshot is used as all the CF's are flushed to disk at once. I also suspect it 
happens sometimes when a commit log segment is flushed and their are a lot of 
dirty CF's. But i've never proved it.

Increase memtable_flush_queue_size following the help in the yaml file. If you 
do not use secondary indexes are you using snapshot?

Hope that helps.
A
-----------------
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 24/06/2013, at 3:41 PM, yue.zhang 
<yue.zh...@chinacache.com<mailto:yue.zh...@chinacache.com>> wrote:


3 node
cent os
CPU 8core memory 32GB
cassandra 1.2.5
my scenario: many counter incr, every node has one client program, performance 
is 400 wps /every clicent (it’s so slowly)

my question:
Ø  nodetool tpstats
---------------------------------
Pool Name                    Active   Pending      Completed   Blocked  All 
time blocked
ReadStage                         0         0           8453         0          
       0
RequestResponseStage              0         0      138303982         0          
       0
MutationStage                     0         0      172002988         0          
       0
ReadRepairStage                   0         0              0         0          
       0
ReplicateOnWriteStage             0         0       82246354         0          
       0
GossipStage                       0         0        1052389         0          
       0
AntiEntropyStage                  0         0              0         0          
       0
MigrationStage                    0         0              0         0          
       0
MemtablePostFlusher               0         0            670         0          
       0
FlushWriter                       0         0            191         0          
      12
MiscStage                         0         0              0         0          
       0
commitlog_archiver                0         0              0         0          
       0
InternalResponseStage             0         0              0         0          
       0
HintedHandoff                     0         0             56         0          
       0
-----------------------------------
FlushWriter “All time blocked”=12,I restart the node,but no use,it’s normally ?

thx

-heipark



Reply via email to