You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Joel Knighton (JIRA)" <ji...@apache.org> on 2016/08/02 18:54:20 UTC

[jira] [Commented] (CASSANDRA-11709) Lock contention when large number of dead nodes come back within short time

    [ https://issues.apache.org/jira/browse/CASSANDRA-11709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15404599#comment-15404599 ] 

Joel Knighton commented on CASSANDRA-11709:
-------------------------------------------

A long, long overdue update. I have a branch here that should greatly decrease the number of times we invalidate the cached ring on nodes restarting/gossip restarting. I believe this should significantly help the problem based on the logs/jstack provided so far. That said, I still haven't had a large enough cluster to fully faithfully reproduce this, so there may be other problems lurking after this (potential) fix.

The branch is available at [CASSANDRA-11709-2.2|https://github.com/jkni/cassandra/tree/CASSANDRA-11709-2.2]. I'm holding off on making this officially patch available while I do further testing; I've been trying to run CI on this but other problems are making it difficult for me to have a conclusive run at the moment. From what I can tell, CI looks pretty good on it.

If anyone who can reproduce this issue is interested in testing this patch in a dev environment, that would be greatly appreciated. I would strongly recommend against putting this patch in an environment of any importance until I test it more.

> Lock contention when large number of dead nodes come back within short time
> ---------------------------------------------------------------------------
>
>                 Key: CASSANDRA-11709
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-11709
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Dikang Gu
>            Assignee: Joel Knighton
>             Fix For: 2.2.x, 3.x
>
>         Attachments: lock.jstack
>
>
> We have a few hundreds nodes across 3 data centers, and we are doing a few millions writes per second into the cluster. 
> We were trying to simulate a data center failure, by disabling the gossip on all the nodes in one data center. After ~20mins, I re-enabled the gossip on those nodes, was doing 5 nodes in each batch, and sleep 5 seconds between the batch.
> After that, I saw the latency of read/write requests increased a lot, and client requests started to timeout.
> On the node, I can see there are huge number of pending tasks in GossipStage. 
> =====
> 2016-05-02_23:55:08.99515 WARN  23:55:08 Gossip stage has 36337 pending tasks; skipping status check (no nodes will be marked down)
> 2016-05-02_23:55:09.36009 INFO  23:55:09 Node /2401:db00:2020:717a:face:0:41:0 state jump to normal
> 2016-05-02_23:55:09.99057 INFO  23:55:09 Node /2401:db00:2020:717a:face:0:43:0 state jump to normal
> 2016-05-02_23:55:10.09742 WARN  23:55:10 Gossip stage has 36421 pending tasks; skipping status check (no nodes will be marked down)
> 2016-05-02_23:55:10.91860 INFO  23:55:10 Node /2401:db00:2020:717a:face:0:45:0 state jump to normal
> 2016-05-02_23:55:11.20100 WARN  23:55:11 Gossip stage has 36558 pending tasks; skipping status check (no nodes will be marked down)
> 2016-05-02_23:55:11.57893 INFO  23:55:11 Node /2401:db00:2030:612a:face:0:49:0 state jump to normal
> 2016-05-02_23:55:12.23405 INFO  23:55:12 Node /2401:db00:2020:7189:face:0:7:0 state jump to normal
> ====
> And I took jstack of the node, I found the read/write threads are blocked by a lock,
> ==================== read thread ==============
> "Thrift:7994" daemon prio=10 tid=0x00007fde91080800 nid=0x5255 waiting for monitor entry [0x00007fde6f8a1000]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>         at org.apache.cassandra.locator.TokenMetadata.cachedOnlyTokenMap(TokenMetadata.java:546)
>         - waiting to lock <0x00007fe4faef4398> (a org.apache.cassandra.locator.TokenMetadata)
>         at org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:111)
>         at org.apache.cassandra.service.StorageService.getLiveNaturalEndpoints(StorageService.java:3155)
>         at org.apache.cassandra.service.StorageProxy.getLiveSortedEndpoints(StorageProxy.java:1526)
>         at org.apache.cassandra.service.StorageProxy.getLiveSortedEndpoints(StorageProxy.java:1521)
>         at org.apache.cassandra.service.AbstractReadExecutor.getReadExecutor(AbstractReadExecutor.java:155)
>         at org.apache.cassandra.service.StorageProxy.fetchRows(StorageProxy.java:1328)
>         at org.apache.cassandra.service.StorageProxy.readRegular(StorageProxy.java:1270)
>         at org.apache.cassandra.service.StorageProxy.read(StorageProxy.java:1195)
>         at org.apache.cassandra.thrift.CassandraServer.readColumnFamily(CassandraServer.java:118)
>         at org.apache.cassandra.thrift.CassandraServer.getSlice(CassandraServer.java:275)
>         at org.apache.cassandra.thrift.CassandraServer.multigetSliceInternal(CassandraServer.java:457)
>         at org.apache.cassandra.thrift.CassandraServer.getSliceInternal(CassandraServer.java:346)
>         at org.apache.cassandra.thrift.CassandraServer.get_slice(CassandraServer.java:325)
>         at org.apache.cassandra.thrift.Cassandra$Processor$get_slice.getResult(Cassandra.java:3659)
>         at org.apache.cassandra.thrift.Cassandra$Processor$get_slice.getResult(Cassandra.java:3643)
>         at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
>         at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
>         at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:205)
>         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)
> =====  writer ===================
> "Thrift:7668" daemon prio=10 tid=0x00007fde90d91000 nid=0x50e9 waiting for monitor entry [0x00007fde78bbc000]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>         at org.apache.cassandra.locator.TokenMetadata.cachedOnlyTokenMap(TokenMetadata.java:546)
>         - waiting to lock <0x00007fe4faef4398> (a org.apache.cassandra.locator.TokenMetadata)
>         at org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:111)
>         at org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:3137)
>         at org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:771)
>         at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:538)
>         at org.apache.cassandra.service.StorageProxy.mutateWithTriggers(StorageProxy.java:613)
>         at org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:1101)
>         at org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:1083)
>         at org.apache.cassandra.thrift.CassandraServer.batch_mutate(CassandraServer.java:976)
>         at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3996)
>         at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3980)
>         at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
>         at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
>         at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:205)
>         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)
> ============== what the lock is obtained for ==========
> "Thrift:6305" daemon prio=10 tid=0x00007fde90176000 nid=0x4af4 runnable [0x00007fde8e618000]
>    java.lang.Thread.State: RUNNABLE
>         at java.nio.ByteBuffer.wrap(ByteBuffer.java:369)
>         at java.nio.ByteBuffer.wrap(ByteBuffer.java:392)
>         at org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:100)
>         at org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:97)
>         at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:369)
>         at java.util.TreeMap.getEntry(TreeMap.java:340)
>         at java.util.TreeMap.get(TreeMap.java:273)
>         at com.google.common.collect.AbstractMapBasedMultimap.put(AbstractMapBasedMultimap.java:192)
>         at com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:121)
>         at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:78)
>         at com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:90)
>         at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:78)
>         at org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
>         at org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:522)
>         at org.apache.cassandra.locator.TokenMetadata.cachedOnlyTokenMap(TokenMetadata.java:551)
>         - locked <0x00007fe4faef4398> (a org.apache.cassandra.locator.TokenMetadata)
>         at org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:111)
>         at org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:3137)
>         at org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:771)
>         at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:538)
>         at org.apache.cassandra.service.StorageProxy.mutateWithTriggers(StorageProxy.java:613)
>         at org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:1101)
>         at org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:1083)
>         at org.apache.cassandra.thrift.CassandraServer.batch_mutate(CassandraServer.java:976)
>         at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3996)
>         at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3980)
>         at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
>         at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
>         at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:205)
>         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)
> =====================
> So looks like too many pending gossips invalidated the tokenMap too frequently, is it a known problem?
> Thanks.



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