The histogram does not look right to me, too many SSTables for an LCS CF. It's a symptom no a cause. If LCS is catching up though it should be more like the distribution in the linked article.
Cheers ----------------- Aaron Morton Freelance Cassandra Developer New Zealand @aaronmorton http://www.thelastpickle.com On 23/01/2013, at 10:57 AM, Jim Cistaro <jcist...@netflix.com> wrote: > What version are you using? Are you seeing any compaction related assertions > in the logs? > > Might be https://issues.apache.org/jira/browse/CASSANDRA-4411 > > We experienced this problem of the count only decreasing to a certain number > and then stopping. If you are idle, it should go to 0. I have not seen it > overestimate for zero, only for non-zero amounts. > > As for timeouts etc, you will need to look at things like nodetool tpstats to > see if you have pending transactions queueing up. > > Jc > > From: Wei Zhu <wz1...@yahoo.com> > Reply-To: "user@cassandra.apache.org" <user@cassandra.apache.org>, Wei Zhu > <wz1...@yahoo.com> > Date: Tuesday, January 22, 2013 12:56 PM > To: "user@cassandra.apache.org" <user@cassandra.apache.org> > Subject: Re: Cassandra pending compaction tasks keeps increasing > > Thanks Aaron and Jim for your reply. The data import is done. We have about > 135G on each node and it's about 28K SStables. For normal operation, we only > have about 90 writes per seconds, but when I ran nodetool compationstats, it > remains at 9 and hardly changes. I guess it's just an estimated number. > > When I ran histogram, > > Offset SSTables Write Latency Read Latency Row Size > Column Count > 1 2644 0 0 0 > 18660057 > 2 8204 0 0 0 > 9824270 > 3 11198 0 0 0 > 6968475 > 4 4269 6 0 0 > 5510745 > 5 517 29 0 0 > 4595205 > > > You can see about half of the reads result in 3 SSTables. Majority of read > latency are under 5ms, only a dozen are over 10ms. We haven't fully turn on > reads yet, only 60 reads per second. We see about 20 read timeout during the > past 12 hours. Not a single warning from Cassandra Log. > > Is it normal for Cassandra to timeout some requests? We set rpc timeout to be > 1s, it shouldn't time out any of them? > > Thanks. > -Wei > > From: aaron morton <aa...@thelastpickle.com> > To: user@cassandra.apache.org > Sent: Monday, January 21, 2013 12:21 AM > Subject: Re: Cassandra pending compaction tasks keeps increasing > > The main guarantee LCS gives you is that most reads will only touch 1 row > http://www.datastax.com/dev/blog/when-to-use-leveled-compaction > > If compaction is falling behind this may not hold. > > nodetool cfhistograms tells you how many SSTables were read from for reads. > It's a recent histogram that resets each time you read from it. > > Also, parallel levelled compaction in 1.2 > http://www.datastax.com/dev/blog/performance-improvements-in-cassandra-1-2 > > Cheers > > ----------------- > Aaron Morton > Freelance Cassandra Developer > New Zealand > > @aaronmorton > http://www.thelastpickle.com > > On 20/01/2013, at 7:49 AM, Jim Cistaro <jcist...@netflix.com> wrote: > >> 1) In addition to iostat, dstat is a good tool to see wht kind of disck >> throuput your are getting. That would be one thing to monitor. >> 2) For LCS, we also see pending compactions skyrocket. During load, LCS >> will create a lot of small sstables which will queue up for compaction. >> 3) For us the biggest concern is not how high the pending count gets, but >> how often it gets back down near zero. If your load is something you can do >> in segments or pause, then you can see how fast the cluster recovers on the >> compactions. >> 4) One thing which we tune per cluster is the size of the files. Increasing >> this from 5MB can sometimes improve things. But I forget if we have ever >> changed this after starting data load. >> >> Is your cluster receiving read traffic during this data migration? If so, I >> would say that read latency is your best measure. If the high number of >> SSTables waiting to compact is not hurting your reads, then you are probably >> ok. Since you are on SSD, there is a good chance the compactions are not >> hurting you. As for compactionthroughput, we set ours high for SSD. You >> usually wont use it all because the compactions are usually single threaded. >> Dstat will help you measure this. >> >> I hope this helps, >> jc >> >> From: Wei Zhu <wz1...@yahoo.com> >> Reply-To: "user@cassandra.apache.org" <user@cassandra.apache.org>, Wei Zhu >> <wz1...@yahoo.com> >> Date: Friday, January 18, 2013 12:10 PM >> To: Cassandr usergroup <user@cassandra.apache.org> >> Subject: Cassandra pending compaction tasks keeps increasing >> >> Hi, >> When I run nodetool compactionstats >> >> I see the number of pending tasks keep going up steadily. >> >> I tried to increase the compactionthroughput, by using >> >> nodetool setcompactionthroughput >> >> I even tried the extreme to set it to 0 to disable the throttling. >> >> I checked iostats and we have SSD for data, the disk util is less than 5% >> which means it's not I/O bound, CPU is also less than 10% >> >> We are using levelcompaction and in the process of migrating data. We have >> 4500 writes per second and very few reads. We have about 70G data now and >> will grow to 150G when the migration finishes. We only have one CF and right >> now the number of SSTable is around 15000, write latency is still under >> 0.1ms. >> >> Anything needs to be concerned? Or anything I can do to reduce the number of >> pending compaction? >> >> Thanks. >> -Wei >> >> > > >