Uploaded image for project: 'Cassandra'
  1. Cassandra
  2. CASSANDRA-18443

Deadlock updating sstable metadata if disk boundaries need reloading

    XMLWordPrintableJSON

Details

    Description

      CompactionStrategyManager.handleNotification holds the read lock while processing notifications. When handling metadata changed notifications, an extra call is made to maybeReloadDiskBoundaries which tries to grab the write lock and deadlocks the thread.

      Partial stacktrace

              at jdk.internal.misc.Unsafe.park(java.base@11.0.16/Native Method)
              - parking to wait for  <0x00000005cc000078> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync)
              at java.util.concurrent.locks.LockSupport.park
              at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt
              at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued
              at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire
              at java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock
              at org.apache.cassandra.db.compaction.CompactionStrategyManager.maybeReloadDiskBoundaries(CompactionStrategyManager.java:495)
              at org.apache.cassandra.db.compaction.CompactionStrategyManager.getCompactionStrategyFor(CompactionStrategyManager.java:343)
              at org.apache.cassandra.db.compaction.CompactionStrategyManager.handleMetadataChangedNotification(CompactionStrategyManager.java:796)
              at org.apache.cassandra.db.compaction.CompactionStrategyManager.handleNotification(CompactionStrategyManager.java:838)
              at org.apache.cassandra.db.lifecycle.Tracker.notifySSTableMetadataChanged(Tracker.java:482)
              at org.apache.cassandra.db.compaction.CompactionStrategyManager.handleNotification(CompactionStrategyManager.java:838)
      

      Deadlocking with the read lock held blocks the SlabpoolCleaner while notifying ColumnFamilyStore so memtables are prevented from being flushed and recycled, causing any thread applying a mutation to the database (at least GossipStage and MutationStage) to be considered down by peers and/or back up with pending requests.

      All the cases investigated were during single sstable upleveling by org.apache.cassandra.db.compaction.SingleSSTableLCSTask added in CASSANDRA-12526.

      Other less critical work was also affected, JMX calls to get estimated remaining compaction tasks, the index summary manager redistributing summaries, the StatusLogger trying to log dropped messages, and the ValidationManager.

      Workaround is to reboot the affected host.

      The fix is to just remove the redundant disk boundary reload check on that path.

      Attachments

        Activity

          People

            jonmeredith Jon Meredith
            jonmeredith Jon Meredith
            Jon Meredith
            Marcus Eriksson
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: