Hi, we just had such an incident again, and this time we were able to grab a jstack-threaddump.
One thread from this stackdump actually shows a ReadVerbHandler calling the RateLimiter. From my understanding this should not be: Thread 30346: (state = BLOCKED) - java.lang.Thread.sleep(long) @bci=0 (Compiled frame; information may be imprecise) - java.lang.Thread.sleep(long, int) @bci=57, line=340 (Compiled frame) - java.util.concurrent.TimeUnit.sleep(long) @bci=23, line=386 (Compiled frame) - com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly(long, java.util.concurrent.TimeUnit) @bci=22, line=273 (Compiled frame) - com.google.common.util.concurrent.RateLimiter$SleepingTicker$1.sleepMicrosUninterruptibly(long) @bci=10, line=701 (Compiled frame) - com.google.common.util.concurrent.RateLimiter.acquire(int) @bci=42, line=405 (Compiled frame) - org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer() @bci=11, line=43 (Compiled frame) - org.apache.cassandra.io.util.RandomAccessReader.seek(long) @bci=147, line=287 (Compiled frame) - org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(long) @bci=22, line=65 (Compiled frame) - org.apache.cassandra.io.sstable.format.SSTableReader.getFileDataInput(long) @bci=5, line=1751 (Compiled frame) - org.apache.cassandra.io.sstable.format.big.IndexedSliceReader.setToRowStart(org.apache.cassandra.db.RowIndexEntry, org.apache.cassandra.io.util.FileDataInput) @bci=13, line=108 (Compiled frame) - org.apache.cassandra.io.sstable.format.big.IndexedSliceReader.<init>(org.apache.cassandra.io.sstable.format.SSTableReader, org.apache.cassandra.db.RowIndexEntry, org.apache.cassandra.io.util.FileDataInput, org.apache.cassandra.db.filter.ColumnSlice[], boolean) @bci=110, line=84 (Compiled frame) - org.apache.cassandra.io.sstable.format.big.SSTableSliceIterator.createReader(org.apache.cassandra.io.sstable.format.SSTableReader, org.apache.cassandra.db.RowIndexEntry, org.apache.cassandra.io.util.FileDataInput, org.apache.cassandra.db.filter.ColumnSlice[], boolean) @bci=54, line=66 (Compiled frame) - org.apache.cassandra.io.sstable.format.big.SSTableSliceIterator.<init>(org.apache.cassandra.io.sstable.format.SSTableReader, org.apache.cassandra.db.DecoratedKey, org.apache.cassandra.db.filter.ColumnSlice[], boolean) @bci=36, line=43 (Compiled frame) - org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(org.apache.cassandra.db.DecoratedKey, org.apache.cassandra.db.filter.ColumnSlice[], boolean) @bci=8, line=75 (Compiled frame) - org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(org.apache.cassandra.io.sstable.format.SSTableReader, org.apache.cassandra.db.DecoratedKey) @bci=10, line=246 (Compiled frame) - org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(org.apache.cassandra.io.sstable.format.SSTableReader) @bci=9, line=62 (Compiled frame) - org.apache.cassandra.db.CollationController.collectAllData(boolean) @bci=350, line=270 (Compiled frame) - org.apache.cassandra.db.CollationController.getTopLevelColumns(boolean) @bci=39, line=64 (Compiled frame) - org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(org.apache.cassandra.db.filter.QueryFilter, int) @bci=40, line=2011 (Compiled frame) - org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(org.apache.cassandra.db.filter.QueryFilter) @bci=141, line=1815 (Compiled frame) - org.apache.cassandra.db.Keyspace.getRow(org.apache.cassandra.db.filter.QueryFilter) @bci=11, line=360 (Compiled frame) - org.apache.cassandra.db.SliceFromReadCommand.getRow(org.apache.cassandra.db.Keyspace) @bci=222, line=85 (Compiled frame) - org.apache.cassandra.db.ReadVerbHandler.doVerb(org.apache.cassandra.net.MessageIn, int) @bci=39, line=38 (Compiled frame) - org.apache.cassandra.net.MessageDeliveryTask.run() @bci=82, line=67 (Compiled frame) - java.util.concurrent.Executors$RunnableAdapter.call() @bci=4, line=511 (Compiled frame) - org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run() @bci=5, line=164 (Compiled frame) - org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run() @bci=12, line=136 (Compiled frame) - org.apache.cassandra.concurrent.SEPWorker.run() @bci=87, line=105 (Compiled frame) - java.lang.Thread.run() @bci=11, line=745 (Compiled frame) Dominik and I would appreciate if you could give some feedback. We think it is a bug :-) Cheers, Christian On Thu, Mar 3, 2016 at 1:08 PM, Dominik Keil <dominik.k...@movilizer.com> wrote: > Hi, > > seeing this here as well. Basically at some point during a repair within > the cluster (incremental, parallel, partitioner range, only one node at a > time) some node (sometimes the repairing node, sometimes another) starts > piling up READs, i.e. pending reads go through the roof. Despite only one > node affected at a time and enough other, well behaving nodes, are > available to satisfy our quorum reads, this impacts the read performance of > the whole cluster. > > We have multiple C* 2.2 (2.2.5) clusters, basically running the same > application but with different amount of load. This symptom only appears on > one of our clusters, which has significantly more usage than most others. > This is also our largest cluster, having about 3 times as many machines as > most other ones (and RF 5 instead of RF 3). > > We did not see this before 2.0 and also only started to see this on that > particular cluster. We didn't see this on any other cluster after upgrading > form 2.0 (but then again, they're not as loaded). > > At first we suspected the incremental repair, because we did have issues > with that as well with heap pressure with 4GB of heap. Went to 6GB and no > more heap pressure but the problem persists. The suspect does not change as > this bad behaviour coincides with repair and specifically with > anticompaction going on. > > We see a clear timely correlation between open file handles rising and at > the same time active validations skyrocketing and the beginning of such an > "event". Same goes for the end of that event, which is clearly timely > correlated to the validations being completed and the number of file > handles dropping dramatically. However, these "beginning markers" seem to > be both present. Active validations skyrocketing without open file handlers > skyrocketing at the same time does not produce these symptoms. > > The only thing we have found so far that made a difference in these > situations is compaction throughput. When we decreased the compaction > throughput in these events, pending reads piled up even more and even more > quickly. Beyond that we're still pretty much in the dark. Anyway something > is locking up Cassandra internally. > > We suspect that there may be a "synchronized" somewhere it shouldn't be > (or should be solved differently) but that's just a guess. We'll try to > produce some jstacks but the events are pretty elusive because they happen > suddenly and don't last very long (except when we're not watching closely > -_-). > > Anyway, here's some graphs to illustrate what I've tried to describe: > > (1) CPU Usage of Cassandra (green) and open file descriptors (yellow, > second y-axis) > > > (2) Threads: new threads per second (orange line, second y-axis) > > > (3) See below; active and pending tasks on second y-axis > > > (4) Compactions and Validations: Active Tasks (Bars, second y-axis) and > Completed Tasks / s (lines) > > > You can see around 08:03 an event started with a sudden rise in active > validations and multiple sudden increases in open file descriptors. The > event lasts until 08:46 with a sudden drop in open file descriptors and a > huge peak in new threads per second. > > During the event you can see Cassandra's CPU usage drops significantly. > Same goes for GC activity (graph not included here, because STW GC only > happens about once every 50 minutes and then takes only a fraction of a > second). > > As you can see there's another such event later on but much smaller and > shorter and between the events the pattern with the validations continues > the same way without problems - only difference: No significant change in > open file descriptor count. > > I have system graphs as well but not included because they show no > problems: CPU usage goes down during that event, no I/O wait on the CPU and > disk OP/s as well as throughput actually go down as well. > > During the depicted time frame there was a repair (incremental, parallel, > partitioner range) running on a different machine within the cluster. We've > switched back to -pr because when running it without -pr these event happen > more often and more articulated but I think that it's just affected by the > same underlying problem. > > Interestingly we had a similar issue in another cluster last night, which > runs C* 2.1.13 and does NOT yet use incremental repair (just full repair > with -pr). > > Any chance something in the read path is affected by the set compaction > throughput and/or running compactions? It definitely seems that Cassandra > is severly restricting itself here. > > Best regards, > Dominik > > > Am 26.02.2016 um 17:42 schrieb horschi: > > Hi, > > I just had a weird behaviour on one of our Cassandra nodes, which I would > like to share: > > Short version: > My pending reads went up from ~0 to the hundreds when I reduced the > compactionthroughput from 16 to 2. > > > Long version: > > One of our more powerful nodes had a few pending reads, while the other > ones didn't. So far nothing special. > > Strangely neither CPU, nor IO Wait, nor disk-ops/s, nor C*-heap was > particularly high. So I was wondering. > > That machine had two compactions and a validation(incremental) running, so > I set the compactionthroughput to 2. To my surprise I saw the pending reads > go up to the hundreds within 5-10 seconds. Setting the compactionthroughput > back to 16 and the pending reads went back to 0 (or at least close to zero). > > I kept the compactionthroughput on 2 for less than a minute. So the issue > is not compactions falling behind. > > I was able to reproduce this behaviour 5-10 times. The pending reads went > up, everytime I *de*creased the compactionthroughput. I watched the > pending reads while the compactionthroughput was on 16, and I never > observed even a two digit pending read count while it was on > compactionthroughput 16. > > Unfortunetaly the machine does not show this behaviour any more. Also it > was only a single machine. > > > > Our setup: > C* 2.2.5 with 256 vnodes + 9 nodes + incremental repair + 6GB heap > > > My question: > Did someone else ever observe such a behaviour? > > Is it perhaps possible that the read-path shares a lock with > repair/compaction that waits on ThrottledReader while holding that lock? > > > kind regards, > Christian > > > -- > *Dominik Keil* > Phone: + 49 (0) 621 150 207 31 > Mobile: + 49 (0) 151 626 602 14 > > Movilizer GmbH > Julius-Hatry-Strasse 1 > 68163 Mannheim > Germany > > movilizer.com > > [image: Visit company website] <http://movilizer.com/> > *Reinvent Your Mobile Enterprise* > > <http://movilizer.com/training> > <http://movilizer.com/training> > > *Be the first to know:* > Twitter <https://twitter.com/Movilizer> | LinkedIn > <https://www.linkedin.com/company/movilizer-gmbh> | Facebook > <https://www.facebook.com/Movilizer> | stack overflow > <http://stackoverflow.com/questions/tagged/movilizer> > > Company's registered office: Mannheim HRB: 700323 / Country Court: > Mannheim Managing Directors: Alberto Zamora, Jörg Bernauer, Oliver Lesche > Please inform us immediately if this e-mail and/or any attachment was > transmitted incompletely or was not intelligible. > > This e-mail and any attachment is for authorized use by the intended > recipient(s) only. It may contain proprietary material, confidential > information and/or be subject to legal privilege. It should not be > copied, disclosed to, retained or used by any other party. If you are not > an intended recipient then please promptly delete this e-mail and any > attachment and all copies and inform the sender.