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

Race condition between flushing and compaction stalls compaction indefinitely

Agile BoardAttach filesAttach ScreenshotBulk Copy AttachmentsBulk Move AttachmentsVotersWatch issueWatchersCreate sub-taskConvert to sub-taskMoveLinkCloneLabelsUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • All
    • None

    Description

      Seen on Cassandra 3.11.4 with OpenJDK 8u212, although I've seen this a few times before, also on 3.11.3. It's a rare issue so I've not bothered with trying to trace it until now.

      DEBUG [NativePoolCleaner] 2019-07-18 01:12:41,799 ColumnFamilyStore.java:1325 - Flushing largest CFS(Keyspace='keyspacename', ColumnFamily='tablename') to free up room. Used total: 0.10/0.33, live: 0.10/0.33, flushing: 0.00/0.00, this: 0.09/0.19
      DEBUG [NativePoolCleaner] 2019-07-18 01:12:41,800 ColumnFamilyStore.java:935 - Enqueuing flush of tablename: 267.930MiB (9%) on-heap, 575.580MiB (19%) off-heap
      DEBUG [PerDiskMemtableFlushWriter_0:204] 2019-07-18 01:12:42,480 Memtable.java:456 - Writing Memtable-tablename@498336646(520.721MiB serialized bytes, 870200 ops, 9%/19% of on/off-heap limit), flushed range = (min(-9223372036854775808), max(9223372036854775807)]
      INFO  [Service Thread] 2019-07-18 01:12:43,616 GCInspector.java:284 - G1 Young Generation GC in 227ms.  G1 Eden Space: 14713618432 -> 0; G1 Old Gen: 13240876928 -> 13259198848; G1 Survivor Space: 276824064 -> 268435456;
      INFO  [Service Thread] 2019-07-18 01:12:56,251 GCInspector.java:284 - G1 Young Generation GC in 206ms.  G1 Eden Space: 14713618432 -> 0; G1 Old Gen: 13259198848 -> 13285123456; G1 Survivor Space: 268435456 -> 285212672;
      DEBUG [PerDiskMemtableFlushWriter_0:204] 2019-07-18 01:12:56,693 Memtable.java:485 - Completed flushing /cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db (524.023MiB) for commitlog position CommitLogPosition(segmentId=1563386911266, position=32127822)
      DEBUG [MemtableFlushWriter:204] 2019-07-18 01:12:57,620 ColumnFamilyStore.java:1233 - Flushed to [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db')] (1 sstables, 518.714MiB), biggest 518.714MiB, smallest 518.714MiB
      WARN  [CompactionExecutor:1617] 2019-07-18 01:12:57,628 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.
      

      This final line then starts repeating about once per minute:

      WARN  [CompactionExecutor:1610] 2019-07-18 01:13:18,898 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.
      
      WARN  [CompactionExecutor:1611] 2019-07-18 01:14:18,899 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.
      
      WARN  [CompactionExecutor:1622] 2019-07-18 01:15:18,899 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.
      
      WARN  [CompactionExecutor:1436] 2019-07-18 01:16:15,073 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.
      
      WARN  [CompactionExecutor:1618] 2019-07-18 01:16:18,899 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.
      
      WARN  [CompactionExecutor:1611] 2019-07-18 01:17:18,900 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.
      
      WARN  [CompactionExecutor:1606] 2019-07-18 01:18:18,900 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.
      
      WARN  [CompactionExecutor:1630] 2019-07-18 01:19:18,902 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.
      
      WARN  [CompactionExecutor:1627] 2019-07-18 01:20:18,904 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.
      
      WARN  [CompactionExecutor:1638] 2019-07-18 01:21:18,904 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.
      
      WARN  [CompactionExecutor:1631] 2019-07-18 01:22:18,905 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.
      
      WARN  [CompactionExecutor:1636] 2019-07-18 01:22:58,220 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292363-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292342-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292344-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292343-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292340-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292338-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292336-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292335-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292337-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292346-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292349-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.
      
      WARN  [CompactionExecutor:1625] 2019-07-18 01:23:18,905 LeveledCompactionStrategy.java:144 - Could not acquire references for compacting SSTables [BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292363-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292348-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292358-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292342-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292344-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292343-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292340-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292338-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292336-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292335-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292337-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292346-big-Data.db'), BigTableReader(path='/cassandra-data/data/keyspacename/tablename-2c5325e042e911e8a07e9db72d27cf67/md-292349-big-Data.db')] which is not a problem per se,unless it happens frequently, in which case it must be reported. Will retry later.

      It will keep going like this for days, until restarted, but compaction won't run until then, so sstables pile up.

      Attachments

        Issue Links

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            marcuse Marcus Eriksson Assign to me
            tvdw Tom van der Woerdt
            Marcus Eriksson
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment