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

MemtableReclaimMemory can get stuck because of lack of timeout in getTopLevelColumns()

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Normal
    • Resolution: Won't Fix
    • 2.1.x
    • Legacy/Core
    • None
    • cassandra 2.1.15

    • Normal

    Description

      This might affect 3.x too, I'm not sure.

      $ nodetool tpstats
      Pool Name                    Active   Pending      Completed   Blocked  All time blocked
      MutationStage                     0         0       32135875         0                 0
      ReadStage                       114         0       29492940         0                 0
      RequestResponseStage              0         0       86090931         0                 0
      ReadRepairStage                   0         0         166645         0                 0
      CounterMutationStage              0         0              0         0                 0
      MiscStage                         0         0              0         0                 0
      HintedHandoff                     0         0             47         0                 0
      GossipStage                       0         0         188769         0                 0
      CacheCleanupExecutor              0         0              0         0                 0
      InternalResponseStage             0         0              0         0                 0
      CommitLogArchiver                 0         0              0         0                 0
      CompactionExecutor                0         0          86835         0                 0
      ValidationExecutor                0         0              0         0                 0
      MigrationStage                    0         0              0         0                 0                                    
      AntiEntropyStage                  0         0              0         0                 0                                    
      PendingRangeCalculator            0         0             92         0                 0                                    
      Sampler                           0         0              0         0                 0                                    
      MemtableFlushWriter               0         0            563         0                 0                                    
      MemtablePostFlush                 0         0           1500         0                 0                                    
      MemtableReclaimMemory             1        29            534         0                 0                                    
      Native-Transport-Requests        41         0       54819182         0              1896                            
      
      "MemtableReclaimMemory:195" - Thread t@6268
         java.lang.Thread.State: WAITING
      	at sun.misc.Unsafe.park(Native Method)
      	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
      	at org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:283)
      	at org.apache.cassandra.utils.concurrent.OpOrder$Barrier.await(OpOrder.java:417)
      	at org.apache.cassandra.db.ColumnFamilyStore$Flush$1.runMayThrow(ColumnFamilyStore.java:1151)
      	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      	at java.lang.Thread.run(Thread.java:745)
      
         Locked ownable synchronizers:
      	- locked <6e7b1160> (a java.util.concurrent.ThreadPoolExecutor$Worker)
      
      "SharedPool-Worker-195" - Thread t@989
         java.lang.Thread.State: RUNNABLE
      	at org.apache.cassandra.db.RangeTombstoneList.addInternal(RangeTombstoneList.java:690)
      	at org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:650)
      	at org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:171)
      	at org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:143)
      	at org.apache.cassandra.db.DeletionInfo.add(DeletionInfo.java:240)
      	at org.apache.cassandra.db.ArrayBackedSortedColumns.delete(ArrayBackedSortedColumns.java:483)
      	at org.apache.cassandra.db.ColumnFamily.addAtom(ColumnFamily.java:153)
      	at org.apache.cassandra.db.filter.QueryFilter$2.getNext(QueryFilter.java:184)
      	at org.apache.cassandra.db.filter.QueryFilter$2.hasNext(QueryFilter.java:156)
      	at org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:146)
      	at org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:125)
      	at org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:99)
      	at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
      	at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
      	at org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:263)
      	at org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:108)
      	at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:82)
      	at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:69)
      	at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:316)
      	at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:62)
      	at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:2015)
      	at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1858)
      	at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:353)
      	at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:85)
      	at org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:47)
      	at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:64)
      	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      	at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164)
      	at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105)
      	at java.lang.Thread.run(Thread.java:745)
      
         Locked ownable synchronizers:
      	- None
      
      "SharedPool-Worker-206" - Thread t@1014
         java.lang.Thread.State: RUNNABLE
      	at org.apache.cassandra.db.RangeTombstoneList.addInternal(RangeTombstoneList.java:690)
      	at org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:650)
      	at org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:171)
      	at org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:143)
      	at org.apache.cassandra.db.DeletionInfo.add(DeletionInfo.java:240)
      	at org.apache.cassandra.db.ArrayBackedSortedColumns.delete(ArrayBackedSortedColumns.java:483)
      	at org.apache.cassandra.db.ColumnFamily.addAtom(ColumnFamily.java:153)
      	at org.apache.cassandra.db.filter.QueryFilter$2.getNext(QueryFilter.java:184)
      	at org.apache.cassandra.db.filter.QueryFilter$2.hasNext(QueryFilter.java:156)
      	at org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:146)
      	at org.apache.cassandra.utils.MergeIterator$ManyToOne.<init>(MergeIterator.java:89)
      	at org.apache.cassandra.utils.MergeIterator.get(MergeIterator.java:48)
      	at org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:105)
      	at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:82)
      	at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:69)
      	at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:316)
      	at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:62)
      	at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:2015)
      	at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1858)
      	at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:353)
      	at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:85)
      	at org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:47)
      	at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:64)
      	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      	at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164)
      	at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105)
      	at java.lang.Thread.run(Thread.java:745)
      
         Locked ownable synchronizers:
      	- None
      

      As you can see MemtableReclaimMemory is waiting on the read barrier to be released, but there are two queries currently being executed which are locking this.

      Since most of the time is spent pretty low in the stack, these read operations will never timeout (they are reading rows with tons of tombstones).

      We also can easily detect or purge the offending line because there is no easy way to find out which partition is currently being read.

      The TombstoneFailureThreshold should also protect us, but it is probably being checked too high in the call stack.

      Looks like RangeTombstoneList or DeletionInfo should also check for DatabaseDescriptor.getTombstoneFailureThreshold()

      Attachments

        Activity

          People

            Unassigned Unassigned
            iksaif Corentin Chary
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: