Re: 100% CPU utilization, ParNew and never completing compactions
Ok, tonight we rolled out on the production cluster. This one has 4 nodes and we dropped and recreated the keyspace before re-processing to avoid all possibility of Everything seemed ok, even if the CPU load was pegged and we saw lots of MUTATION dropped message, but after all the reprocessing was done we noticed data loss, as in QA found some reports missing data. Looking at the app logs it showed 300 rows being written, but in C* there were only 4 rows. We brought up a brand new cluster, going to c3.2xlarge (more than doubling per node CPU) and increased the cluster to 6 nodes and turned processing down to do one media record at a time (still means a lot of rows written with a fanout of 50 async inserts at once). Even with that we're seeing fairly frequent MUTATION dropped messages. Clearly we're doing something fundamentally wrong but other than changing my inserts to batches, I just don't know what else i can do. We're pushing data loads that single server relational databases wouldn't be too concerned about right now. Any suggestions at all would be greatly appreciated. arne On Dec 16, 2014, at 4:48 PM, Ryan Svihla rsvi...@datastax.com wrote: What version of Cassandra? On Dec 16, 2014 6:36 PM, Arne Claassen a...@emotient.com wrote: That's just the thing. There is nothing in the logs except the constant ParNew collections like 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 8000634888 But the load is staying continuously high. There's always some compaction on just that one table, media_tracks_raw going on and those values rarely changed (certainly the remaining time is meaningless) pending tasks: 17 compaction typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 444294932 1310653468 bytes33.90% Compaction mediamedia_tracks_raw 131931354 3411631999 bytes 3.87% Compaction mediamedia_tracks_raw30308970 23097672194 bytes 0.13% Compaction mediamedia_tracks_raw 899216961 1815591081 bytes49.53% Active compaction remaining time : 0h27m56s Here's a sample of a query trace: activity | timestamp| source| source_elapsed --+--+---+ execute_cql3_query | 00:11:46,612 | 10.140.22.236 | 0 Parsing select * from media_tracks_raw where id =74fe9449-8ac4-accb-a723-4bad024101e3 limit 100; | 00:11:46,612 | 10.140.22.236 | 47 Preparing statement | 00:11:46,612 | 10.140.22.236 |234 Sending message to /10.140.21.54 | 00:11:46,619 | 10.140.22.236 | 7190 Message received from /10.140.22.236 | 00:11:46,622 | 10.140.21.54 | 12 Executing single-partition query on media_tracks_raw | 00:11:46,644 | 10.140.21.54 | 21971 Acquiring sstable references | 00:11:46,644 | 10.140.21.54 | 22029 Merging memtable tombstones | 00:11:46,644 | 10.140.21.54 | 22131 Bloom filter allows skipping sstable 1395 | 00:11:46,644 | 10.140.21.54 | 22245 Bloom filter allows skipping sstable 1394 | 00:11:46,644 | 10.140.21.54 | 22279 Bloom filter allows skipping sstable 1391 | 00:11:46,644 | 10.140.21.54 | 22293 Bloom filter allows skipping sstable 1381 | 00:11:46,644 | 10.140.21.54 | 22304 Bloom filter allows skipping sstable 1376 | 00:11:46,644 | 10.140.21.54 | 22317 Bloom filter allows skipping sstable 1368 | 00:11:46,644 | 10.140.21.54 | 22328 Bloom filter allows skipping sstable 1365 | 00:11:46,644 | 10.140.21.54 | 22340
Re: 100% CPU utilization, ParNew and never completing compactions
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 typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 271651482 563615497 bytes48.20% Compaction mediamedia_tracks_raw30308910 21676695677 bytes 0.14% Compaction mediamedia_tracks_raw 1198384080 1815603161 bytes66.00% Active compaction remaining time : 0h22m24s 5 minutes later: [root@cassandra-37919c3a ~]# nodetool compactionstats pending tasks: 9 compaction typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 271651482 563615497 bytes48.20% Compaction mediamedia_tracks_raw30308910 21676695677 bytes 0.14% Compaction mediamedia_tracks_raw 1198384080 1815603161 bytes66.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
Re: 100% CPU utilization, ParNew and never completing compactions
What's heap usage at? On Tue, Dec 16, 2014 at 1: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 typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 271651482 563615497 bytes48.20% Compaction mediamedia_tracks_raw30308910 21676695677 bytes 0.14% Compaction mediamedia_tracks_raw 1198384080 1815603161 bytes66.00% Active compaction remaining time : 0h22m24s 5 minutes later: [root@cassandra-37919c3a ~]# nodetool compactionstats pending tasks: 9 compaction typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 271651482 563615497 bytes48.20% Compaction mediamedia_tracks_raw30308910 21676695677 bytes 0.14% Compaction mediamedia_tracks_raw 1198384080 1815603161 bytes66.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.
Re: 100% CPU utilization, ParNew and never completing compactions
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 typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 271651482 563615497 bytes48.20% Compaction mediamedia_tracks_raw30308910 21676695677 bytes 0.14% Compaction mediamedia_tracks_raw 1198384080 1815603161 bytes66.00% Active compaction remaining time : 0h22m24s 5 minutes later: [root@cassandra-37919c3a ~]# nodetool compactionstats pending tasks: 9 compaction typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 271651482 563615497 bytes48.20% Compaction mediamedia_tracks_raw30308910 21676695677 bytes 0.14% Compaction mediamedia_tracks_raw 1198384080 1815603161 bytes66.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
Re: 100% CPU utilization, ParNew and never completing compactions
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 typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 271651482 563615497 bytes48.20% Compaction mediamedia_tracks_raw 30308910 21676695677 bytes 0.14% Compaction mediamedia_tracks_raw 1198384080 1815603161 bytes66.00% Active compaction remaining time : 0h22m24s 5 minutes later: [root@cassandra-37919c3a ~]# nodetool compactionstats pending tasks: 9 compaction typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 271651482 563615497 bytes48.20% Compaction mediamedia_tracks_raw 30308910 21676695677 bytes 0.14% Compaction mediamedia_tracks_raw 1198384080 1815603161 bytes66.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.
Re: 100% CPU utilization, ParNew and never completing compactions
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 typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 271651482 563615497 bytes48.20% Compaction mediamedia_tracks_raw 30308910 21676695677 bytes 0.14% Compaction mediamedia_tracks_raw 1198384080 1815603161 bytes66.00% Active compaction remaining time : 0h22m24s 5 minutes later: [root@cassandra-37919c3a ~]# nodetool compactionstats pending tasks: 9 compaction typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 271651482 563615497 bytes48.20% Compaction mediamedia_tracks_raw 30308910 21676695677 bytes 0.14% Compaction mediamedia_tracks_raw 1198384080 1815603161 bytes66.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
Re: 100% CPU utilization, ParNew and never completing compactions
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 typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 271651482 563615497 bytes48.20% Compaction mediamedia_tracks_raw 30308910 21676695677 bytes 0.14% Compaction mediamedia_tracks_raw 1198384080 1815603161 bytes66.00% Active compaction remaining time : 0h22m24s 5 minutes later: [root@cassandra-37919c3a ~]# nodetool compactionstats pending tasks: 9 compaction typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 271651482 563615497 bytes48.20% Compaction mediamedia_tracks_raw 30308910 21676695677 bytes 0.14% Compaction mediamedia_tracks_raw 1198384080 1815603161 bytes66.00% Active compaction remaining time : 0h22m24s Sure the pending tasks went down by one, but the rest is identical.
Re: 100% CPU utilization, ParNew and never completing compactions
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 typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 271651482 563615497 bytes48.20% Compaction mediamedia_tracks_raw 30308910 21676695677 bytes 0.14% Compaction
Re: 100% CPU utilization, ParNew and never completing compactions
So heap of that size without some tuning will create a number of problems (high cpu usage one of them), I suggest either 8GB heap and 400mb parnew (which I'd only set that low for that low cpu count) , or attempt the tunings as indicated in https://issues.apache.org/jira/browse/CASSANDRA-8150 On Tue, Dec 16, 2014 at 3:06 PM, Arne Claassen a...@emotient.com wrote: 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 ~]#
Re: 100% CPU utilization, ParNew and never completing compactions
also based on replayed batches..are you using batches to load data? On Tue, Dec 16, 2014 at 3:12 PM, Ryan Svihla rsvi...@datastax.com wrote: So heap of that size without some tuning will create a number of problems (high cpu usage one of them), I suggest either 8GB heap and 400mb parnew (which I'd only set that low for that low cpu count) , or attempt the tunings as indicated in https://issues.apache.org/jira/browse/CASSANDRA-8150 On Tue, Dec 16, 2014 at 3:06 PM, Arne Claassen a...@emotient.com wrote: 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
Re: 100% CPU utilization, ParNew and never completing compactions
The starting configuration I had, which is still running on two of the nodes, was 6GB Heap, 1024MB parnew which is close to what you are suggesting and those have been pegged at load 4 for the over 12 hours with hardly and read or write traffic. I will set one to 8GB/400MB and see if its load changes. On Tue, Dec 16, 2014 at 1:12 PM, Ryan Svihla rsvi...@datastax.com wrote: So heap of that size without some tuning will create a number of problems (high cpu usage one of them), I suggest either 8GB heap and 400mb parnew (which I'd only set that low for that low cpu count) , or attempt the tunings as indicated in https://issues.apache.org/jira/browse/CASSANDRA-8150 On Tue, Dec 16, 2014 at 3:06 PM, Arne Claassen a...@emotient.com wrote: 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
Re: 100% CPU utilization, ParNew and never completing compactions
So 1024 is still a good 2.5 times what I'm suggesting, 6GB is hardly enough to run Cassandra well in, especially if you're going full bore on loads. However, you maybe just flat out be CPU bound on your write throughput, how many TPS and what size writes do you have? Also what is your widest row? Final question what is compaction throughput at? On Tue, Dec 16, 2014 at 3:20 PM, Arne Claassen a...@emotient.com wrote: The starting configuration I had, which is still running on two of the nodes, was 6GB Heap, 1024MB parnew which is close to what you are suggesting and those have been pegged at load 4 for the over 12 hours with hardly and read or write traffic. I will set one to 8GB/400MB and see if its load changes. On Tue, Dec 16, 2014 at 1:12 PM, Ryan Svihla rsvi...@datastax.com wrote: So heap of that size without some tuning will create a number of problems (high cpu usage one of them), I suggest either 8GB heap and 400mb parnew (which I'd only set that low for that low cpu count) , or attempt the tunings as indicated in https://issues.apache.org/jira/browse/CASSANDRA-8150 On Tue, Dec 16, 2014 at 3:06 PM, Arne Claassen a...@emotient.com wrote: 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
Re: 100% CPU utilization, ParNew and never completing compactions
Actually not sure why the machine was originally configured at 6GB since we even started it on an r3.large with 15GB. Re: Batches Not using batches. I actually have that as a separate question on the list. Currently I fan out async single inserts and I'm wondering if batches are better since my data is inherently inserted in blocks of ordered rows for a single partition key. Re: Traffic There isn't all that much traffic. Inserts come in as blocks per partition key, but then can be 5k-200k rows for that partition key. Each of these rows is less than 100k. It's small, lots of ordered rows. It's frame and sub-frame information for media. and rows for one piece of media is inserted at once (the partition key). For the last 12 hours, where the load on all these machine has been stuck there's been virtually no traffic at all. This is the nodes basically sitting idle, except that they had load of 4 each. BTW, how do you determine widest row or for that matter number of tombstones in a row? thanks, arne On Tue, Dec 16, 2014 at 1:24 PM, Ryan Svihla rsvi...@datastax.com wrote: So 1024 is still a good 2.5 times what I'm suggesting, 6GB is hardly enough to run Cassandra well in, especially if you're going full bore on loads. However, you maybe just flat out be CPU bound on your write throughput, how many TPS and what size writes do you have? Also what is your widest row? Final question what is compaction throughput at? On Tue, Dec 16, 2014 at 3:20 PM, Arne Claassen a...@emotient.com wrote: The starting configuration I had, which is still running on two of the nodes, was 6GB Heap, 1024MB parnew which is close to what you are suggesting and those have been pegged at load 4 for the over 12 hours with hardly and read or write traffic. I will set one to 8GB/400MB and see if its load changes. On Tue, Dec 16, 2014 at 1:12 PM, Ryan Svihla rsvi...@datastax.com wrote: So heap of that size without some tuning will create a number of problems (high cpu usage one of them), I suggest either 8GB heap and 400mb parnew (which I'd only set that low for that low cpu count) , or attempt the tunings as indicated in https://issues.apache.org/jira/browse/CASSANDRA-8150 On Tue, Dec 16, 2014 at 3:06 PM, Arne Claassen a...@emotient.com wrote: 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
Re: 100% CPU utilization, ParNew and never completing compactions
Can you define what is virtual no traffic sorry to be repetitive about that, but I've worked on a lot of clusters in the past year and people have wildly different ideas what that means. unlogged batches of the same partition key are definitely a performance optimization. Typically async is much faster and easier on the cluster when you're using multip partition key batches. nodetool cfhistograms keyspace tablename On Tue, Dec 16, 2014 at 3:42 PM, Arne Claassen a...@emotient.com wrote: Actually not sure why the machine was originally configured at 6GB since we even started it on an r3.large with 15GB. Re: Batches Not using batches. I actually have that as a separate question on the list. Currently I fan out async single inserts and I'm wondering if batches are better since my data is inherently inserted in blocks of ordered rows for a single partition key. Re: Traffic There isn't all that much traffic. Inserts come in as blocks per partition key, but then can be 5k-200k rows for that partition key. Each of these rows is less than 100k. It's small, lots of ordered rows. It's frame and sub-frame information for media. and rows for one piece of media is inserted at once (the partition key). For the last 12 hours, where the load on all these machine has been stuck there's been virtually no traffic at all. This is the nodes basically sitting idle, except that they had load of 4 each. BTW, how do you determine widest row or for that matter number of tombstones in a row? thanks, arne On Tue, Dec 16, 2014 at 1:24 PM, Ryan Svihla rsvi...@datastax.com wrote: So 1024 is still a good 2.5 times what I'm suggesting, 6GB is hardly enough to run Cassandra well in, especially if you're going full bore on loads. However, you maybe just flat out be CPU bound on your write throughput, how many TPS and what size writes do you have? Also what is your widest row? Final question what is compaction throughput at? On Tue, Dec 16, 2014 at 3:20 PM, Arne Claassen a...@emotient.com wrote: The starting configuration I had, which is still running on two of the nodes, was 6GB Heap, 1024MB parnew which is close to what you are suggesting and those have been pegged at load 4 for the over 12 hours with hardly and read or write traffic. I will set one to 8GB/400MB and see if its load changes. On Tue, Dec 16, 2014 at 1:12 PM, Ryan Svihla rsvi...@datastax.com wrote: So heap of that size without some tuning will create a number of problems (high cpu usage one of them), I suggest either 8GB heap and 400mb parnew (which I'd only set that low for that low cpu count) , or attempt the tunings as indicated in https://issues.apache.org/jira/browse/CASSANDRA-8150 On Tue, Dec 16, 2014 at 3:06 PM, Arne Claassen a...@emotient.com wrote: 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
Re: 100% CPU utilization, ParNew and never completing compactions
No problem with the follow up questions. I'm on a crash course here trying to understand what makes C* tick so I appreciate all feedback. We reprocessed all media (1200 partition keys) last night where partition keys had somewhere between 4k and 200k rows. After that completed, no traffic went to cluster at all for ~8 hours and throughout today, we may get a couple (less than 10) queries per second and maybe 3-4 write batches per hour. I assume the last value in the Partition Size histogram is the largest row: 20924300 bytes: 79 25109160 bytes: 57 The majority seems clustered around 20 bytes. I will look at switching my inserts to unlogged batches since they are always for one partition key. On Tue, Dec 16, 2014 at 1:47 PM, Ryan Svihla rsvi...@datastax.com wrote: Can you define what is virtual no traffic sorry to be repetitive about that, but I've worked on a lot of clusters in the past year and people have wildly different ideas what that means. unlogged batches of the same partition key are definitely a performance optimization. Typically async is much faster and easier on the cluster when you're using multip partition key batches. nodetool cfhistograms keyspace tablename On Tue, Dec 16, 2014 at 3:42 PM, Arne Claassen a...@emotient.com wrote: Actually not sure why the machine was originally configured at 6GB since we even started it on an r3.large with 15GB. Re: Batches Not using batches. I actually have that as a separate question on the list. Currently I fan out async single inserts and I'm wondering if batches are better since my data is inherently inserted in blocks of ordered rows for a single partition key. Re: Traffic There isn't all that much traffic. Inserts come in as blocks per partition key, but then can be 5k-200k rows for that partition key. Each of these rows is less than 100k. It's small, lots of ordered rows. It's frame and sub-frame information for media. and rows for one piece of media is inserted at once (the partition key). For the last 12 hours, where the load on all these machine has been stuck there's been virtually no traffic at all. This is the nodes basically sitting idle, except that they had load of 4 each. BTW, how do you determine widest row or for that matter number of tombstones in a row? thanks, arne On Tue, Dec 16, 2014 at 1:24 PM, Ryan Svihla rsvi...@datastax.com wrote: So 1024 is still a good 2.5 times what I'm suggesting, 6GB is hardly enough to run Cassandra well in, especially if you're going full bore on loads. However, you maybe just flat out be CPU bound on your write throughput, how many TPS and what size writes do you have? Also what is your widest row? Final question what is compaction throughput at? On Tue, Dec 16, 2014 at 3:20 PM, Arne Claassen a...@emotient.com wrote: The starting configuration I had, which is still running on two of the nodes, was 6GB Heap, 1024MB parnew which is close to what you are suggesting and those have been pegged at load 4 for the over 12 hours with hardly and read or write traffic. I will set one to 8GB/400MB and see if its load changes. On Tue, Dec 16, 2014 at 1:12 PM, Ryan Svihla rsvi...@datastax.com wrote: So heap of that size without some tuning will create a number of problems (high cpu usage one of them), I suggest either 8GB heap and 400mb parnew (which I'd only set that low for that low cpu count) , or attempt the tunings as indicated in https://issues.apache.org/jira/browse/CASSANDRA-8150 On Tue, Dec 16, 2014 at 3:06 PM, Arne Claassen a...@emotient.com wrote: 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
Re: 100% CPU utilization, ParNew and never completing compactions
Ok based on those numbers I have a theory.. can you show me nodetool tptats for all 3 nodes? On Tue, Dec 16, 2014 at 4:04 PM, Arne Claassen a...@emotient.com wrote: No problem with the follow up questions. I'm on a crash course here trying to understand what makes C* tick so I appreciate all feedback. We reprocessed all media (1200 partition keys) last night where partition keys had somewhere between 4k and 200k rows. After that completed, no traffic went to cluster at all for ~8 hours and throughout today, we may get a couple (less than 10) queries per second and maybe 3-4 write batches per hour. I assume the last value in the Partition Size histogram is the largest row: 20924300 bytes: 79 25109160 bytes: 57 The majority seems clustered around 20 bytes. I will look at switching my inserts to unlogged batches since they are always for one partition key. On Tue, Dec 16, 2014 at 1:47 PM, Ryan Svihla rsvi...@datastax.com wrote: Can you define what is virtual no traffic sorry to be repetitive about that, but I've worked on a lot of clusters in the past year and people have wildly different ideas what that means. unlogged batches of the same partition key are definitely a performance optimization. Typically async is much faster and easier on the cluster when you're using multip partition key batches. nodetool cfhistograms keyspace tablename On Tue, Dec 16, 2014 at 3:42 PM, Arne Claassen a...@emotient.com wrote: Actually not sure why the machine was originally configured at 6GB since we even started it on an r3.large with 15GB. Re: Batches Not using batches. I actually have that as a separate question on the list. Currently I fan out async single inserts and I'm wondering if batches are better since my data is inherently inserted in blocks of ordered rows for a single partition key. Re: Traffic There isn't all that much traffic. Inserts come in as blocks per partition key, but then can be 5k-200k rows for that partition key. Each of these rows is less than 100k. It's small, lots of ordered rows. It's frame and sub-frame information for media. and rows for one piece of media is inserted at once (the partition key). For the last 12 hours, where the load on all these machine has been stuck there's been virtually no traffic at all. This is the nodes basically sitting idle, except that they had load of 4 each. BTW, how do you determine widest row or for that matter number of tombstones in a row? thanks, arne On Tue, Dec 16, 2014 at 1:24 PM, Ryan Svihla rsvi...@datastax.com wrote: So 1024 is still a good 2.5 times what I'm suggesting, 6GB is hardly enough to run Cassandra well in, especially if you're going full bore on loads. However, you maybe just flat out be CPU bound on your write throughput, how many TPS and what size writes do you have? Also what is your widest row? Final question what is compaction throughput at? On Tue, Dec 16, 2014 at 3:20 PM, Arne Claassen a...@emotient.com wrote: The starting configuration I had, which is still running on two of the nodes, was 6GB Heap, 1024MB parnew which is close to what you are suggesting and those have been pegged at load 4 for the over 12 hours with hardly and read or write traffic. I will set one to 8GB/400MB and see if its load changes. On Tue, Dec 16, 2014 at 1:12 PM, Ryan Svihla rsvi...@datastax.com wrote: So heap of that size without some tuning will create a number of problems (high cpu usage one of them), I suggest either 8GB heap and 400mb parnew (which I'd only set that low for that low cpu count) , or attempt the tunings as indicated in https://issues.apache.org/jira/browse/CASSANDRA-8150 On Tue, Dec 16, 2014 at 3:06 PM, Arne Claassen a...@emotient.com wrote: 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%.
Re: 100% CPU utilization, ParNew and never completing compactions
Of course QA decided to start a test batch (still relatively low traffic), so I hope it doesn't throw the tpstats off too much Node 1: Pool NameActive Pending Completed Blocked All time blocked MutationStage 0 0 13804928 0 0 ReadStage 0 0 10975 0 0 RequestResponseStage 0 07725378 0 0 ReadRepairStage 0 0 1247 0 0 ReplicateOnWriteStage 0 0 0 0 0 MiscStage 0 0 0 0 0 HintedHandoff 1 1 50 0 0 FlushWriter 0 0306 0 31 MemoryMeter 0 0719 0 0 GossipStage 0 0 286505 0 0 CacheCleanupExecutor 0 0 0 0 0 InternalResponseStage 0 0 0 0 0 CompactionExecutor414159 0 0 ValidationExecutor0 0 0 0 0 MigrationStage0 0 0 0 0 commitlog_archiver0 0 0 0 0 AntiEntropyStage 0 0 0 0 0 PendingRangeCalculator0 0 11 0 0 MemtablePostFlusher 0 0 1781 0 0 Message type Dropped READ 0 RANGE_SLICE 0 _TRACE 0 MUTATION391041 COUNTER_MUTATION 0 BINARY 0 REQUEST_RESPONSE 0 PAGED_RANGE 0 READ_REPAIR 0 Node 2: Pool NameActive Pending Completed Blocked All time blocked MutationStage 0 0 997042 0 0 ReadStage 0 0 2623 0 0 RequestResponseStage 0 0 706650 0 0 ReadRepairStage 0 0275 0 0 ReplicateOnWriteStage 0 0 0 0 0 MiscStage 0 0 0 0 0 HintedHandoff 2 2 12 0 0 FlushWriter 0 0 37 0 4 MemoryMeter 0 0 70 0 0 GossipStage 0 0 14927 0 0 CacheCleanupExecutor 0 0 0 0 0 InternalResponseStage 0 0 0 0 0 CompactionExecutor4 7 94 0 0 ValidationExecutor0 0 0 0 0 MigrationStage0 0 0 0 0 commitlog_archiver0 0 0 0 0 AntiEntropyStage 0 0 0 0 0 PendingRangeCalculator0 0 3 0 0 MemtablePostFlusher 0 0114 0 0 Message type Dropped READ 0 RANGE_SLICE 0 _TRACE 0 MUTATION 0 COUNTER_MUTATION 0 BINARY 0 REQUEST_RESPONSE 0 PAGED_RANGE 0 READ_REPAIR 0 Node 3: Pool NameActive Pending Completed Blocked All time blocked MutationStage 0 01539324 0 0 ReadStage 0 0 2571 0 0 RequestResponseStage 0 0 373300 0 0 ReadRepairStage 0 0325 0 0 ReplicateOnWriteStage 0 0 0 0 0 MiscStage 0 0 0 0 0 HintedHandoff 1 1 21 0 0 FlushWriter 0 0 38 0 5 MemoryMeter 0 0
Re: 100% CPU utilization, ParNew and never completing compactions
so you've got some blocked flush writers but you have a incredibly large number of dropped mutations, are you using secondary indexes? and if so how many? what is your flush queue set to? On Tue, Dec 16, 2014 at 4:43 PM, Arne Claassen a...@emotient.com wrote: Of course QA decided to start a test batch (still relatively low traffic), so I hope it doesn't throw the tpstats off too much Node 1: Pool NameActive Pending Completed Blocked All time blocked MutationStage 0 0 13804928 0 0 ReadStage 0 0 10975 0 0 RequestResponseStage 0 07725378 0 0 ReadRepairStage 0 0 1247 0 0 ReplicateOnWriteStage 0 0 0 0 0 MiscStage 0 0 0 0 0 HintedHandoff 1 1 50 0 0 FlushWriter 0 0306 0 31 MemoryMeter 0 0719 0 0 GossipStage 0 0 286505 0 0 CacheCleanupExecutor 0 0 0 0 0 InternalResponseStage 0 0 0 0 0 CompactionExecutor414159 0 0 ValidationExecutor0 0 0 0 0 MigrationStage0 0 0 0 0 commitlog_archiver0 0 0 0 0 AntiEntropyStage 0 0 0 0 0 PendingRangeCalculator0 0 11 0 0 MemtablePostFlusher 0 0 1781 0 0 Message type Dropped READ 0 RANGE_SLICE 0 _TRACE 0 MUTATION391041 COUNTER_MUTATION 0 BINARY 0 REQUEST_RESPONSE 0 PAGED_RANGE 0 READ_REPAIR 0 Node 2: Pool NameActive Pending Completed Blocked All time blocked MutationStage 0 0 997042 0 0 ReadStage 0 0 2623 0 0 RequestResponseStage 0 0 706650 0 0 ReadRepairStage 0 0275 0 0 ReplicateOnWriteStage 0 0 0 0 0 MiscStage 0 0 0 0 0 HintedHandoff 2 2 12 0 0 FlushWriter 0 0 37 0 4 MemoryMeter 0 0 70 0 0 GossipStage 0 0 14927 0 0 CacheCleanupExecutor 0 0 0 0 0 InternalResponseStage 0 0 0 0 0 CompactionExecutor4 7 94 0 0 ValidationExecutor0 0 0 0 0 MigrationStage0 0 0 0 0 commitlog_archiver0 0 0 0 0 AntiEntropyStage 0 0 0 0 0 PendingRangeCalculator0 0 3 0 0 MemtablePostFlusher 0 0114 0 0 Message type Dropped READ 0 RANGE_SLICE 0 _TRACE 0 MUTATION 0 COUNTER_MUTATION 0 BINARY 0 REQUEST_RESPONSE 0 PAGED_RANGE 0 READ_REPAIR 0 Node 3: Pool NameActive Pending Completed Blocked All time blocked MutationStage 0 01539324 0 0 ReadStage 0 0 2571 0 0 RequestResponseStage 0 0 373300 0 0 ReadRepairStage 0 0325 0 0
Re: 100% CPU utilization, ParNew and never completing compactions
Not using any secondary indicies and memtable_flush_queue_size is the default 4. But let me tell you how data is mutated right now, maybe that will give you an insight on how this is happening Basically the frame data table has the following primary key: PRIMARY KEY ((id), trackid, timestamp) Generally data is inserted once. So day to day writes are all new rows. However, when out process for generating analytics for these rows changes, we run the media back through again, causing overwrites. Up until last night, this was just a new insert because the PK never changed so it was always 1-to-1 overwrite of every row. Last night was the first time that a new change went in where the PK could actually change so now the process is always, DELETE by partition key, insert all rows for partition key, repeat. We two tables that have similar frame data projections and some other aggregates with much smaller row count per partition key. hope that helps, arne On Dec 16, 2014, at 2:46 PM, Ryan Svihla rsvi...@datastax.com wrote: so you've got some blocked flush writers but you have a incredibly large number of dropped mutations, are you using secondary indexes? and if so how many? what is your flush queue set to? On Tue, Dec 16, 2014 at 4:43 PM, Arne Claassen a...@emotient.com wrote: Of course QA decided to start a test batch (still relatively low traffic), so I hope it doesn't throw the tpstats off too much Node 1: Pool NameActive Pending Completed Blocked All time blocked MutationStage 0 0 13804928 0 0 ReadStage 0 0 10975 0 0 RequestResponseStage 0 07725378 0 0 ReadRepairStage 0 0 1247 0 0 ReplicateOnWriteStage 0 0 0 0 0 MiscStage 0 0 0 0 0 HintedHandoff 1 1 50 0 0 FlushWriter 0 0306 0 31 MemoryMeter 0 0719 0 0 GossipStage 0 0 286505 0 0 CacheCleanupExecutor 0 0 0 0 0 InternalResponseStage 0 0 0 0 0 CompactionExecutor414159 0 0 ValidationExecutor0 0 0 0 0 MigrationStage0 0 0 0 0 commitlog_archiver0 0 0 0 0 AntiEntropyStage 0 0 0 0 0 PendingRangeCalculator0 0 11 0 0 MemtablePostFlusher 0 0 1781 0 0 Message type Dropped READ 0 RANGE_SLICE 0 _TRACE 0 MUTATION391041 COUNTER_MUTATION 0 BINARY 0 REQUEST_RESPONSE 0 PAGED_RANGE 0 READ_REPAIR 0 Node 2: Pool NameActive Pending Completed Blocked All time blocked MutationStage 0 0 997042 0 0 ReadStage 0 0 2623 0 0 RequestResponseStage 0 0 706650 0 0 ReadRepairStage 0 0275 0 0 ReplicateOnWriteStage 0 0 0 0 0 MiscStage 0 0 0 0 0 HintedHandoff 2 2 12 0 0 FlushWriter 0 0 37 0 4 MemoryMeter 0 0 70 0 0 GossipStage 0 0 14927 0 0 CacheCleanupExecutor 0 0 0 0 0 InternalResponseStage 0 0 0 0 0 CompactionExecutor4 7 94 0 0 ValidationExecutor0 0 0 0
Re: 100% CPU utilization, ParNew and never completing compactions
so a delete is really another write for gc_grace_seconds (default 10 days), if you get enough tombstones it can make managing your cluster a challenge as is. open up cqlsh, turn on tracing and try a few queries..how many tombstones are scanned for a given query? It's possible the heap problems you're seeing are actually happening on the query side and not on the ingest side, the severity of this depends on driver and cassandra version, but older drivers and versions of cassandra could easily overload heap with expensive selects, when layered over tombstones it's certainly becomes a possibility this is your root cause. Now this will primarily create more load on compaction and depending on your cassandra version there maybe some other issue at work, but something I can tell you is every time I see 1 dropped mutation I see a cluster that was overloaded enough it had to shed load. If I see 200k I see a cluster/configuration/hardware that is badly overloaded. I suggest the following - trace some of the queries used in prod - monitor your ingest rate, see at what levels you run into issues (GCInspector log messages, dropped mutations, etc) - heap configuration we mentioned earlier..go ahead and monitor heap usage, if it hits 75% repeated this is an indication of heavy load - monitor dropped mutations..any dropped mutation is evidence of an overloaded server, again the root cause can be many other problems that are solvable with current hardware, and LOTS of people runs with nodes with similar configuration. On Tue, Dec 16, 2014 at 5:08 PM, Arne Claassen a...@emotient.com wrote: Not using any secondary indicies and memtable_flush_queue_size is the default 4. But let me tell you how data is mutated right now, maybe that will give you an insight on how this is happening Basically the frame data table has the following primary key: PRIMARY KEY ((id), trackid, timestamp) Generally data is inserted once. So day to day writes are all new rows. However, when out process for generating analytics for these rows changes, we run the media back through again, causing overwrites. Up until last night, this was just a new insert because the PK never changed so it was always 1-to-1 overwrite of every row. Last night was the first time that a new change went in where the PK could actually change so now the process is always, DELETE by partition key, insert all rows for partition key, repeat. We two tables that have similar frame data projections and some other aggregates with much smaller row count per partition key. hope that helps, arne On Dec 16, 2014, at 2:46 PM, Ryan Svihla rsvi...@datastax.com wrote: so you've got some blocked flush writers but you have a incredibly large number of dropped mutations, are you using secondary indexes? and if so how many? what is your flush queue set to? On Tue, Dec 16, 2014 at 4:43 PM, Arne Claassen a...@emotient.com wrote: Of course QA decided to start a test batch (still relatively low traffic), so I hope it doesn't throw the tpstats off too much Node 1: Pool NameActive Pending Completed Blocked All time blocked MutationStage 0 0 13804928 0 0 ReadStage 0 0 10975 0 0 RequestResponseStage 0 07725378 0 0 ReadRepairStage 0 0 1247 0 0 ReplicateOnWriteStage 0 0 0 0 0 MiscStage 0 0 0 0 0 HintedHandoff 1 1 50 0 0 FlushWriter 0 0306 0 31 MemoryMeter 0 0719 0 0 GossipStage 0 0 286505 0 0 CacheCleanupExecutor 0 0 0 0 0 InternalResponseStage 0 0 0 0 0 CompactionExecutor414159 0 0 ValidationExecutor0 0 0 0 0 MigrationStage0 0 0 0 0 commitlog_archiver0 0 0 0 0 AntiEntropyStage 0 0 0 0 0 PendingRangeCalculator0 0 11 0 0 MemtablePostFlusher 0 0 1781 0 0 Message type Dropped READ 0 RANGE_SLICE 0 _TRACE
Re: 100% CPU utilization, ParNew and never completing compactions
I just did a wide set of selects and ran across no tombstones. But while on the subject of gc_grace_seconds, any reason, on a small cluster not to set it to something low like a single day. It seems like 10 days is only need to large clusters undergoing long partition splits, or am i misunderstanding gc_grace_seconds. Now, given all that, does any of this explain a high load when the cluster is idle? Is it compaction catching up and would manual forced compaction alleviate that? thanks, arne On Dec 16, 2014, at 3:28 PM, Ryan Svihla rsvi...@datastax.com wrote: so a delete is really another write for gc_grace_seconds (default 10 days), if you get enough tombstones it can make managing your cluster a challenge as is. open up cqlsh, turn on tracing and try a few queries..how many tombstones are scanned for a given query? It's possible the heap problems you're seeing are actually happening on the query side and not on the ingest side, the severity of this depends on driver and cassandra version, but older drivers and versions of cassandra could easily overload heap with expensive selects, when layered over tombstones it's certainly becomes a possibility this is your root cause. Now this will primarily create more load on compaction and depending on your cassandra version there maybe some other issue at work, but something I can tell you is every time I see 1 dropped mutation I see a cluster that was overloaded enough it had to shed load. If I see 200k I see a cluster/configuration/hardware that is badly overloaded. I suggest the following trace some of the queries used in prod monitor your ingest rate, see at what levels you run into issues (GCInspector log messages, dropped mutations, etc) heap configuration we mentioned earlier..go ahead and monitor heap usage, if it hits 75% repeated this is an indication of heavy load monitor dropped mutations..any dropped mutation is evidence of an overloaded server, again the root cause can be many other problems that are solvable with current hardware, and LOTS of people runs with nodes with similar configuration. On Tue, Dec 16, 2014 at 5:08 PM, Arne Claassen a...@emotient.com wrote: Not using any secondary indicies and memtable_flush_queue_size is the default 4. But let me tell you how data is mutated right now, maybe that will give you an insight on how this is happening Basically the frame data table has the following primary key: PRIMARY KEY ((id), trackid, timestamp) Generally data is inserted once. So day to day writes are all new rows. However, when out process for generating analytics for these rows changes, we run the media back through again, causing overwrites. Up until last night, this was just a new insert because the PK never changed so it was always 1-to-1 overwrite of every row. Last night was the first time that a new change went in where the PK could actually change so now the process is always, DELETE by partition key, insert all rows for partition key, repeat. We two tables that have similar frame data projections and some other aggregates with much smaller row count per partition key. hope that helps, arne On Dec 16, 2014, at 2:46 PM, Ryan Svihla rsvi...@datastax.com wrote: so you've got some blocked flush writers but you have a incredibly large number of dropped mutations, are you using secondary indexes? and if so how many? what is your flush queue set to? On Tue, Dec 16, 2014 at 4:43 PM, Arne Claassen a...@emotient.com wrote: Of course QA decided to start a test batch (still relatively low traffic), so I hope it doesn't throw the tpstats off too much Node 1: Pool NameActive Pending Completed Blocked All time blocked MutationStage 0 0 13804928 0 0 ReadStage 0 0 10975 0 0 RequestResponseStage 0 07725378 0 0 ReadRepairStage 0 0 1247 0 0 ReplicateOnWriteStage 0 0 0 0 0 MiscStage 0 0 0 0 0 HintedHandoff 1 1 50 0 0 FlushWriter 0 0306 0 31 MemoryMeter 0 0719 0 0 GossipStage 0 0 286505 0 0 CacheCleanupExecutor 0 0 0 0 0 InternalResponseStage 0 0 0 0 0 CompactionExecutor414159 0 0
Re: 100% CPU utilization, ParNew and never completing compactions
manual forced compactions create more problems than they solve, if you have no evidence of tombstones in your selects (which seems odd, can you share some of the tracing output?), then I'm not sure what it would solve for you. Compaction running could explain a high load, logs messages with ERRORS, WARN, GCInspector are all meaningful there, I suggest search jira for your version to see if there are any interesting bugs. On Tue, Dec 16, 2014 at 6:14 PM, Arne Claassen a...@emotient.com wrote: I just did a wide set of selects and ran across no tombstones. But while on the subject of gc_grace_seconds, any reason, on a small cluster not to set it to something low like a single day. It seems like 10 days is only need to large clusters undergoing long partition splits, or am i misunderstanding gc_grace_seconds. Now, given all that, does any of this explain a high load when the cluster is idle? Is it compaction catching up and would manual forced compaction alleviate that? thanks, arne On Dec 16, 2014, at 3:28 PM, Ryan Svihla rsvi...@datastax.com wrote: so a delete is really another write for gc_grace_seconds (default 10 days), if you get enough tombstones it can make managing your cluster a challenge as is. open up cqlsh, turn on tracing and try a few queries..how many tombstones are scanned for a given query? It's possible the heap problems you're seeing are actually happening on the query side and not on the ingest side, the severity of this depends on driver and cassandra version, but older drivers and versions of cassandra could easily overload heap with expensive selects, when layered over tombstones it's certainly becomes a possibility this is your root cause. Now this will primarily create more load on compaction and depending on your cassandra version there maybe some other issue at work, but something I can tell you is every time I see 1 dropped mutation I see a cluster that was overloaded enough it had to shed load. If I see 200k I see a cluster/configuration/hardware that is badly overloaded. I suggest the following - trace some of the queries used in prod - monitor your ingest rate, see at what levels you run into issues (GCInspector log messages, dropped mutations, etc) - heap configuration we mentioned earlier..go ahead and monitor heap usage, if it hits 75% repeated this is an indication of heavy load - monitor dropped mutations..any dropped mutation is evidence of an overloaded server, again the root cause can be many other problems that are solvable with current hardware, and LOTS of people runs with nodes with similar configuration. On Tue, Dec 16, 2014 at 5:08 PM, Arne Claassen a...@emotient.com wrote: Not using any secondary indicies and memtable_flush_queue_size is the default 4. But let me tell you how data is mutated right now, maybe that will give you an insight on how this is happening Basically the frame data table has the following primary key: PRIMARY KEY ((id), trackid, timestamp) Generally data is inserted once. So day to day writes are all new rows. However, when out process for generating analytics for these rows changes, we run the media back through again, causing overwrites. Up until last night, this was just a new insert because the PK never changed so it was always 1-to-1 overwrite of every row. Last night was the first time that a new change went in where the PK could actually change so now the process is always, DELETE by partition key, insert all rows for partition key, repeat. We two tables that have similar frame data projections and some other aggregates with much smaller row count per partition key. hope that helps, arne On Dec 16, 2014, at 2:46 PM, Ryan Svihla rsvi...@datastax.com wrote: so you've got some blocked flush writers but you have a incredibly large number of dropped mutations, are you using secondary indexes? and if so how many? what is your flush queue set to? On Tue, Dec 16, 2014 at 4:43 PM, Arne Claassen a...@emotient.com wrote: Of course QA decided to start a test batch (still relatively low traffic), so I hope it doesn't throw the tpstats off too much Node 1: Pool NameActive Pending Completed Blocked All time blocked MutationStage 0 0 13804928 0 0 ReadStage 0 0 10975 0 0 RequestResponseStage 0 07725378 0 0 ReadRepairStage 0 0 1247 0 0 ReplicateOnWriteStage 0 0 0 0 0 MiscStage 0 0 0 0 0 HintedHandoff 1 1 50 0 0 FlushWriter 0 0306 0
Re: 100% CPU utilization, ParNew and never completing compactions
That's just the thing. There is nothing in the logs except the constant ParNew collections like 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 8000634888 But the load is staying continuously high. There's always some compaction on just that one table, media_tracks_raw going on and those values rarely changed (certainly the remaining time is meaningless) pending tasks: 17 compaction typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 444294932 1310653468 bytes33.90% Compaction mediamedia_tracks_raw 131931354 3411631999 bytes 3.87% Compaction mediamedia_tracks_raw30308970 23097672194 bytes 0.13% Compaction mediamedia_tracks_raw 899216961 1815591081 bytes49.53% Active compaction remaining time : 0h27m56s Here's a sample of a query trace: activity | timestamp| source| source_elapsed --+--+---+ execute_cql3_query | 00:11:46,612 | 10.140.22.236 | 0 Parsing select * from media_tracks_raw where id =74fe9449-8ac4-accb-a723-4bad024101e3 limit 100; | 00:11:46,612 | 10.140.22.236 | 47 Preparing statement | 00:11:46,612 | 10.140.22.236 |234 Sending message to /10.140.21.54 | 00:11:46,619 | 10.140.22.236 | 7190 Message received from /10.140.22.236 | 00:11:46,622 | 10.140.21.54 | 12 Executing single-partition query on media_tracks_raw | 00:11:46,644 | 10.140.21.54 | 21971 Acquiring sstable references | 00:11:46,644 | 10.140.21.54 | 22029 Merging memtable tombstones | 00:11:46,644 | 10.140.21.54 | 22131 Bloom filter allows skipping sstable 1395 | 00:11:46,644 | 10.140.21.54 | 22245 Bloom filter allows skipping sstable 1394 | 00:11:46,644 | 10.140.21.54 | 22279 Bloom filter allows skipping sstable 1391 | 00:11:46,644 | 10.140.21.54 | 22293 Bloom filter allows skipping sstable 1381 | 00:11:46,644 | 10.140.21.54 | 22304 Bloom filter allows skipping sstable 1376 | 00:11:46,644 | 10.140.21.54 | 22317 Bloom filter allows skipping sstable 1368 | 00:11:46,644 | 10.140.21.54 | 22328 Bloom filter allows skipping sstable 1365 | 00:11:46,644 | 10.140.21.54 | 22340 Bloom filter allows skipping sstable 1351 | 00:11:46,644 | 10.140.21.54 | 22352 Bloom filter allows skipping sstable 1367 | 00:11:46,644 | 10.140.21.54 | 22363 Bloom filter allows skipping sstable 1380 | 00:11:46,644 | 10.140.21.54 | 22374 Bloom filter allows skipping sstable 1343 | 00:11:46,644 | 10.140.21.54 | 22386 Bloom filter allows skipping sstable 1342 | 00:11:46,644 | 10.140.21.54 | 22397 Bloom filter allows skipping sstable 1334 | 00:11:46,644 | 10.140.21.54 | 22408 Bloom filter allows skipping sstable 1377 | 00:11:46,644 | 10.140.21.54 | 22429 Bloom filter allows skipping sstable 1330 | 00:11:46,644 | 10.140.21.54 | 22441 Bloom filter allows skipping sstable 1329 | 00:11:46,644 | 10.140.21.54 | 22452 Bloom
Re: 100% CPU utilization, ParNew and never completing compactions
What version of Cassandra? On Dec 16, 2014 6:36 PM, Arne Claassen a...@emotient.com wrote: That's just the thing. There is nothing in the logs except the constant ParNew collections like 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 8000634888 But the load is staying continuously high. There's always some compaction on just that one table, media_tracks_raw going on and those values rarely changed (certainly the remaining time is meaningless) pending tasks: 17 compaction typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 444294932 1310653468 bytes33.90% Compaction mediamedia_tracks_raw 131931354 3411631999 bytes 3.87% Compaction mediamedia_tracks_raw30308970 23097672194 bytes 0.13% Compaction mediamedia_tracks_raw 899216961 1815591081 bytes49.53% Active compaction remaining time : 0h27m56s Here's a sample of a query trace: activity | timestamp| source| source_elapsed --+--+---+ execute_cql3_query | 00:11:46,612 | 10.140.22.236 | 0 Parsing select * from media_tracks_raw where id =74fe9449-8ac4-accb-a723-4bad024101e3 limit 100; | 00:11:46,612 | 10.140.22.236 | 47 Preparing statement | 00:11:46,612 | 10.140.22.236 |234 Sending message to /10.140.21.54 | 00:11:46,619 | 10.140.22.236 | 7190 Message received from /10.140.22.236 | 00:11:46,622 | 10.140.21.54 | 12 Executing single-partition query on media_tracks_raw | 00:11:46,644 | 10.140.21.54 | 21971 Acquiring sstable references | 00:11:46,644 | 10.140.21.54 | 22029 Merging memtable tombstones | 00:11:46,644 | 10.140.21.54 | 22131 Bloom filter allows skipping sstable 1395 | 00:11:46,644 | 10.140.21.54 | 22245 Bloom filter allows skipping sstable 1394 | 00:11:46,644 | 10.140.21.54 | 22279 Bloom filter allows skipping sstable 1391 | 00:11:46,644 | 10.140.21.54 | 22293 Bloom filter allows skipping sstable 1381 | 00:11:46,644 | 10.140.21.54 | 22304 Bloom filter allows skipping sstable 1376 | 00:11:46,644 | 10.140.21.54 | 22317 Bloom filter allows skipping sstable 1368 | 00:11:46,644 | 10.140.21.54 | 22328 Bloom filter allows skipping sstable 1365 | 00:11:46,644 | 10.140.21.54 | 22340 Bloom filter allows skipping sstable 1351 | 00:11:46,644 | 10.140.21.54 | 22352 Bloom filter allows skipping sstable 1367 | 00:11:46,644 | 10.140.21.54 | 22363 Bloom filter allows skipping sstable 1380 | 00:11:46,644 | 10.140.21.54 | 22374 Bloom filter allows skipping sstable 1343 | 00:11:46,644 | 10.140.21.54 | 22386 Bloom filter allows skipping sstable 1342 | 00:11:46,644 | 10.140.21.54 | 22397 Bloom filter allows skipping sstable 1334 | 00:11:46,644 | 10.140.21.54 | 22408 Bloom filter allows skipping sstable 1377 | 00:11:46,644 | 10.140.21.54 | 22429 Bloom filter allows skipping sstable 1330 | 00:11:46,644 | 10.140.21.54 | 22441 Bloom filter allows skipping sstable 1329 | 00:11:46,644 | 10.140.21.54 | 22452 Bloom filter allows skipping sstable 1328 | 00:11:46,644 | 10.140.21.54 | 22463 Bloom filter allows skipping sstable 1327 | 00:11:46,644 | 10.140.21.54 | 22475
Re: 100% CPU utilization, ParNew and never completing compactions
Cassandra 2.0.10 and Datastax Java Driver 2.1.1 On Dec 16, 2014, at 4:48 PM, Ryan Svihla rsvi...@datastax.com wrote: What version of Cassandra? On Dec 16, 2014 6:36 PM, Arne Claassen a...@emotient.com wrote: That's just the thing. There is nothing in the logs except the constant ParNew collections like 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 8000634888 But the load is staying continuously high. There's always some compaction on just that one table, media_tracks_raw going on and those values rarely changed (certainly the remaining time is meaningless) pending tasks: 17 compaction typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 444294932 1310653468 bytes33.90% Compaction mediamedia_tracks_raw 131931354 3411631999 bytes 3.87% Compaction mediamedia_tracks_raw30308970 23097672194 bytes 0.13% Compaction mediamedia_tracks_raw 899216961 1815591081 bytes49.53% Active compaction remaining time : 0h27m56s Here's a sample of a query trace: activity | timestamp| source| source_elapsed --+--+---+ execute_cql3_query | 00:11:46,612 | 10.140.22.236 | 0 Parsing select * from media_tracks_raw where id =74fe9449-8ac4-accb-a723-4bad024101e3 limit 100; | 00:11:46,612 | 10.140.22.236 | 47 Preparing statement | 00:11:46,612 | 10.140.22.236 |234 Sending message to /10.140.21.54 | 00:11:46,619 | 10.140.22.236 | 7190 Message received from /10.140.22.236 | 00:11:46,622 | 10.140.21.54 | 12 Executing single-partition query on media_tracks_raw | 00:11:46,644 | 10.140.21.54 | 21971 Acquiring sstable references | 00:11:46,644 | 10.140.21.54 | 22029 Merging memtable tombstones | 00:11:46,644 | 10.140.21.54 | 22131 Bloom filter allows skipping sstable 1395 | 00:11:46,644 | 10.140.21.54 | 22245 Bloom filter allows skipping sstable 1394 | 00:11:46,644 | 10.140.21.54 | 22279 Bloom filter allows skipping sstable 1391 | 00:11:46,644 | 10.140.21.54 | 22293 Bloom filter allows skipping sstable 1381 | 00:11:46,644 | 10.140.21.54 | 22304 Bloom filter allows skipping sstable 1376 | 00:11:46,644 | 10.140.21.54 | 22317 Bloom filter allows skipping sstable 1368 | 00:11:46,644 | 10.140.21.54 | 22328 Bloom filter allows skipping sstable 1365 | 00:11:46,644 | 10.140.21.54 | 22340 Bloom filter allows skipping sstable 1351 | 00:11:46,644 | 10.140.21.54 | 22352 Bloom filter allows skipping sstable 1367 | 00:11:46,644 | 10.140.21.54 | 22363 Bloom filter allows skipping sstable 1380 | 00:11:46,644 | 10.140.21.54 | 22374 Bloom filter allows skipping sstable 1343 | 00:11:46,644 | 10.140.21.54 | 22386 Bloom filter allows skipping sstable 1342 | 00:11:46,644 | 10.140.21.54 | 22397 Bloom filter allows skipping sstable 1334 | 00:11:46,644 | 10.140.21.54 | 22408 Bloom filter allows skipping sstable 1377 | 00:11:46,644 | 10.140.21.54 | 22429 Bloom
Re: 100% CPU utilization, ParNew and never completing compactions
Maybe checking which thread(s) would hint what's going on? (see http://www.boxjar.com/using-top-and-jstack-to-find-the-java-thread-that-is-hogging-the-cpu/). On Wed, Dec 17, 2014 at 1:51 AM, Arne Claassen a...@emotient.com wrote: Cassandra 2.0.10 and Datastax Java Driver 2.1.1 On Dec 16, 2014, at 4:48 PM, Ryan Svihla rsvi...@datastax.com wrote: What version of Cassandra? On Dec 16, 2014 6:36 PM, Arne Claassen a...@emotient.com wrote: That's just the thing. There is nothing in the logs except the constant ParNew collections like 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 8000634888 But the load is staying continuously high. There's always some compaction on just that one table, media_tracks_raw going on and those values rarely changed (certainly the remaining time is meaningless) pending tasks: 17 compaction typekeyspace table completed total unit progress Compaction mediamedia_tracks_raw 444294932 1310653468 bytes33.90% Compaction mediamedia_tracks_raw 131931354 3411631999 bytes 3.87% Compaction mediamedia_tracks_raw30308970 23097672194 bytes 0.13% Compaction mediamedia_tracks_raw 899216961 1815591081 bytes49.53% Active compaction remaining time : 0h27m56s Here's a sample of a query trace: activity | timestamp| source| source_elapsed --+--+---+ execute_cql3_query | 00:11:46,612 | 10.140.22.236 | 0 Parsing select * from media_tracks_raw where id =74fe9449-8ac4-accb-a723-4bad024101e3 limit 100; | 00:11:46,612 | 10.140.22.236 | 47 Preparing statement | 00:11:46,612 | 10.140.22.236 |234 Sending message to /10.140.21.54 | 00:11:46,619 | 10.140.22.236 | 7190 Message received from /10.140.22.236 | 00:11:46,622 | 10.140.21.54 | 12 Executing single-partition query on media_tracks_raw | 00:11:46,644 | 10.140.21.54 | 21971 Acquiring sstable references | 00:11:46,644 | 10.140.21.54 | 22029 Merging memtable tombstones | 00:11:46,644 | 10.140.21.54 | 22131 Bloom filter allows skipping sstable 1395 | 00:11:46,644 | 10.140.21.54 | 22245 Bloom filter allows skipping sstable 1394 | 00:11:46,644 | 10.140.21.54 | 22279 Bloom filter allows skipping sstable 1391 | 00:11:46,644 | 10.140.21.54 | 22293 Bloom filter allows skipping sstable 1381 | 00:11:46,644 | 10.140.21.54 | 22304 Bloom filter allows skipping sstable 1376 | 00:11:46,644 | 10.140.21.54 | 22317 Bloom filter allows skipping sstable 1368 | 00:11:46,644 | 10.140.21.54 | 22328 Bloom filter allows skipping sstable 1365 | 00:11:46,644 | 10.140.21.54 | 22340 Bloom filter allows skipping sstable 1351 | 00:11:46,644 | 10.140.21.54 | 22352 Bloom filter allows skipping sstable 1367 | 00:11:46,644 | 10.140.21.54 | 22363 Bloom filter allows skipping sstable 1380 | 00:11:46,644 | 10.140.21.54 | 22374 Bloom filter allows skipping sstable 1343 | 00:11:46,644 | 10.140.21.54 | 22386 Bloom filter allows skipping sstable 1342 | 00:11:46,644 | 10.140.21.54 | 22397 Bloom filter allows skipping sstable 1334 | 00:11:46,644 |