> To put in other words, Cassandra will lock down all tables until all pending > flush requests fit in the pending queue. This was the first issue I looked at in my Cassandra SF talk http://www.datastax.com/events/cassandrasummit2012/presentations
I've seen it occur more often with lots-o-secondary indexes. > We had plenty of write I/O available. We also had free memory. I increased > the memtable_flush_writers to "2" and memtable_flush_queue_size to "8". We > haven't had any timeouts for a number of days now. Cool. Cheers ----------------- Aaron Morton Freelance Developer @aaronmorton http://www.thelastpickle.com On 24/09/2012, at 6:09 AM, Michael Theroux <mthero...@yahoo.com> wrote: > There were no errors in the log (other than the messages dropped exception > pasted below), and the node does recover. We have only a small number of > secondary indexes (3 in the whole system). > > However, I went through the cassandra code, and I believe I've worked through > this problem. > > Just to finish out this thread, I realized that when you see: > > INFO [ScheduledTasks:1] 2012-09-17 06:28:03,840 StatusLogger.java (line > 72) FlushWriter 1 5 0 > > It is an issue. Cassandra will at various times enqueue many memtables for > flushing. By default, the queue size for this is 4. If more than 5 > memtables get queued for flushing (4 + 1 for the one currently being > flushed), a lock will be acquired and held across all tables until all > memtables that need to be flushed are enqueued. If it takes more than > rpc_timeout_time_in_ms time to flush enough information to allow all the > pending memtables to be enqueued, a "messages dropped" will occur. To put in > other words, Cassandra will lock down all tables until all pending flush > requests fit in the pending queue. If your queue size is 4, and 8 tables > need to be flushed, Cassandra will lock down all tables until a minimum of 3 > memtables are flushed. > > With this in mind, I went through the cassandra log and found this was indeed > the case looking at log entries similar to these: > > INFO [OptionalTasks:1] 2012-09-16 05:54:29,750 ColumnFamilyStore.java (line > 643) Enqueuing flush of Memtable-p@1525015234(18686281/341486464 > serialized/live bytes, 29553 ops) > ... > INFO [FlushWriter:29] 2012-09-16 05:54:29,768 Memtable.java (line 266) > Writing Memtable-p@1525015234(18686281/341486464 serialized/live bytes, 29553 > ops) > ... > INFO [FlushWriter:29] 2012-09-16 05:54:30,254 Memtable.java (line 307) > Completed flushing /data/cassandra/data/open/people/open-p-hd-441-Data.db > > I was able to figure out what the rpc_timeout_in_ms needed to be to > temporarily prevent the problem. > > We had plenty of write I/O available. We also had free memory. I increased > the memtable_flush_writers to "2" and memtable_flush_queue_size to "8". We > haven't had any timeouts for a number of days now. > > Thanks for your help, > -Mike > > On Sep 18, 2012, at 5:14 AM, aaron morton wrote: > >> Any errors in the log ? >> >> The node recovers ? >> >> Do you use secondary indexes ? If so check comments for >> memtable_flush_queue_size in the yaml. if this value is too low writes may >> back up. But I would not expect it to cause dropped messages. >> >>> nodetool info also shows we have over a gig of available memory on the JVM >>> heap of each node. >> >> Not all memory is created equal :) >> ParNew is kicking in to GC the Eden space in the New Heap. >> >> It may just be that the node is getting hammered by something and IO is >> getting overwhelmed. If you can put the logs up someone might take a look. >> >> Cheers >> >> ----------------- >> Aaron Morton >> Freelance Developer >> @aaronmorton >> http://www.thelastpickle.com >> >> On 18/09/2012, at 3:46 PM, Michael Theroux <mthero...@yahoo.com> wrote: >> >>> Thanks for the response. >>> >>> We are on version 1.1.2. We don't see the MutationStage back up. The dump >>> from the messages dropped error doesn't show a backup, but also watching >>> "nodetool tpstats" doesn't show any backup there. >>> >>> nodetool info also shows we have over a gig of available memory on the JVM >>> heap of each node. >>> >>> The earliest GCInspector traces I see before one of the more recent >>> incidents in which messages were dropped are: >>> >>> INFO [ScheduledTasks:1] 2012-09-18 02:25:53,928 GCInspector.java (line >>> 122) GC for ParNew: 396 ms for 1 collections, 2064505088 used; max is >>> 4253024256 >>> >>> NFO [ScheduledTasks:1] 2012-09-18 02:25:55,929 GCInspector.java (line >>> 122) GC for ParNew: 485 ms for 1 collections, 1961875064 used; max is >>> 4253024256 >>> >>> INFO [ScheduledTasks:1] 2012-09-18 02:25:57,930 GCInspector.java (line >>> 122) GC for ParNew: 265 ms for 1 collections, 1968074096 used; max is >>> 4253024256 >>> >>> But this was 45 minutes before messages were dropped. >>> >>> It's appreciated, >>> -Mike >>> >>> On Sep 17, 2012, at 11:27 PM, aaron morton wrote: >>> >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,839 StatusLogger.java (line >>>>> 72) MemtablePostFlusher 1 5 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,840 StatusLogger.java (line >>>>> 72) FlushWriter 1 5 0 >>>> Looks suspiciously like >>>> http://mail-archives.apache.org/mod_mbox/cassandra-user/201209.mbox/%3c9fb0e801-b1ed-41c4-9939-bafbddf15...@thelastpickle.com%3E >>>> >>>> What version are you on ? >>>> >>>> Are there any ERROR log messages before this ? >>>> >>>> Are you seeing MutationStage back up ? >>>> >>>> Are you see log messages from GCInspector ? >>>> >>>> Cheers >>>> >>>> ----------------- >>>> Aaron Morton >>>> Freelance Developer >>>> @aaronmorton >>>> http://www.thelastpickle.com >>>> >>>> On 18/09/2012, at 2:16 AM, Michael Theroux <mthero...@yahoo.com> wrote: >>>> >>>>> Hello, >>>>> >>>>> While under load, we have occasionally been seeing "messages dropped" >>>>> errors in our cassandra log. Doing some research, I understand this is >>>>> part of Cassandra's design to shed load, and we should look at the >>>>> tpstats-like output to determine what should be done to resolve the >>>>> situation. Typically, you will see lots of messages blocked or pending, >>>>> and that might be an indicator that a specific part of hardware needs to >>>>> be improved/tuned/upgraded. >>>>> >>>>> However, looking at the output we are getting, I'm finding it difficult >>>>> to see what needs to be tuned, as it looks to me cassandra is handling >>>>> the load within the mutation stage: >>>>> >>>>> NFO [ScheduledTasks:1] 2012-09-17 06:28:03,266 MessagingService.java >>>>> (line 658) 119 MUTATION messages dropped in last 5000ms >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,645 StatusLogger.java (line >>>>> 57) Pool Name Active Pending Blocked >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,836 StatusLogger.java (line >>>>> 72) ReadStage 3 3 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,837 StatusLogger.java (line >>>>> 72) RequestResponseStage 0 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,837 StatusLogger.java (line >>>>> 72) ReadRepairStage 0 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,837 StatusLogger.java (line >>>>> 72) MutationStage 0 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,838 StatusLogger.java (line >>>>> 72) ReplicateOnWriteStage 0 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,838 StatusLogger.java (line >>>>> 72) GossipStage 0 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,839 StatusLogger.java (line >>>>> 72) AntiEntropyStage 0 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,839 StatusLogger.java (line >>>>> 72) MigrationStage 0 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,839 StatusLogger.java (line >>>>> 72) StreamStage 0 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,839 StatusLogger.java (line >>>>> 72) MemtablePostFlusher 1 5 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,840 StatusLogger.java (line >>>>> 72) FlushWriter 1 5 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,840 StatusLogger.java (line >>>>> 72) MiscStage 0 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,840 StatusLogger.java (line >>>>> 72) commitlog_archiver 0 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,841 StatusLogger.java (line >>>>> 72) InternalResponseStage 0 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,841 StatusLogger.java (line >>>>> 72) AntiEntropySessions 0 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,851 StatusLogger.java (line >>>>> 72) HintedHandoff 0 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,851 StatusLogger.java (line >>>>> 77) CompactionManager 0 0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,852 StatusLogger.java (line >>>>> 89) MessagingService n/a 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,852 StatusLogger.java (line >>>>> 99) Cache Type Size Capacity >>>>> KeysToSave >>>>> Provider >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,853 StatusLogger.java (line >>>>> 100) KeyCache 2184533 2184533 >>>>> all >>>>> >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,853 StatusLogger.java (line >>>>> 106) RowCache 0 0 >>>>> all >>>>> org.apache.cassandra.cache.SerializingCacheProvider >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,853 StatusLogger.java (line >>>>> 113) ColumnFamily Memtable ops,data >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,853 StatusLogger.java (line >>>>> 116) system.NodeIdInfo 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,854 StatusLogger.java (line >>>>> 116) system.IndexInfo 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,854 StatusLogger.java (line >>>>> 116) system.LocationInfo 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,854 StatusLogger.java (line >>>>> 116) system.Versions 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,855 StatusLogger.java (line >>>>> 116) system.schema_keyspaces 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,855 StatusLogger.java (line >>>>> 116) system.Migrations 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,855 StatusLogger.java (line >>>>> 116) system.schema_columnfamilies 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,856 StatusLogger.java (line >>>>> 116) system.schema_columns 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,856 StatusLogger.java (line >>>>> 116) system.HintsColumnFamily 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,856 StatusLogger.java (line >>>>> 116) system.Schema 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,857 StatusLogger.java (line >>>>> 116) prod.comp 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,857 StatusLogger.java (line >>>>> 116) prod.byp 224,61863 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,857 StatusLogger.java (line >>>>> 116) prod.byn 30,6106 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,857 StatusLogger.java (line >>>>> 116) prod.p 3354,3792954 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,858 StatusLogger.java (line >>>>> 116) prod.rel 788,153294 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,858 StatusLogger.java (line >>>>> 116) prod.i 46,271517 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,859 StatusLogger.java (line >>>>> 116) prod.users 18489,5650405 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,859 StatusLogger.java (line >>>>> 116) prod.caches 40,7512998 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,859 StatusLogger.java (line >>>>> 116) prod.sessions 8085,13063121 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,859 StatusLogger.java (line >>>>> 116) prod.content 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,860 StatusLogger.java (line >>>>> 116) prod.enr 436,341190 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,860 StatusLogger.java (line >>>>> 116) prod.byt 23800,9880339 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,861 StatusLogger.java (line >>>>> 116) prod.aliases 63603,10475769 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,861 StatusLogger.java (line >>>>> 116) prod.ran 1500,329387 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,861 StatusLogger.java (line >>>>> 116) prod.cmap 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,861 StatusLogger.java (line >>>>> 116) prod.pmap 73032,61237450 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,862 StatusLogger.java (line >>>>> 116) prod.pic 5421,4438066 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,862 StatusLogger.java (line >>>>> 116) prod.watchdog 0,0 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,863 StatusLogger.java (line >>>>> 116) prod.s 31,40924 >>>>> INFO [ScheduledTasks:1] 2012-09-17 06:28:03,863 StatusLogger.java (line >>>>> 116) prod.m 21690,64262284 >>>>> >>>>> >>>>> We are using a write consistency level of LOCAL_QUROUM over a six node >>>>> cluster. Any observations are welcome, >>>>> >>>>> Thanks, >>>>> -Mike >>>> >>> >> >