Changed the 15GB node to 25GB heap and the nice CPU is down to ~20% now. Checked my dev cluster to see if the ParNew log entries are just par for the course, but not seeing them there. However, both have the following every 30 seconds:
DEBUG [BatchlogTasks:1] 2014-12-16 21:00:44,898 BatchlogManager.java (line 165) Started replayAllFailedBatches DEBUG [MemtablePostFlusher:1] 2014-12-16 21:00:44,899 ColumnFamilyStore.java (line 866) forceFlush requested but everything is clean in batchlog DEBUG [BatchlogTasks:1] 2014-12-16 21:00:44,899 BatchlogManager.java (line 200) Finished replayAllFailedBatches Is that just routine scheduled house-keeping or a sign of something else? On Tue, Dec 16, 2014 at 12:52 PM, Arne Claassen <a...@emotient.com> wrote: > > Sorry, I meant 15GB heap on the one machine that has less nice CPU% now. > The others are 6GB > > On Tue, Dec 16, 2014 at 12:50 PM, Arne Claassen <a...@emotient.com> wrote: >> >> AWS r3.xlarge, 30GB, but only using a Heap of 10GB, new 2GB because we >> might go c3.2xlarge instead if CPU is more important than RAM >> Storage is optimized EBS SSD (but iostat shows no real IO going on) >> Each node only has about 10GB with ownership of 67%, 64.7% & 68.3%. >> >> The node on which I set the Heap to 10GB from 6GB the utlilization has >> dropped to 46%nice now, but the ParNew log messages still continue at the >> same pace. I'm gonna up the HEAP to 20GB for a bit, see if that brings that >> nice CPU further down. >> >> No TombstoneOverflowingExceptions. >> >> On Tue, Dec 16, 2014 at 11:50 AM, Ryan Svihla <rsvi...@datastax.com> >> wrote: >>> >>> What's CPU, RAM, Storage layer, and data density per node? Exact heap >>> settings would be nice. In the logs look for TombstoneOverflowingException >>> >>> >>> On Tue, Dec 16, 2014 at 1:36 PM, Arne Claassen <a...@emotient.com> >>> wrote: >>>> >>>> I'm running 2.0.10. >>>> >>>> The data is all time series data and as we change our pipeline, we've >>>> been periodically been reprocessing the data sources, which causes each >>>> time series to be overwritten, i.e. every row per partition key is deleted >>>> and re-written, so I assume i've been collecting a bunch of tombstones. >>>> >>>> Also, the presence of the ever present and never completing compaction >>>> types, i assumed were an artifact of tombstoning, but i fully admit to >>>> conjecture based on about ~20 blog posts and stackoverflow questions i've >>>> surveyed. >>>> >>>> I doubled the Heap on one node and it changed nothing regarding the >>>> load or the ParNew log statements. New Generation Usage is 50%, Eden itself >>>> is 56%. >>>> >>>> Anything else i should look at and report, let me know. >>>> >>>> On Tue, Dec 16, 2014 at 11:14 AM, Jonathan Lacefield < >>>> jlacefi...@datastax.com> wrote: >>>>> >>>>> Hello, >>>>> >>>>> What version of Cassandra are you running? >>>>> >>>>> If it's 2.0, we recently experienced something similar with 8447 >>>>> [1], which 8485 [2] should hopefully resolve. >>>>> >>>>> Please note that 8447 is not related to tombstones. Tombstone >>>>> processing can put a lot of pressure on the heap as well. Why do you think >>>>> you have a lot of tombstones in that one particular table? >>>>> >>>>> [1] https://issues.apache.org/jira/browse/CASSANDRA-8447 >>>>> [2] https://issues.apache.org/jira/browse/CASSANDRA-8485 >>>>> >>>>> Jonathan >>>>> >>>>> [image: datastax_logo.png] >>>>> >>>>> Jonathan Lacefield >>>>> >>>>> Solution Architect | (404) 822 3487 | jlacefi...@datastax.com >>>>> >>>>> [image: linkedin.png] <http://www.linkedin.com/in/jlacefield/> [image: >>>>> facebook.png] <https://www.facebook.com/datastax> [image: twitter.png] >>>>> <https://twitter.com/datastax> [image: g+.png] >>>>> <https://plus.google.com/+Datastax/about> >>>>> <http://feeds.feedburner.com/datastax> <https://github.com/datastax/> >>>>> >>>>> On Tue, Dec 16, 2014 at 2:04 PM, Arne Claassen <a...@emotient.com> >>>>> wrote: >>>>>> >>>>>> I have a three node cluster that has been sitting at a load of 4 (for >>>>>> each node), 100% CPI utilization (although 92% nice) for that last 12 >>>>>> hours, ever since some significant writes finished. I'm trying to >>>>>> determine >>>>>> what tuning I should be doing to get it out of this state. The debug log >>>>>> is >>>>>> just an endless series of: >>>>>> >>>>>> DEBUG [ScheduledTasks:1] 2014-12-16 19:03:35,042 GCInspector.java >>>>>> (line 118) GC for ParNew: 166 ms for 10 collections, 4400928736 used; max >>>>>> is 8000634880 >>>>>> DEBUG [ScheduledTasks:1] 2014-12-16 19:03:36,043 GCInspector.java >>>>>> (line 118) GC for ParNew: 165 ms for 10 collections, 4440011176 used; max >>>>>> is 8000634880 >>>>>> DEBUG [ScheduledTasks:1] 2014-12-16 19:03:37,043 GCInspector.java >>>>>> (line 118) GC for ParNew: 135 ms for 8 collections, 4402220568 used; >>>>>> max is 8000634880 >>>>>> >>>>>> iostat shows virtually no I/O. >>>>>> >>>>>> Compaction may enter into this, but i don't really know what to make >>>>>> of compaction stats since they never change: >>>>>> >>>>>> [root@cassandra-37919c3a ~]# nodetool compactionstats >>>>>> pending tasks: 10 >>>>>> compaction type keyspace table >>>>>> completed total unit progress >>>>>> Compaction mediamedia_tracks_raw >>>>>> 271651482 563615497 bytes 48.20% >>>>>> Compaction mediamedia_tracks_raw >>>>>> 30308910 21676695677 bytes 0.14% >>>>>> Compaction mediamedia_tracks_raw >>>>>> 1198384080 1815603161 bytes 66.00% >>>>>> Active compaction remaining time : 0h22m24s >>>>>> >>>>>> 5 minutes later: >>>>>> >>>>>> [root@cassandra-37919c3a ~]# nodetool compactionstats >>>>>> pending tasks: 9 >>>>>> compaction type keyspace table >>>>>> completed total unit progress >>>>>> Compaction mediamedia_tracks_raw >>>>>> 271651482 563615497 bytes 48.20% >>>>>> Compaction mediamedia_tracks_raw >>>>>> 30308910 21676695677 bytes 0.14% >>>>>> Compaction mediamedia_tracks_raw >>>>>> 1198384080 1815603161 bytes 66.00% >>>>>> Active compaction remaining time : 0h22m24s >>>>>> >>>>>> Sure the pending tasks went down by one, but the rest is identical. >>>>>> media_tracks_raw likely has a bunch of tombstones (can't figure out how >>>>>> to >>>>>> get stats on that). >>>>>> >>>>>> Is this behavior something that indicates that i need more Heap, >>>>>> larger new generation? Should I be manually running compaction on tables >>>>>> with lots of tombstones? >>>>>> >>>>>> Any suggestions or places to educate myself better on performance >>>>>> tuning would be appreciated. >>>>>> >>>>>> arne >>>>>> >>>>> >>> >>> -- >>> >>> [image: datastax_logo.png] <http://www.datastax.com/> >>> >>> Ryan Svihla >>> >>> Solution Architect >>> >>> [image: twitter.png] <https://twitter.com/foundev> [image: linkedin.png] >>> <http://www.linkedin.com/pub/ryan-svihla/12/621/727/> >>> >>> DataStax is the fastest, most scalable distributed database technology, >>> delivering Apache Cassandra to the world’s most innovative enterprises. >>> Datastax is built to be agile, always-on, and predictably scalable to any >>> size. With more than 500 customers in 45 countries, DataStax is the >>> database technology and transactional backbone of choice for the worlds >>> most innovative companies such as Netflix, Adobe, Intuit, and eBay. >>> >>>