[ https://issues.apache.org/jira/browse/CASSANDRA-20607?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Berenguer Blasi updated CASSANDRA-20607: ---------------------------------------- Fix Version/s: 4.1.x 5.0.x 6.x > Gossip thread slows down when forceBlockingFlush is called on system.peers > and system.peers_v2 tables > ----------------------------------------------------------------------------------------------------- > > Key: CASSANDRA-20607 > URL: https://issues.apache.org/jira/browse/CASSANDRA-20607 > Project: Apache Cassandra > Issue Type: Bug > Reporter: Hieu Ngo > Priority: Normal > Fix For: 4.1.x, 5.0.x, 6.x > > > The metrics system at Dropbox.com (Vortex2) uses cassandra as the data > storage layer to store metric and metadata data. The system is undergoing a > migration project to move cassandra hosted on premise to an in-house > kubernetes-based deployment system. > Whenever a pod running cassandra is rotated out during a deployment, we have > seen many nodes deemed as Down Normal pretty much at the same time for a > brief period (15 - 30s), resulting in excessive hints created, timed out and > unavailable exceptions. > It turns out that the bug is at the forceBlockingFlush call on system.peers > and system.peers_v2 tables happening within the single-threaded Gossip Stage. > The flush takes too long, occupying the Gossip STage and causing gossip tasks > to pile up. Since gossip tasks aren't sent properly and timely, nodes in the > system start marking other nodes as Down Normal. From the stack trace, we > believe the forceBlockFlush is called as part of the host ip update when the > rotated pod changes its IP. > We have created a patch internally to change the flush to nonBlockingFlush if > the flag cassandra.load_ring_state=false. The rationale for the fix is that > since the ring state isn't loaded from those peers tables during startup, > there is no need to wait for the flush. However, we haven't figured out why > the blocking flush to system.peers and system.peers_v2 tables takes so long. > The asks for this bug are: > 1. Figure out why blocking flush to system.peers and system.peers_v2 tables > takes so long. > 2. Create a fix to change the flush to non-blocking if > cassandra.load_ring_state is set to false. We notice that the > cassandra.load_ring_state is removed in cassandra 5.0 so the fix may only be > applicable to 4.x and older versions. > > Here is the stack trace of GossipStage: > > {code:java} > "GossipStage:1" #205 daemon prio=5 os_prio=0 cpu=113114.96ms > elapsed=233376.38s tid=0x00007fad2002c480 nid=0x3a5 waiting on condition > [0x00007faf9613d000] > java.lang.Thread.State: WAITING (parking) > at jdk.internal.misc.Unsafe.park(java.base@11.0.25/Native Method) > at > java.util.concurrent.locks.LockSupport.park(java.base@11.0.25/LockSupport.java:323) > at > org.apache.cassandra.utils.concurrent.WaitQueue$Standard$AbstractSignal.await(WaitQueue.java:289) > at > org.apache.cassandra.utils.concurrent.WaitQueue$Standard$AbstractSignal.await(WaitQueue.java:282) > at > org.apache.cassandra.utils.concurrent.Awaitable$AsyncAwaitable.await(Awaitable.java:306) > at > org.apache.cassandra.utils.concurrent.AsyncFuture.await(AsyncFuture.java:154) > at > org.apache.cassandra.utils.concurrent.AsyncFuture.await(AsyncFuture.java:46) > at > org.apache.cassandra.utils.concurrent.AbstractFuture.get(AbstractFuture.java:245) > at org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:501) > at org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:477) > at > org.apache.cassandra.db.SystemKeyspace.forceBlockingFlush(SystemKeyspace.java:947) > at > org.apache.cassandra.db.SystemKeyspace.removeEndpoint(SystemKeyspace.java:916) > locked <0x0000000101dfccc0> (a java.lang.Class for > org.apache.cassandra.db.SystemKeyspace) > at > org.apache.cassandra.service.StorageService.removeEndpoint(StorageService.java:3339) > at > org.apache.cassandra.service.StorageService.updateTokenMetadata(StorageService.java:3052) > at > org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:3176) > at > org.apache.cassandra.service.StorageService.onChange(StorageService.java:2638) > at > org.apache.cassandra.gms.Gossiper.doOnChangeNotifications(Gossiper.java:1711) > at > org.apache.cassandra.service.StorageService.onJoin(StorageService.java:3652) > at > org.apache.cassandra.gms.Gossiper.handleMajorStateChange(Gossiper.java:1481) > at org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:1627) > at > org.apache.cassandra.gms.GossipDigestAck2VerbHandler.doVerb(GossipDigestAck2VerbHandler.java:50) > at org.apache.cassandra.net.InboundSink.lambda$new$0(InboundSink.java:78) > at > org.apache.cassandra.net.InboundSink$$Lambda$603/0x000000080069c040.accept(Unknown > Source) > at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:97) > at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:45) > at > org.apache.cassandra.net.InboundMessageHandler$ProcessMessage.run(InboundMessageHandler.java:430) > at > org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:133) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.25/ThreadPoolExecutor.java:1128) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.25/ThreadPoolExecutor.java:628) > at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) > at java.lang.Thread.run(java.base@11.0.25/Thread.java:829) > {code} > > Here is the PerDiskMemtableFlushWriter that the GossipStage was waiting for: > > {code:java} > "PerDiskMemtableFlushWriter_0:470" #170632 daemon prio=5 os_prio=0 > cpu=42316.16ms elapsed=44.53s tid=0x00007faee40c91c0 nid=0x79f5 runnable > [0x00007faf70613000] > java.lang.Thread.State: RUNNABLE > at > org.apache.cassandra.db.marshal.AbstractType.writeValue(AbstractType.java:469) > at > org.apache.cassandra.db.ClusteringPrefix$Serializer.serializeValuesWithoutSize(ClusteringPrefix.java:397) > at > org.apache.cassandra.db.Clustering$Serializer.serialize(Clustering.java:133) > at > org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:188) > at > org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:137) > at org.apache.cassandra.db.ColumnIndex.add(ColumnIndex.java:258) > at org.apache.cassandra.db.ColumnIndex.buildRowIndex(ColumnIndex.java:122) > at > org.apache.cassandra.io.sstable.format.big.BigTableWriter.append(BigTableWriter.java:222) > at > org.apache.cassandra.io.sstable.SimpleSSTableMultiWriter.append(SimpleSSTableMultiWriter.java:48) > at > org.apache.cassandra.db.memtable.Flushing$FlushRunnable.writeSortedContents(Flushing.java:163) > at > org.apache.cassandra.db.memtable.Flushing$FlushRunnable.call(Flushing.java:183) > at > org.apache.cassandra.db.memtable.Flushing$FlushRunnable.call(Flushing.java:122) > at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61) > at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.25/ThreadPoolExecutor.java:1128) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.25/ThreadPoolExecutor.java:628) > at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) > at java.lang.Thread.run(java.base@11.0.25/Thread.java:829) {code} > -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org