You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Jeff Ferland <jb...@tubularlabs.com> on 2015/10/30 02:12:45 UTC

Cassandra stalls and dropped messages not due to GC

Using DSE 4.8.1 / 2.1.11.872, Java version 1.8.0_66

We upgraded our cluster this weekend and have been having issues with dropped mutations since then. Intensely investigating a single node and toying with settings has revealed that GC stalls don’t make up enough time to explain the 10 seconds of apparent stall that would cause a hangup.

tpstats output typically shows active threads in the low single digits and pending similar or 0. Before a failure, pending MutationStage will skyrocket into 5+ digits. System.log regularly shows the gossiper complaining, then slow log complaints, then logs dropped mutations.

For the entire minute of 00:55, the gc logging shows no single pause > .14 seconds and most of them much smaller. Abbreviated GC log after switching to g1gc (problem also exhibited before G1GC):

2015-10-30T00:55:00.550+0000: 6752.857: [GC pause (G1 Evacuation Pause) (young)
2015-10-30T00:55:02.843+0000: 6755.150: [GC pause (GCLocker Initiated GC) (young)
2015-10-30T00:55:05.241+0000: 6757.548: [GC pause (G1 Evacuation Pause) (young)
2015-10-30T00:55:07.755+0000: 6760.062: [GC pause (G1 Evacuation Pause) (young)
2015-10-30T00:55:10.532+0000: 6762.839: [GC pause (G1 Evacuation Pause) (young)
2015-10-30T00:55:13.080+0000: 6765.387: [GC pause (G1 Evacuation Pause) (young)
2015-10-30T00:55:15.914+0000: 6768.221: [GC pause (G1 Evacuation Pause) (young)
2015-10-30T00:55:18.619+0000: 6770.926: [GC pause (GCLocker Initiated GC) (young)
2015-10-30T00:55:23.270+0000: 6775.578: [GC pause (GCLocker Initiated GC) (young)
2015-10-30T00:55:28.662+0000: 6780.969: [GC pause (GCLocker Initiated GC) (young)
2015-10-30T00:55:33.326+0000: 6785.633: [GC pause (G1 Evacuation Pause) (young)
2015-10-30T00:55:36.600+0000: 6788.907: [GC pause (G1 Evacuation Pause) (young)
2015-10-30T00:55:40.050+0000: 6792.357: [GC pause (G1 Evacuation Pause) (young)
2015-10-30T00:55:43.728+0000: 6796.035: [GC pause (G1 Evacuation Pause) (young)
2015-10-30T00:55:48.216+0000: 6800.523: [GC pause (G1 Evacuation Pause) (young)
2015-10-30T00:55:53.621+0000: 6805.928: [GC pause (G1 Evacuation Pause) (young)
2015-10-30T00:55:59.048+0000: 6811.355: [GC pause (GCLocker Initiated GC) (young)

System log snippet of the pattern I’m seeing:

WARN  [GossipTasks:1] 2015-10-30 00:55:25,129  Gossiper.java:747 - Gossip stage has 1 pending tasks; skipping status check (no nodes will be marked down)
INFO  [CompactionExecutor:210] 2015-10-30 00:55:26,006  CompactionTask.java:141 - Compacting [SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8283-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8286-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8284-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8285-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8287-Data.db')]
WARN  [GossipTasks:1] 2015-10-30 00:55:26,230  Gossiper.java:747 - Gossip stage has 3 pending tasks; skipping status check (no nodes will be marked down)
WARN  [GossipTasks:1] 2015-10-30 00:55:27,330  Gossiper.java:747 - Gossip stage has 5 pending tasks; skipping status check (no nodes will be marked down)
WARN  [GossipTasks:1] 2015-10-30 00:55:28,431  Gossiper.java:747 - Gossip stage has 7 pending tasks; skipping status check (no nodes will be marked down)
WARN  [GossipTasks:1] 2015-10-30 00:55:29,531  Gossiper.java:747 - Gossip stage has 10 pending tasks; skipping status check (no nodes will be marked down)
INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,448  CqlSlowLogWriter.java:151 - Recording statements with duration of 16042 in slow log
INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,451  CqlSlowLogWriter.java:151 - Recording statements with duration of 16047 in slow log
INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,453  CqlSlowLogWriter.java:151 - Recording statements with duration of 16018 in slow log
INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,454  CqlSlowLogWriter.java:151 - Recording statements with duration of 16042 in slow log
INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,455  CqlSlowLogWriter.java:151 - Recording statements with duration of 16024 in slow log
INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,457  CqlSlowLogWriter.java:151 - Recording statements with duration of 16045 in slow log
INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,458  CqlSlowLogWriter.java:151 - Recording statements with duration of 16041 in slow log
INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,459  CqlSlowLogWriter.java:151 - Recording statements with duration of 16040 in slow log
INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,461  CqlSlowLogWriter.java:151 - Recording statements with duration of 16037 in slow log
WARN  [GossipTasks:1] 2015-10-30 00:55:32,697  FailureDetector.java:257 - Not marking nodes down due to local pause of 8668629909 > 5000000000
INFO  [CompactionExecutor:212] 2015-10-30 00:55:32,864  ColumnFamilyStore.java:900 - Enqueuing flush of compactions_in_progress: 164 (0%) on-heap, 0 (0%) off-heap
INFO  [MemtableFlushWriter:846] 2015-10-30 00:55:32,865  Memtable.java:347 - Writing Memtable-compactions_in_progress@969357824(0.008KiB serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
INFO  [MemtableFlushWriter:846] 2015-10-30 00:55:32,865  Memtable.java:382 - Completed flushing /mnt/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-tmp-ka-1461769-Data.db (0.000KiB) for commitlog position ReplayPosition(segmentId=1446159752729, position=8699402)
INFO  [ScheduledTasks:1] 2015-10-30 00:55:35,346  MessagingService.java:930 - MUTATION messages were dropped in last 5000 ms: 67432 for internal timeout and 0 for cross node timeout
INFO  [ScheduledTasks:1] 2015-10-30 00:55:35,350  MessagingService.java:930 - READ_REPAIR messages were dropped in last 5000 ms: 209 for internal timeout and 0 for cross node timeout

I’m starting to suspect I’ll have to take regular stack dumps and catch a bad moment, but I figured I’d reach out here before going down that road to see if somebody else had ever experienced a similar kind of pain.

-Jeff


Re: Cassandra stalls and dropped messages not due to GC

Posted by Jeff Ferland <jb...@tubularlabs.com>.
> On Nov 2, 2015, at 11:35 AM, Nate McCall <na...@thelastpickle.com> wrote:
> Forgive me, but what is CMS?
> 
> Sorry - ConcurrentMarkSweep garbage collector. 

Ah, my brain was trying to think in terms of something Cassandra specific. I have full GC logging on and since moving to G1, I haven’t had any >500ms GC cycles and the >200ms logger triggers about once every 2 minutes. I don’t intend to roll off that given positive confirmation that the cycles seem to be working well and GC logs don’t line up with outages. Also, the issue proved to be the same on CMS as on G1.

> 
> No. I’ve tried some mitigations since tuning thread pool sizes and GC, but the problem begins with only an upgrade of Cassandra. No other system packages, kernels, etc.
> 
> 
> 
> From what 2.0 version did you upgrade? If it was < 2.0.7, you would need to run 'nodetool upgradesstables'  but I'm not sure the issue would manifest that way. Otherwise, double check the DSE release notes and upgrade guide. I've not had any issues like this going from 2.0.x to 2.1.x on vanilla C*. 

2.0.14 or higher. I don’t recall what version of DSE I standardized on last, but probably 2.0.16. In any case, the format moved from jb to ka.

I checked into the related source code and from there gripped my logs where I’m seeing messages like (most extreme example):

WARN  [PERIODIC-COMMIT-LOG-SYNCER] 2015-11-02 23:10:16,478  AbstractCommitLogService.java:105 - Out of 38 commit log syncs over the past 307s with average duration of 2087.32ms, 3 have exceeded the configured commit interval by an average of 15515.67ms

I seem to have 3-4 32MB commit logs created per minute. In a quick experiment, I’ve run nodetool flush just now and reduced a 5.7G directory to 58M. I’m going to flush all the nodes and see if that’s somehow related where it’s just holding commit logs too long. (Did I miss the configuration for maximum memtable age?)

-Jeff

Re: Cassandra stalls and dropped messages not due to GC

Posted by Nate McCall <na...@thelastpickle.com>.
>
>
> Forgive me, but what is CMS?
>

Sorry - ConcurrentMarkSweep garbage collector.


>
> No. I’ve tried some mitigations since tuning thread pool sizes and GC, but
> the problem begins with only an upgrade of Cassandra. No other system
> packages, kernels, etc.
>
>
>
>From what 2.0 version did you upgrade? If it was < 2.0.7, you would need to
run 'nodetool upgradesstables'  but I'm not sure the issue would manifest
that way. Otherwise, double check the DSE release notes and upgrade guide.
I've not had any issues like this going from 2.0.x to 2.1.x on vanilla C*.



-- 
-----------------
Nate McCall
Austin, TX
@zznate

Co-Founder & Sr. Technical Consultant
Apache Cassandra Consulting
http://www.thelastpickle.com

Re: Cassandra stalls and dropped messages not due to GC

Posted by Jeff Ferland <jb...@tubularlabs.com>.
Having caught a node in an undesirable state, many of my threads are reading like this:
"SharedPool-Worker-5" #875 daemon prio=5 os_prio=0 tid=0x00007f3e14196800 nid=0x96ce waiting on condition [0x00007f3ddb835000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
        at org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:283)
        at org.apache.cassandra.db.commitlog.PeriodicCommitLogService.maybeWaitForSync(PeriodicCommitLogService.java:44)
        at org.apache.cassandra.db.commitlog.AbstractCommitLogService.finishWriteFor(AbstractCommitLogService.java:152)
        at org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:252)
        at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:379)
        at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:359)
        at org.apache.cassandra.db.Mutation.apply(Mutation.java:214)
        at org.apache.cassandra.db.MutationVerbHandler.doVerb(MutationVerbHandler.java:54)
        at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:64)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164)
        at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105)
        at java.lang.Thread.run(Thread.java:745)

But commit log loading seems evenly spaced and low enough in volume:
/mnt/cassandra/commitlog$ ls -lht | head
total 7.2G
-rw-r--r-- 1 cassandra cassandra 32M Nov  2 18:50 CommitLog-4-1446162051324.log
-rw-r--r-- 1 cassandra cassandra 32M Nov  2 18:50 CommitLog-4-1446162051323.log
-rw-r--r-- 1 cassandra cassandra 32M Nov  2 18:50 CommitLog-4-1446162051322.log
-rw-r--r-- 1 cassandra cassandra 32M Nov  2 18:49 CommitLog-4-1446162051321.log
-rw-r--r-- 1 cassandra cassandra 32M Nov  2 18:49 CommitLog-4-1446162051320.log
-rw-r--r-- 1 cassandra cassandra 32M Nov  2 18:48 CommitLog-4-1446162051319.log
-rw-r--r-- 1 cassandra cassandra 32M Nov  2 18:48 CommitLog-4-1446162051318.log
-rw-r--r-- 1 cassandra cassandra 32M Nov  2 18:47 CommitLog-4-1446162051317.log
-rw-r--r-- 1 cassandra cassandra 32M Nov  2 18:46 CommitLog-4-1446162051316.log

Commit logs are on 10 second periodic setting:
commitlog_sync: periodic
commitlog_sync_period_in_ms: 10000

SSDs are fully trimmed out and mounted with discard since it snuck into my head that could be an issue. Still stuck diagnosing this.

> On Oct 30, 2015, at 3:37 PM, Nate McCall <na...@thelastpickle.com> wrote:
> 
> Does tpstats show unusually high counts for blocked flush writers? 

The “All Time Blocked” metric is 0 across my entire cluster.

> As Sebastian suggests, running ttop will paint a clearer picture about what is happening within C*. I would however recommend going back to CMS in this case as that is the devil we all know and more folks will be able to offer advice on seeing its output (and it removes a delta). 

Forgive me, but what is CMS?

> 
> It’s starting to look to me like it’s possibly related to brief IO spikes that are smaller than my usual graphing granularity. It feels surprising to me that these would affect the Gossip threads, but it’s the best current lead I have with my debugging right now. More to come when I learn it.
> 
> Probably not the case since this was a result of an upgrade, but I've seen similar behavior on systems where some kernels had issues with irqbalance doing the right thing and would end up parking most interrupts on CPU0 (like say for the disk and ethernet modules) regardless of the number of cores. Check out proc via 'cat /proc/interrupts' and make sure the interrupts are spread out of CPU cores. You can steer them off manually at runtime if they are not spread out. 

Interrupt loading is even.

> Also, did you upgrade anything besides Cassandra?

No. I’ve tried some mitigations since tuning thread pool sizes and GC, but the problem begins with only an upgrade of Cassandra. No other system packages, kernels, etc.

-Jeff



Re: Cassandra stalls and dropped messages not due to GC

Posted by Nate McCall <na...@thelastpickle.com>.
Does tpstats show unusually high counts for blocked flush writers?

As Sebastian suggests, running ttop will paint a clearer picture about what
is happening within C*. I would however recommend going back to CMS in this
case as that is the devil we all know and more folks will be able to offer
advice on seeing its output (and it removes a delta).


> It’s starting to look to me like it’s possibly related to brief IO spikes
> that are smaller than my usual graphing granularity. It feels surprising to
> me that these would affect the Gossip threads, but it’s the best current
> lead I have with my debugging right now. More to come when I learn it.
>

Probably not the case since this was a result of an upgrade, but I've seen
similar behavior on systems where some kernels had issues with irqbalance
doing the right thing and would end up parking most interrupts on CPU0
(like say for the disk and ethernet modules) regardless of the number of
cores. Check out proc via 'cat /proc/interrupts' and make sure the
interrupts are spread out of CPU cores. You can steer them off manually at
runtime if they are not spread out.

Also, did you upgrade anything besides Cassandra?


-- 
-----------------
Nate McCall
Austin, TX
@zznate

Co-Founder & Sr. Technical Consultant
Apache Cassandra Consulting
http://www.thelastpickle.com

Re: Cassandra stalls and dropped messages not due to GC

Posted by Jeff Ferland <jb...@tubularlabs.com>.
Upgraded from 2.0.x. Using the other commit log sync method and 10 seconds. Enabling batch mode is like swallowing a grenade.

It’s starting to look to me like it’s possibly related to brief IO spikes that are smaller than my usual graphing granularity. It feels surprising to me that these would affect the Gossip threads, but it’s the best current lead I have with my debugging right now. More to come when I learn it.

-Jeff

> On Oct 29, 2015, at 6:36 PM, Graham Sanderson <gr...@vast.com> wrote:
> 
> you didn’t say what you upgraded from, but if it is 2.0.x, then look at CASSANDRA-9504
> 
> If so and you use
> commitlog_sync: batch
> Then you probably want to set
> commitlog_sync_batch_window_in_ms: 1 (or 2)
> Note I’m only slightly convinced this is the cause because of your READ_REPAIR issues (though if you are dropping a lot of MUTATIONS under load and your machines are overloaded, you’d be doing more READ_REPAIR than usual probably)
> 
>> On Oct 29, 2015, at 8:12 PM, Jeff Ferland <jbf@tubularlabs.com <ma...@tubularlabs.com>> wrote:
>> 
>> Using DSE 4.8.1 / 2.1.11.872, Java version 1.8.0_66
>> 
>> We upgraded our cluster this weekend and have been having issues with dropped mutations since then. Intensely investigating a single node and toying with settings has revealed that GC stalls don’t make up enough time to explain the 10 seconds of apparent stall that would cause a hangup.
>> 
>> tpstats output typically shows active threads in the low single digits and pending similar or 0. Before a failure, pending MutationStage will skyrocket into 5+ digits. System.log regularly shows the gossiper complaining, then slow log complaints, then logs dropped mutations.
>> 
>> For the entire minute of 00:55, the gc logging shows no single pause > .14 seconds and most of them much smaller. Abbreviated GC log after switching to g1gc (problem also exhibited before G1GC):
>> 
>> 2015-10-30T00:55:00.550+0000: 6752.857: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:02.843+0000: 6755.150: [GC pause (GCLocker Initiated GC) (young)
>> 2015-10-30T00:55:05.241+0000: 6757.548: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:07.755+0000: 6760.062: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:10.532+0000: 6762.839: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:13.080+0000: 6765.387: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:15.914+0000: 6768.221: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:18.619+0000: 6770.926: [GC pause (GCLocker Initiated GC) (young)
>> 2015-10-30T00:55:23.270+0000: 6775.578: [GC pause (GCLocker Initiated GC) (young)
>> 2015-10-30T00:55:28.662+0000: 6780.969: [GC pause (GCLocker Initiated GC) (young)
>> 2015-10-30T00:55:33.326+0000: 6785.633: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:36.600+0000: 6788.907: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:40.050+0000: 6792.357: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:43.728+0000: 6796.035: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:48.216+0000: 6800.523: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:53.621+0000: 6805.928: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:59.048+0000: 6811.355: [GC pause (GCLocker Initiated GC) (young)
>> 
>> System log snippet of the pattern I’m seeing:
>> 
>> WARN  [GossipTasks:1] 2015-10-30 00:55:25,129  Gossiper.java:747 - Gossip stage has 1 pending tasks; skipping status check (no nodes will be marked down)
>> INFO  [CompactionExecutor:210] 2015-10-30 00:55:26,006  CompactionTask.java:141 - Compacting [SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8283-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8286-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8284-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8285-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8287-Data.db')]
>> WARN  [GossipTasks:1] 2015-10-30 00:55:26,230  Gossiper.java:747 - Gossip stage has 3 pending tasks; skipping status check (no nodes will be marked down)
>> WARN  [GossipTasks:1] 2015-10-30 00:55:27,330  Gossiper.java:747 - Gossip stage has 5 pending tasks; skipping status check (no nodes will be marked down)
>> WARN  [GossipTasks:1] 2015-10-30 00:55:28,431  Gossiper.java:747 - Gossip stage has 7 pending tasks; skipping status check (no nodes will be marked down)
>> WARN  [GossipTasks:1] 2015-10-30 00:55:29,531  Gossiper.java:747 - Gossip stage has 10 pending tasks; skipping status check (no nodes will be marked down)
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,448  CqlSlowLogWriter.java:151 - Recording statements with duration of 16042 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,451  CqlSlowLogWriter.java:151 - Recording statements with duration of 16047 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,453  CqlSlowLogWriter.java:151 - Recording statements with duration of 16018 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,454  CqlSlowLogWriter.java:151 - Recording statements with duration of 16042 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,455  CqlSlowLogWriter.java:151 - Recording statements with duration of 16024 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,457  CqlSlowLogWriter.java:151 - Recording statements with duration of 16045 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,458  CqlSlowLogWriter.java:151 - Recording statements with duration of 16041 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,459  CqlSlowLogWriter.java:151 - Recording statements with duration of 16040 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,461  CqlSlowLogWriter.java:151 - Recording statements with duration of 16037 in slow log
>> WARN  [GossipTasks:1] 2015-10-30 00:55:32,697  FailureDetector.java:257 - Not marking nodes down due to local pause of 8668629909 > 5000000000
>> INFO  [CompactionExecutor:212] 2015-10-30 00:55:32,864  ColumnFamilyStore.java:900 - Enqueuing flush of compactions_in_progress: 164 (0%) on-heap, 0 (0%) off-heap
>> INFO  [MemtableFlushWriter:846] 2015-10-30 00:55:32,865  Memtable.java:347 - Writing Memtable-compactions_in_progress@969357824(0.008KiB serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
>> INFO  [MemtableFlushWriter:846] 2015-10-30 00:55:32,865  Memtable.java:382 - Completed flushing /mnt/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-tmp-ka-1461769-Data.db (0.000KiB) for commitlog position ReplayPosition(segmentId=1446159752729, position=8699402)
>> INFO  [ScheduledTasks:1] 2015-10-30 00:55:35,346  MessagingService.java:930 - MUTATION messages were dropped in last 5000 ms: 67432 for internal timeout and 0 for cross node timeout
>> INFO  [ScheduledTasks:1] 2015-10-30 00:55:35,350  MessagingService.java:930 - READ_REPAIR messages were dropped in last 5000 ms: 209 for internal timeout and 0 for cross node timeout
>> 
>> I’m starting to suspect I’ll have to take regular stack dumps and catch a bad moment, but I figured I’d reach out here before going down that road to see if somebody else had ever experienced a similar kind of pain.
>> 
>> -Jeff
>> 
> 


Re: Cassandra stalls and dropped messages not due to GC

Posted by Graham Sanderson <gr...@vast.com>.
Only if you actually change cassandra.yaml (that was the change in 2.1.6 which is why it matters what version he upgraded from)

> On Oct 29, 2015, at 10:06 PM, Sebastian Estevez <se...@datastax.com> wrote:
> 
> The thing about the CASSANDRA-9504 theory is that it was solved in 2.1.6 and Jeff's running 2.1.11.
> 
> @Jeff
> 
> How often does this happen? Can you watch ttop as soon as you notice increased read/write latencies?
> 
> wget https://bintray.com/artifact/download/aragozin/generic/sjk-plus-0.3.6.jar
>  <https://bintray.com/artifact/download/aragozin/generic/sjk-plus-0.3.6.jar>java -jar sjk-plus-0.3.6.jar ttop -s localhost:7199 -n 30 -o CPU
> 
> This should at least tell you which Cassandra threads are causing high memory allocations  and CPU consumption.
> 
> All the best,
> 
>  <http://www.datastax.com/>
> Sebastián Estévez
> Solutions Architect | 954 905 8615 | sebastian.estevez@datastax.com <ma...@datastax.com>
>  <https://www.linkedin.com/company/datastax>  <https://www.facebook.com/datastax>  <https://twitter.com/datastax>  <https://plus.google.com/+Datastax/about>  <http://feeds.feedburner.com/datastax>
>  <http://goog_410786983/>
> 
>  <http://www.datastax.com/gartner-magic-quadrant-odbms>
> 
> 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. 
> 
> On Thu, Oct 29, 2015 at 9:36 PM, Graham Sanderson <graham@vast.com <ma...@vast.com>> wrote:
> you didn’t say what you upgraded from, but if it is 2.0.x, then look at CASSANDRA-9504
> 
> If so and you use
> commitlog_sync: batch
> Then you probably want to set
> commitlog_sync_batch_window_in_ms: 1 (or 2)
> Note I’m only slightly convinced this is the cause because of your READ_REPAIR issues (though if you are dropping a lot of MUTATIONS under load and your machines are overloaded, you’d be doing more READ_REPAIR than usual probably)
> 
>> On Oct 29, 2015, at 8:12 PM, Jeff Ferland <jbf@tubularlabs.com <ma...@tubularlabs.com>> wrote:
>> 
>> Using DSE 4.8.1 / 2.1.11.872, Java version 1.8.0_66
>> 
>> We upgraded our cluster this weekend and have been having issues with dropped mutations since then. Intensely investigating a single node and toying with settings has revealed that GC stalls don’t make up enough time to explain the 10 seconds of apparent stall that would cause a hangup.
>> 
>> tpstats output typically shows active threads in the low single digits and pending similar or 0. Before a failure, pending MutationStage will skyrocket into 5+ digits. System.log regularly shows the gossiper complaining, then slow log complaints, then logs dropped mutations.
>> 
>> For the entire minute of 00:55, the gc logging shows no single pause > .14 seconds and most of them much smaller. Abbreviated GC log after switching to g1gc (problem also exhibited before G1GC):
>> 
>> 2015-10-30T00:55:00.550+0000: 6752.857: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:02.843+0000: 6755.150: [GC pause (GCLocker Initiated GC) (young)
>> 2015-10-30T00:55:05.241+0000: 6757.548: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:07.755+0000: 6760.062: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:10.532+0000: 6762.839: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:13.080+0000: 6765.387: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:15.914+0000: 6768.221: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:18.619+0000: 6770.926: [GC pause (GCLocker Initiated GC) (young)
>> 2015-10-30T00:55:23.270+0000: 6775.578: [GC pause (GCLocker Initiated GC) (young)
>> 2015-10-30T00:55:28.662+0000: 6780.969: [GC pause (GCLocker Initiated GC) (young)
>> 2015-10-30T00:55:33.326+0000: 6785.633: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:36.600+0000: 6788.907: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:40.050+0000: 6792.357: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:43.728+0000: 6796.035: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:48.216+0000: 6800.523: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:53.621+0000: 6805.928: [GC pause (G1 Evacuation Pause) (young)
>> 2015-10-30T00:55:59.048+0000: 6811.355: [GC pause (GCLocker Initiated GC) (young)
>> 
>> System log snippet of the pattern I’m seeing:
>> 
>> WARN  [GossipTasks:1] 2015-10-30 00:55:25,129  Gossiper.java:747 - Gossip stage has 1 pending tasks; skipping status check (no nodes will be marked down)
>> INFO  [CompactionExecutor:210] 2015-10-30 00:55:26,006  CompactionTask.java:141 - Compacting [SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8283-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8286-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8284-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8285-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8287-Data.db')]
>> WARN  [GossipTasks:1] 2015-10-30 00:55:26,230  Gossiper.java:747 - Gossip stage has 3 pending tasks; skipping status check (no nodes will be marked down)
>> WARN  [GossipTasks:1] 2015-10-30 00:55:27,330  Gossiper.java:747 - Gossip stage has 5 pending tasks; skipping status check (no nodes will be marked down)
>> WARN  [GossipTasks:1] 2015-10-30 00:55:28,431  Gossiper.java:747 - Gossip stage has 7 pending tasks; skipping status check (no nodes will be marked down)
>> WARN  [GossipTasks:1] 2015-10-30 00:55:29,531  Gossiper.java:747 - Gossip stage has 10 pending tasks; skipping status check (no nodes will be marked down)
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,448  CqlSlowLogWriter.java:151 - Recording statements with duration of 16042 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,451  CqlSlowLogWriter.java:151 - Recording statements with duration of 16047 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,453  CqlSlowLogWriter.java:151 - Recording statements with duration of 16018 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,454  CqlSlowLogWriter.java:151 - Recording statements with duration of 16042 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,455  CqlSlowLogWriter.java:151 - Recording statements with duration of 16024 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,457  CqlSlowLogWriter.java:151 - Recording statements with duration of 16045 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,458  CqlSlowLogWriter.java:151 - Recording statements with duration of 16041 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,459  CqlSlowLogWriter.java:151 - Recording statements with duration of 16040 in slow log
>> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,461  CqlSlowLogWriter.java:151 - Recording statements with duration of 16037 in slow log
>> WARN  [GossipTasks:1] 2015-10-30 00:55:32,697  FailureDetector.java:257 - Not marking nodes down due to local pause of 8668629909 <tel:8668629909> > 5000000000
>> INFO  [CompactionExecutor:212] 2015-10-30 00:55:32,864  ColumnFamilyStore.java:900 - Enqueuing flush of compactions_in_progress: 164 (0%) on-heap, 0 (0%) off-heap
>> INFO  [MemtableFlushWriter:846] 2015-10-30 00:55:32,865  Memtable.java:347 - Writing Memtable-compactions_in_progress@969357824(0.008KiB serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
>> INFO  [MemtableFlushWriter:846] 2015-10-30 00:55:32,865  Memtable.java:382 - Completed flushing /mnt/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-tmp-ka-1461769-Data.db (0.000KiB) for commitlog position ReplayPosition(segmentId=1446159752729, position=8699402)
>> INFO  [ScheduledTasks:1] 2015-10-30 00:55:35,346  MessagingService.java:930 - MUTATION messages were dropped in last 5000 ms: 67432 for internal timeout and 0 for cross node timeout
>> INFO  [ScheduledTasks:1] 2015-10-30 00:55:35,350  MessagingService.java:930 - READ_REPAIR messages were dropped in last 5000 ms: 209 for internal timeout and 0 for cross node timeout
>> 
>> I’m starting to suspect I’ll have to take regular stack dumps and catch a bad moment, but I figured I’d reach out here before going down that road to see if somebody else had ever experienced a similar kind of pain.
>> 
>> -Jeff
>> 
> 
> 


Re: Cassandra stalls and dropped messages not due to GC

Posted by Sebastian Estevez <se...@datastax.com>.
The thing about the CASSANDRA-9504 theory is that it was solved in 2.1.6
and Jeff's running 2.1.11.

@Jeff

How often does this happen? Can you watch ttop as soon as you notice
increased read/write latencies?

wget
> https://bintray.com/artifact/download/aragozin/generic/sjk-plus-0.3.6.jar
> java -jar sjk-plus-0.3.6.jar ttop -s localhost:7199 -n 30 -o CPU


This should at least tell you which Cassandra threads are causing high
memory allocations  and CPU consumption.

All the best,


[image: datastax_logo.png] <http://www.datastax.com/>

Sebastián Estévez

Solutions Architect | 954 905 8615 | sebastian.estevez@datastax.com

[image: linkedin.png] <https://www.linkedin.com/company/datastax> [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>
<http://goog_410786983>


<http://www.datastax.com/gartner-magic-quadrant-odbms>

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.

On Thu, Oct 29, 2015 at 9:36 PM, Graham Sanderson <gr...@vast.com> wrote:

> you didn’t say what you upgraded from, but if it is 2.0.x, then look at
> CASSANDRA-9504
>
> If so and you use
>
> commitlog_sync: batch
>
> Then you probably want to set
>
> commitlog_sync_batch_window_in_ms: 1 (or 2)
>
> Note I’m only slightly convinced this is the cause because of your
> READ_REPAIR issues (though if you are dropping a lot of MUTATIONS under
> load and your machines are overloaded, you’d be doing more READ_REPAIR than
> usual probably)
>
> On Oct 29, 2015, at 8:12 PM, Jeff Ferland <jb...@tubularlabs.com> wrote:
>
> Using DSE 4.8.1 / 2.1.11.872, Java version 1.8.0_66
>
> We upgraded our cluster this weekend and have been having issues with
> dropped mutations since then. Intensely investigating a single node and
> toying with settings has revealed that GC stalls don’t make up enough time
> to explain the 10 seconds of apparent stall that would cause a hangup.
>
> tpstats output typically shows active threads in the low single digits and
> pending similar or 0. Before a failure, pending MutationStage will
> skyrocket into 5+ digits. System.log regularly shows the gossiper
> complaining, then slow log complaints, then logs dropped mutations.
>
> For the entire minute of 00:55, the gc logging shows no single pause > .14
> seconds and most of them much smaller. Abbreviated GC log after switching
> to g1gc (problem also exhibited before G1GC):
>
> 2015-10-30T00:55:00.550+0000: 6752.857: [GC pause (G1 Evacuation Pause)
> (young)
> 2015-10-30T00:55:02.843+0000: 6755.150: [GC pause (GCLocker Initiated GC)
> (young)
> 2015-10-30T00:55:05.241+0000: 6757.548: [GC pause (G1 Evacuation Pause)
> (young)
> 2015-10-30T00:55:07.755+0000: 6760.062: [GC pause (G1 Evacuation Pause)
> (young)
> 2015-10-30T00:55:10.532+0000: 6762.839: [GC pause (G1 Evacuation Pause)
> (young)
> 2015-10-30T00:55:13.080+0000: 6765.387: [GC pause (G1 Evacuation Pause)
> (young)
> 2015-10-30T00:55:15.914+0000: 6768.221: [GC pause (G1 Evacuation Pause)
> (young)
> 2015-10-30T00:55:18.619+0000: 6770.926: [GC pause (GCLocker Initiated GC)
> (young)
> 2015-10-30T00:55:23.270+0000: 6775.578: [GC pause (GCLocker Initiated GC)
> (young)
> 2015-10-30T00:55:28.662+0000: 6780.969: [GC pause (GCLocker Initiated GC)
> (young)
> 2015-10-30T00:55:33.326+0000: 6785.633: [GC pause (G1 Evacuation Pause)
> (young)
> 2015-10-30T00:55:36.600+0000: 6788.907: [GC pause (G1 Evacuation Pause)
> (young)
> 2015-10-30T00:55:40.050+0000: 6792.357: [GC pause (G1 Evacuation Pause)
> (young)
> 2015-10-30T00:55:43.728+0000: 6796.035: [GC pause (G1 Evacuation Pause)
> (young)
> 2015-10-30T00:55:48.216+0000: 6800.523: [GC pause (G1 Evacuation Pause)
> (young)
> 2015-10-30T00:55:53.621+0000: 6805.928: [GC pause (G1 Evacuation Pause)
> (young)
> 2015-10-30T00:55:59.048+0000: 6811.355: [GC pause (GCLocker Initiated GC)
> (young)
>
> System log snippet of the pattern I’m seeing:
>
> WARN  [GossipTasks:1] 2015-10-30 00:55:25,129  Gossiper.java:747 - Gossip
> stage has 1 pending tasks; skipping status check (no nodes will be marked
> down)
> INFO  [CompactionExecutor:210] 2015-10-30 00:55:26,006
>  CompactionTask.java:141 - Compacting
> [SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8283-Data.db'),
> SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8286-Data.db'),
> SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8284-Data.db'),
> SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8285-Data.db'),
> SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8287-Data.db')]
> WARN  [GossipTasks:1] 2015-10-30 00:55:26,230  Gossiper.java:747 - Gossip
> stage has 3 pending tasks; skipping status check (no nodes will be marked
> down)
> WARN  [GossipTasks:1] 2015-10-30 00:55:27,330  Gossiper.java:747 - Gossip
> stage has 5 pending tasks; skipping status check (no nodes will be marked
> down)
> WARN  [GossipTasks:1] 2015-10-30 00:55:28,431  Gossiper.java:747 - Gossip
> stage has 7 pending tasks; skipping status check (no nodes will be marked
> down)
> WARN  [GossipTasks:1] 2015-10-30 00:55:29,531  Gossiper.java:747 - Gossip
> stage has 10 pending tasks; skipping status check (no nodes will be marked
> down)
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,448
>  CqlSlowLogWriter.java:151 - Recording statements with duration of 16042 in
> slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,451
>  CqlSlowLogWriter.java:151 - Recording statements with duration of 16047 in
> slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,453
>  CqlSlowLogWriter.java:151 - Recording statements with duration of 16018 in
> slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,454
>  CqlSlowLogWriter.java:151 - Recording statements with duration of 16042 in
> slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,455
>  CqlSlowLogWriter.java:151 - Recording statements with duration of 16024 in
> slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,457
>  CqlSlowLogWriter.java:151 - Recording statements with duration of 16045 in
> slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,458
>  CqlSlowLogWriter.java:151 - Recording statements with duration of 16041 in
> slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,459
>  CqlSlowLogWriter.java:151 - Recording statements with duration of 16040 in
> slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,461
>  CqlSlowLogWriter.java:151 - Recording statements with duration of 16037 in
> slow log
> WARN  [GossipTasks:1] 2015-10-30 00:55:32,697  FailureDetector.java:257 -
> Not marking nodes down due to local pause of 8668629909 > 5000000000
> INFO  [CompactionExecutor:212] 2015-10-30 00:55:32,864
>  ColumnFamilyStore.java:900 - Enqueuing flush of compactions_in_progress:
> 164 (0%) on-heap, 0 (0%) off-heap
> INFO  [MemtableFlushWriter:846] 2015-10-30 00:55:32,865  Memtable.java:347
> - Writing Memtable-compactions_in_progress@969357824(0.008KiB serialized
> bytes, 1 ops, 0%/0% of on/off-heap limit)
> INFO  [MemtableFlushWriter:846] 2015-10-30 00:55:32,865  Memtable.java:382
> - Completed flushing
> /mnt/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-tmp-ka-1461769-Data.db
> (0.000KiB) for commitlog position ReplayPosition(segmentId=1446159752729,
> position=8699402)
> INFO  [ScheduledTasks:1] 2015-10-30 00:55:35,346
>  MessagingService.java:930 - MUTATION messages were dropped in last 5000
> ms: 67432 for internal timeout and 0 for cross node timeout
> INFO  [ScheduledTasks:1] 2015-10-30 00:55:35,350
>  MessagingService.java:930 - READ_REPAIR messages were dropped in last 5000
> ms: 209 for internal timeout and 0 for cross node timeout
>
> I’m starting to suspect I’ll have to take regular stack dumps and catch a
> bad moment, but I figured I’d reach out here before going down that road to
> see if somebody else had ever experienced a similar kind of pain.
>
> -Jeff
>
>
>

Re: Cassandra stalls and dropped messages not due to GC

Posted by Graham Sanderson <gr...@vast.com>.
you didn’t say what you upgraded from, but if it is 2.0.x, then look at CASSANDRA-9504

If so and you use
commitlog_sync: batch
Then you probably want to set
commitlog_sync_batch_window_in_ms: 1 (or 2)
Note I’m only slightly convinced this is the cause because of your READ_REPAIR issues (though if you are dropping a lot of MUTATIONS under load and your machines are overloaded, you’d be doing more READ_REPAIR than usual probably)

> On Oct 29, 2015, at 8:12 PM, Jeff Ferland <jb...@tubularlabs.com> wrote:
> 
> Using DSE 4.8.1 / 2.1.11.872, Java version 1.8.0_66
> 
> We upgraded our cluster this weekend and have been having issues with dropped mutations since then. Intensely investigating a single node and toying with settings has revealed that GC stalls don’t make up enough time to explain the 10 seconds of apparent stall that would cause a hangup.
> 
> tpstats output typically shows active threads in the low single digits and pending similar or 0. Before a failure, pending MutationStage will skyrocket into 5+ digits. System.log regularly shows the gossiper complaining, then slow log complaints, then logs dropped mutations.
> 
> For the entire minute of 00:55, the gc logging shows no single pause > .14 seconds and most of them much smaller. Abbreviated GC log after switching to g1gc (problem also exhibited before G1GC):
> 
> 2015-10-30T00:55:00.550+0000: 6752.857: [GC pause (G1 Evacuation Pause) (young)
> 2015-10-30T00:55:02.843+0000: 6755.150: [GC pause (GCLocker Initiated GC) (young)
> 2015-10-30T00:55:05.241+0000: 6757.548: [GC pause (G1 Evacuation Pause) (young)
> 2015-10-30T00:55:07.755+0000: 6760.062: [GC pause (G1 Evacuation Pause) (young)
> 2015-10-30T00:55:10.532+0000: 6762.839: [GC pause (G1 Evacuation Pause) (young)
> 2015-10-30T00:55:13.080+0000: 6765.387: [GC pause (G1 Evacuation Pause) (young)
> 2015-10-30T00:55:15.914+0000: 6768.221: [GC pause (G1 Evacuation Pause) (young)
> 2015-10-30T00:55:18.619+0000: 6770.926: [GC pause (GCLocker Initiated GC) (young)
> 2015-10-30T00:55:23.270+0000: 6775.578: [GC pause (GCLocker Initiated GC) (young)
> 2015-10-30T00:55:28.662+0000: 6780.969: [GC pause (GCLocker Initiated GC) (young)
> 2015-10-30T00:55:33.326+0000: 6785.633: [GC pause (G1 Evacuation Pause) (young)
> 2015-10-30T00:55:36.600+0000: 6788.907: [GC pause (G1 Evacuation Pause) (young)
> 2015-10-30T00:55:40.050+0000: 6792.357: [GC pause (G1 Evacuation Pause) (young)
> 2015-10-30T00:55:43.728+0000: 6796.035: [GC pause (G1 Evacuation Pause) (young)
> 2015-10-30T00:55:48.216+0000: 6800.523: [GC pause (G1 Evacuation Pause) (young)
> 2015-10-30T00:55:53.621+0000: 6805.928: [GC pause (G1 Evacuation Pause) (young)
> 2015-10-30T00:55:59.048+0000: 6811.355: [GC pause (GCLocker Initiated GC) (young)
> 
> System log snippet of the pattern I’m seeing:
> 
> WARN  [GossipTasks:1] 2015-10-30 00:55:25,129  Gossiper.java:747 - Gossip stage has 1 pending tasks; skipping status check (no nodes will be marked down)
> INFO  [CompactionExecutor:210] 2015-10-30 00:55:26,006  CompactionTask.java:141 - Compacting [SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8283-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8286-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8284-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8285-Data.db'), SSTableReader(path='/mnt/cassandra/data/system/hints/system-hints-ka-8287-Data.db')]
> WARN  [GossipTasks:1] 2015-10-30 00:55:26,230  Gossiper.java:747 - Gossip stage has 3 pending tasks; skipping status check (no nodes will be marked down)
> WARN  [GossipTasks:1] 2015-10-30 00:55:27,330  Gossiper.java:747 - Gossip stage has 5 pending tasks; skipping status check (no nodes will be marked down)
> WARN  [GossipTasks:1] 2015-10-30 00:55:28,431  Gossiper.java:747 - Gossip stage has 7 pending tasks; skipping status check (no nodes will be marked down)
> WARN  [GossipTasks:1] 2015-10-30 00:55:29,531  Gossiper.java:747 - Gossip stage has 10 pending tasks; skipping status check (no nodes will be marked down)
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,448  CqlSlowLogWriter.java:151 - Recording statements with duration of 16042 in slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,451  CqlSlowLogWriter.java:151 - Recording statements with duration of 16047 in slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,453  CqlSlowLogWriter.java:151 - Recording statements with duration of 16018 in slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,454  CqlSlowLogWriter.java:151 - Recording statements with duration of 16042 in slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,455  CqlSlowLogWriter.java:151 - Recording statements with duration of 16024 in slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,457  CqlSlowLogWriter.java:151 - Recording statements with duration of 16045 in slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,458  CqlSlowLogWriter.java:151 - Recording statements with duration of 16041 in slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,459  CqlSlowLogWriter.java:151 - Recording statements with duration of 16040 in slow log
> INFO  [CqlSlowLog-Writer-thread-0] 2015-10-30 00:55:32,461  CqlSlowLogWriter.java:151 - Recording statements with duration of 16037 in slow log
> WARN  [GossipTasks:1] 2015-10-30 00:55:32,697  FailureDetector.java:257 - Not marking nodes down due to local pause of 8668629909 > 5000000000
> INFO  [CompactionExecutor:212] 2015-10-30 00:55:32,864  ColumnFamilyStore.java:900 - Enqueuing flush of compactions_in_progress: 164 (0%) on-heap, 0 (0%) off-heap
> INFO  [MemtableFlushWriter:846] 2015-10-30 00:55:32,865  Memtable.java:347 - Writing Memtable-compactions_in_progress@969357824(0.008KiB serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
> INFO  [MemtableFlushWriter:846] 2015-10-30 00:55:32,865  Memtable.java:382 - Completed flushing /mnt/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-tmp-ka-1461769-Data.db (0.000KiB) for commitlog position ReplayPosition(segmentId=1446159752729, position=8699402)
> INFO  [ScheduledTasks:1] 2015-10-30 00:55:35,346  MessagingService.java:930 - MUTATION messages were dropped in last 5000 ms: 67432 for internal timeout and 0 for cross node timeout
> INFO  [ScheduledTasks:1] 2015-10-30 00:55:35,350  MessagingService.java:930 - READ_REPAIR messages were dropped in last 5000 ms: 209 for internal timeout and 0 for cross node timeout
> 
> I’m starting to suspect I’ll have to take regular stack dumps and catch a bad moment, but I figured I’d reach out here before going down that road to see if somebody else had ever experienced a similar kind of pain.
> 
> -Jeff
>