Details
-
Improvement
-
Status: Resolved
-
Normal
-
Resolution: Fixed
-
None
-
Performance
-
Normal
-
All
-
None
-
Description
CASSANDRA-16286 achieved its goal of making sure that concurrent increments to the ring version would independently increment the version (i.e. not "merge" multiple invalidations into single versions), but it also unnecessarily replaced the volatile read on ringVersion w/ making readVersion non-volatile and acquiring the read lock on the fair ReadWriteLock in TokenMetadata. This can result in unnecessary queueing w/ high CPU usage/read volume. For example, you might see this on a 4.0 cluster...
"Native-Transport-Requests-99" #271 daemon prio=5 os_prio=0 cpu=5822566.56ms elapsed=19477779.40s tid=0x00007fcc96c31b00 nid=0xb7bd waiting on condition [0x00007fcb7f144000] java.lang.Thread.State: WAITING (parking) at jdk.internal.misc.Unsafe.park(java.base@11.0.16/Native Method) - parking to wait for <0x00000005c0ab92a0> (a java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) at java.util.concurrent.locks.LockSupport.park(java.base@11.0.16/LockSupport.java:194) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(java.base@11.0.16/AbstractQueuedSynchronizer.java:885) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireShared(java.base@11.0.16/AbstractQueuedSynchronizer.java:1009) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireShared(java.base@11.0.16/AbstractQueuedSynchronizer.java:1324) at java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.lock(java.base@11.0.16/ReentrantReadWriteLock.java:738) at org.apache.cassandra.locator.TokenMetadata.getRingVersion(TokenMetadata.java:1389) at org.apache.cassandra.locator.AbstractReplicationStrategy.getCachedReplicas(AbstractReplicationStrategy.java:82) at org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalReplicas(AbstractReplicationStrategy.java:116) at org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalReplicasForToken(AbstractReplicationStrategy.java:109) at org.apache.cassandra.locator.ReplicaLayout.forTokenWriteLiveAndDown(ReplicaLayout.java:209) at org.apache.cassandra.locator.ReplicaPlans.forWrite(ReplicaPlans.java:328) at org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:1426) at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:937)
Reverting to a volatile read makes this no longer possible, but keeps the fix from CASSANDRA-16286 intact.
Attachments
Attachments
Issue Links
- is caused by
-
CASSANDRA-16286 Make TokenMetadata's ring version increments atomic
- Resolved
- links to