> (probably > worth mentioning that we have a lot more cf's than the 4 in that log > excerpt): Yeah, that and running on multiple SSD's per node :)
With SSD's I would look at increasing the memtable_flush_writers and leaving the memtable_flush_queue_size in place. When you see IO util too high then try the flush_queue_size. May as well use the IOPS > One thing I don't have a feel for is what the effects of increasing > the number of flushwriters would be, would you happen to have any > guidance on that? They both affect the calculation used when determining when to flush SSTables. Higher values result in cassandra being more conservative is in a worse case scenario it could have memtable_flush_writers + memtable_flush_queue_size full of memtables (there are other considerations). So it may result in smaller sstables, and more compaction. Cheers ----------------- Aaron Morton Freelance Cassandra Consultant New Zealand @aaronmorton http://www.thelastpickle.com On 20/03/2013, at 5:12 AM, Jasdeep Hundal <dsjas...@gmail.com> wrote: > No secondary indexes and I don't think it's IO. Commit log/data are on > separate SSD's (and C* is the only user of these disks), and the total > amount of data being written to the cluster is much less than the > SSD's are capable of writing (including replication I think we're at > about 10% of what the disks are capable of) (not sure how much of an > effect compaction has). > > Using jstack, we found some switchLock contention related to a flush > being blocked from joining the memtable flush queue. This was blocking > row mutations to that memtable. Increasing the queue size/the number > of flush writers has seemed to help the issue significantly (probably > worth mentioning that we have a lot more cf's than the 4 in that log > excerpt): > Pool Name Active Pending Completed Blocked > All time blocked > FlushWriter 9 9 29336 > 0 39 > > One thing I don't have a feel for is what the effects of increasing > the number of flushwriters would be, would you happen to have any > guidance on that? > > Thanks, > Jasdeep > > > On Tue, Mar 19, 2013 at 1:16 AM, aaron morton <aa...@thelastpickle.com> wrote: >> I checked the flushwriter thread pool stats and saw this: >> Pool Name Active Pending Completed Blocked >> All time blocked >> FlushWriter 1 5 86183 >> 1 17582 >> >> That's not good. >> Is the IO system over utilised ? >> >> In my setup, I have one extremely high traffic column family that is >> >> Any secondary indexes? If so see the comments for memtable_flush_queue_size >> in the yaml file. >> >> From the following log output, it looks like the cf with the large >> data load is blocking the flush of the other cf's. >> >> Not sure that is the case. >> In the log messages the commit log is rotating and needs to free up an old >> log segment (on the OptionalTasks thread) so it is flushing all of the CF's >> that have something written to the log segment. >> >> >> This could be that IO is not keeping up, it's unlikely to be a switch lock >> issue if you only have a 4 CF's. Also have you checked for GC messages in >> the C* logs ? >> >> Cheers >> >> >> ----------------- >> Aaron Morton >> Freelance Cassandra Consultant >> New Zealand >> >> @aaronmorton >> http://www.thelastpickle.com >> >> On 19/03/2013, at 12:25 PM, Jasdeep Hundal <dsjas...@gmail.com> wrote: >> >> Thanks for the info, got a couple of follow up questions, and just as >> a note, this is on Cassandra 1.2.0. >> >> I checked the flushwriter thread pool stats and saw this: >> Pool Name Active Pending Completed Blocked >> All time blocked >> FlushWriter 1 5 86183 >> 1 17582 >> >> Also, memtable_flush_queue_size is set to 4, and >> memtable_flush_writers is set to 1 >> >> In my setup, I have one extremely high traffic column family that is >> flushing lots of data at once (occasionally hitting hundreds of >> megabytes), and several smaller cf's for which flushes involve only a >> few bytes of data. >> >> From the following log output, it looks like the cf with the large >> data load is blocking the flush of the other cf's. Would increasing >> memtable_flush_queue_size (I've got plenty of memory) and >> memtable_flush_writers allow the smaller cf flushes to return faster? >> Or given that I see the smaller cf's being flushed when not much has >> been written to them, should I try reducing >> commit_log_segment_size_in_mb. >> >> 168026:2013-03-18 17:53:41,938 INFO [OptionalTasks:1] >> org.apache.cassandra.db.ColumnFamilyStore.switchMemtable >> ColumnFamilyStore.java:647) - Enqueuing flush of >> Memtable-data@2098591494(458518528/458518528 serialized/live bytes, >> 111732 ops) >> 168028:2013-03-18 17:53:47,064 INFO [OptionalTasks:1] >> org.apache.cassandra.db.ColumnFamilyStore.switchMemtable >> (ColumnFamilyStore.java:647) - Enqueuing flush of >> Memtable-metadata@252512204(2295/2295 serialized/live bytes, 64 ops) >> 168029:2013-03-18 17:53:47,065 INFO [OptionalTasks:1] >> org.apache.cassandra.db.ColumnFamilyStore.switchMemtable >> (ColumnFamilyStore.java:647) - Enqueuing flush of >> Memtable-counters@544926156(363/363 serialized/live bytes, 12 ops) >> 168030:2013-03-18 17:53:47,066 INFO [OptionalTasks:1] >> org.apache.cassandra.db.ColumnFamilyStore.switchMemtable >> (ColumnFamilyStore.java:647) - Enqueuing flush of >> Memtable-container_counters@1703633084(430/430 serialized/live bytes, >> 83 ops) >> 168032:2013-03-18 17:53:51,950 INFO [FlushWriter:3] >> org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents >> (Memtable.java:458) - Completed flushing >> /mnt/test/jasdeep/data/jasdeep-data-ia-720454-Data.db (391890130 >> bytes) for commitlog position ReplayPosition(segmentId=1363628611044, >> position=21069295) >> 168050:2013-03-18 17:53:55,948 INFO [FlushWriter:3] >> org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents >> (Memtable.java:458) - Completed flushing >> /mnt/test/jasdeep/metadata/jasdeep-metadata-ia-1280-Data.db (833 >> bytes) for commitlog position ReplayPosition(segmentId=1363628611047, >> position=4213859) >> 168052:2013-03-18 17:53:55,966 INFO [FlushWriter:3] >> org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents >> (Memtable.java:458) - Completed flushing >> /mnt/test/jasdeep/counters/jasdeep-counters-ia-1204-Data.db (342 >> bytes) for commitlog position ReplayPosition(segmentId=1363628611047, >> position=4213859) >> >> Thanks again, >> Jasdeep >> >> >> >> On Mon, Mar 18, 2013 at 10:24 AM, aaron morton <aa...@thelastpickle.com> >> wrote: >> >> 1. With a ConsistencyLevel of quorum, does >> FBUtilities.waitForFutures() wait for read repair to complete before >> returning? >> >> No >> That's just a utility method. >> Nothing on the read path waits for Read Repair, and controlled by >> read_repair_chance CF property, it's all async to the client request. >> There is no CL, the messages are sent to individual nodes. >> >> 2. When read repair applies a mutation, it needs to obtain a lock for >> the associated memtable. >> >> What lock are you referring to? >> When Read Repair (the RowDataResolver) wants to send a mutation it uses the >> MessageServer. On the write path there is a server wide RW lock call the >> sync lock. >> >> I've seen readrepair spend a few seconds stalling in >> org.apache.cassandra.db.Table.apply). >> >> This could be contention around the sync lock, look for blocked tasks in >> the flush writer thread pool. >> >> I did a talk on cassandra internals at Apache Con 3 weeks ago, not sure when >> the video is going to be up but here are the slides >> http://www.slideshare.net/aaronmorton/apachecon-nafeb2013 >> >> Cheers >> >> ----------------- >> Aaron Morton >> Freelance Cassandra Consultant >> New Zealand >> >> @aaronmorton >> http://www.thelastpickle.com >> >> On 16/03/2013, at 12:21 PM, Jasdeep Hundal <dsjas...@gmail.com> wrote: >> >> I've got a couple of questions related issues I'm encountering using >> Cassandra under a heavy write load: >> >> 1. With a ConsistencyLevel of quorum, does >> FBUtilities.waitForFutures() wait for read repair to complete before >> returning? >> 2. When read repair applies a mutation, it needs to obtain a lock for >> the associated memtable. Does compaction obtain this same lock? (I'm >> asking because I've seen readrepair spend a few seconds stalling in >> org.apache.cassandra.db.Table.apply). >> >> Thanks, >> Jasdeep >> >> >>