You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Sebastian Estevez (JIRA)" <ji...@apache.org> on 2015/05/01 00:01:05 UTC

[jira] [Updated] (CASSANDRA-9279) Gossip (and mutations) lock up on startup

     [ https://issues.apache.org/jira/browse/CASSANDRA-9279?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Sebastian Estevez updated CASSANDRA-9279:
-----------------------------------------
    Summary: Gossip (and mutations) lock up on startup  (was: Gossip locks up on startup)

> Gossip (and mutations) lock up on startup
> -----------------------------------------
>
>                 Key: CASSANDRA-9279
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-9279
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Sebastian Estevez
>         Attachments: Screen Shot 2015-04-30 at 4.41.57 PM.png
>
>
> Cluster running 2.0.14.352 on EC2 - c3.4xl's
> 2 nodes out of 8 exhibited the following behavior
> When starting up the node we noticed it was gray in OpsCenter. Other monitoring tool showed it as up. 
> Turned out gossip tasks were piling up and we could see the following in the system.log:
> {code}
>  WARN [GossipTasks:1] 2015-04-30 20:22:29,512 Gossiper.java (line 671) Gossip stage has 4270 pending tasks; skipping status check (no nodes will be marked down)
>  WARN [GossipTasks:1] 2015-04-30 20:22:30,612 Gossiper.java (line 671) Gossip stage has 4272 pending tasks; skipping status check (no nodes will be marked down)
>  WARN [GossipTasks:1] 2015-04-30 20:22:31,713 Gossiper.java (line 671) Gossip stage has 4273 pending tasks; skipping status check (no nodes will be marked down)
> ...
> {code}
> and tpstats shows blocked tasks--gossip and mutations:
> {code}
> GossipStage                       1      3904          29384         0                 0
> {code}
> the CPU's are inactive (See attachment)
> and dstat output:
> {code}
> You did not select any stats, using -cdngy by default.
> ----total-cpu-usage---- -dsk/total- -net/total- ---paging-- ---system--
> usr sys idl wai hiq siq| read  writ| recv  send|  in   out | int   csw
>   2   0  97   0   0   0|1324k 1381k|   0     0 |   0     0 |6252  5548
>   0   0 100   0   0   0|   0    64k|  42k 1017k|   0     0 |3075  2537
>   0   0  99   0   0   0|   0  8192B|  39k  794k|   0     0 |6999  7039
>   0   0 100   0   0   0|   0     0 |  39k  759k|   0     0 |3067  2726
>   0   0  99   0   0   0|   0   184k|  48k 1086k|   0     0 |4829  4178
>   0   0  99   0   0   0|   0  8192B|  34k  802k|   0     0 |1671  1240
>   0   0 100   0   0   0|   0  8192B|  48k 1067k|   0     0 |1878  1193
> {code}
> I managed to grab a thread dump:
> https://gist.githubusercontent.com/anonymous/3b7b4698c32032603493/raw/read.md
> and dmesg:
> https://gist.githubusercontent.com/anonymous/5982b15337c9afbd5d49/raw/f3c2e4411b9d59e90f4615d93c7c1ad25922e170/read.md
> Restarting the node solved the issue (it came up normally), we don't know what is causing it but apparently (per the thread dump) gossip threads are blocked writing the system keyspace and the writes waiting on the commitlog.
> Gossip:
> {code}
> "GossipStage:1" daemon prio=10 tid=0x00007ffa23471800 nid=0xa13fa waiting on condition [0x00007ff9cbe26000]
>    java.lang.Thread.State: WAITING (parking)
> 	at sun.misc.Unsafe.park(Native Method)
> 	- parking to wait for  <0x00000005d3f50960> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync)
> 	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireShared(AbstractQueuedSynchronizer.java:964)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireShared(AbstractQueuedSynchronizer.java:1282)
> 	at java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.lock(ReentrantReadWriteLock.java:731)
> 	at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:351)
> 	at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:336)
> 	at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:211)
> 	at org.apache.cassandra.cql3.statements.ModificationStatement.executeInternal(ModificationStatement.java:709)
> 	at org.apache.cassandra.cql3.QueryProcessor.processInternal(QueryProcessor.java:208)
> 	at org.apache.cassandra.db.SystemKeyspace.updatePeerInfo(SystemKeyspace.java:379)
> 	- locked <0x00000005d3f41ed8> (a java.lang.Class for org.apache.cassandra.db.SystemKeyspace)
> 	at org.apache.cassandra.service.StorageService.updatePeerInfo(StorageService.java:1414)
> 	at org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:1524)
> 	at org.apache.cassandra.service.StorageService.onChange(StorageService.java:1350)
> 	at org.apache.cassandra.gms.Gossiper.doOnChangeNotifications(Gossiper.java:1083)
> 	at org.apache.cassandra.gms.Gossiper.applyNewStates(Gossiper.java:1065)
> 	at org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:1023)
> 	at org.apache.cassandra.gms.GossipDigestAckVerbHandler.doVerb(GossipDigestAckVerbHandler.java:58)
> 	at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:62)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> 	at java.lang.Thread.run(Thread.java:744)
>    Locked ownable synchronizers:
> 	- <0x0000000609517438> (a java.util.concurrent.ThreadPoolExecutor$Worker)
> ...
> {code}
> Mutation:
> {code}
> "MutationStage:32" daemon prio=10 tid=0x00007ffa2339c800 nid=0xa1399 waiting on condition [0x00007ff9cd6c8000]
>    java.lang.Thread.State: WAITING (parking)
> 	at sun.misc.Unsafe.park(Native Method)
> 	- parking to wait for  <0x00000005d486a888> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> 	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2043)
> 	at java.util.concurrent.LinkedBlockingQueue.put(LinkedBlockingQueue.java:349)
> 	at org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService.add(PeriodicCommitLogExecutorService.java:106)
> 	at org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:206)
> 	at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:357)
> 	at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:336)
> 	at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:211)
> 	at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:56)
> 	at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:62)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> 	at java.lang.Thread.run(Thread.java:744)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)