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

Node stopped serving write requests when a table has a lot of sstables

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

Details

    • Bug
    • Status: Resolved
    • Normal
    • Resolution: Fixed
    • None
    • Local/Compaction
    • None
    • Normal

    Description

      Cluster was flooded with SSTables. A table had ~20000 sstables. Write requests started failing.

      Steps to reproduce:

      • Create cluster with 3 nodes
      • Specify
        memtable_heap_space_in_mb: 10
        

        in cassandra.yaml

      • Create table standard1 in keyspace1 (for the cassandra-stress tool) with the script create.cql. Please note
         compaction = {'class': 'SizeTieredCompactionStrategy', 'enabled': 'false'} 

        i.e. compaction will be turned off for now.

      • Populate node with data:
         cassandra-stress write n=1000000000 -node 127.0.0.1 
      • After node was populated, put both read and write pressure on it:
          cassandra-stress read n=1000000000 -node 127.0.0.1 
          cassandra-stress write n=1000000000 -node 127.0.0.1 
      • While still under pressure, enable LeveledCompactionStrategy
          echo "ALTER TABLE keyspace1.standard1 WITH compaction = { 'class' : 'LeveledCompactionStrategy', 'sstable_size_in_mb' : 1 }; DESC keyspace1.standard1; exit" | bin/cqlsh; 

      Results:
      Write requests failing.
      'bin/nodetool cfstats' and 'bin/nodetool compactionstats' commands hanging, if issued from the node running cassandra-stress tool.

      If issued from another node:

       $ bin/nodetool cfstats
      ...
      Table: standard1
                      SSTable count: 22637
                      SSTables in each level: [22651/4, 0, 0, 0, 0, 0, 0, 0, 0] 
      ...
      
      $ bin/nodetool compactionstats
      
      pending tasks: 12656
                                           id   compaction type    keyspace       table   completed       total    unit   progress
         935bbc00-d03b-11e7-a47d-2b44293495b8        Compaction   keyspace1   standard1    59556014    59557860   bytes    100.00%
         a29ee660-d03b-11e7-a47d-2b44293495b8        Compaction   keyspace1   standard1    80432114   742151655   bytes     10.84%
         9766e400-d03b-11e7-a47d-2b44293495b8        Compaction   keyspace1   standard1    58891604    58893215   bytes    100.00%
         9cdc9880-d03b-11e7-a47d-2b44293495b8        Compaction   keyspace1   standard1    20289449    20290800   bytes     99.99%
         90f98910-d03b-11e7-a47d-2b44293495b8        Compaction   keyspace1   standard1    59689824    59695545   bytes     99.99%
         986ede20-d03b-11e7-a47d-2b44293495b8        Compaction   keyspace1   standard1    40598594    40598820   bytes    100.00%
         9cd322a0-d03b-11e7-a47d-2b44293495b8        Compaction   keyspace1   standard1    60756739    60766660   bytes     99.98% 
      

      Special note about 'bin/nodetool compactionstats' - picture above is quite typical for this issue. I.e. compaction tasks manage to make it through, but hinder near the full completion (around 99.9 %).

      Maybe the root of the problem is in this thread (see stack.txt):

      "CompactionExecutor:1748" #4649 daemon prio=1 os_prio=4 tid=0x00007f35a0096100 nid=0x65f6 runnable [0x00007f3228bce000]
         java.lang.Thread.State: RUNNABLE
        at org.apache.cassandra.dht.AbstractBounds.<init>(AbstractBounds.java:53)
        at org.apache.cassandra.dht.Bounds.<init>(Bounds.java:42)
        at org.apache.cassandra.db.compaction.LeveledManifest.overlapping(LeveledManifest.java:562)
        at org.apache.cassandra.db.compaction.LeveledManifest.overlapping(LeveledManifest.java:549)
        at org.apache.cassandra.db.compaction.LeveledManifest.getCandidatesFor(LeveledManifest.java:624)
        at org.apache.cassandra.db.compaction.LeveledManifest.getCompactionCandidates(LeveledManifest.java:378)
        - locked <0x000000070d4c3bc8> (a org.apache.cassandra.db.compaction.LeveledManifest)
        at org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:105)
        - locked <0x000000070d6cb2c8> (a org.apache.cassandra.db.compaction.LeveledCompactionStrategy)
        at org.apache.cassandra.db.compaction.CompactionStrategyManager.getNextBackgroundTask(CompactionStrategyManager.java:102)
        - locked <0x00000006467268b8> (a org.apache.cassandra.db.compaction.CompactionStrategyManager)
        at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:258)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
        at org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$4/1671507048.run(Unknown Source)
        at java.lang.Thread.run(Thread.java:748)
      

      As I see it, thread is running a cycle, which has O(n^2) dependency on the number of SSTables, while still holding the lock.

      Say this one:

      "CompactionExecutor:1743" #4644 daemon prio=1 os_prio=4 tid=0x00007f35a01591a0 nid=0x65f1 waiting for monitor entry [0x00007f322d016000]
         java.lang.Thread.State: BLOCKED (on object monitor)
      	at org.apache.cassandra.db.compaction.CompactionStrategyManager.handleNotification(CompactionStrategyManager.java:252)
      	- waiting to lock <0x00000006467268b8> (a org.apache.cassandra.db.compaction.CompactionStrategyManager)
      	at org.apache.cassandra.db.lifecycle.Tracker.notifyDeleting(Tracker.java:448)
      	at org.apache.cassandra.db.lifecycle.LogTransaction.obsoleted(LogTransaction.java:161)
      	at org.apache.cassandra.db.lifecycle.Helpers.prepareForObsoletion(Helpers.java:134)
      	at org.apache.cassandra.db.lifecycle.LifecycleTransaction.doPrepare(LifecycleTransaction.java:199)
      	at org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
      	at org.apache.cassandra.io.sstable.SSTableRewriter.doPrepare(SSTableRewriter.java:376)
      	at org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
      	at org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.doPrepare(CompactionAwareWriter.java:84)
      	at org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
      	at org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.finish(Transactional.java:184)
      	at org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.finish(CompactionAwareWriter.java:94)
      	at org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:206)
      	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
      	at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:89)
      	at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:61)
      	at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:264)
      	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
      	at org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$4/1671507048.run(Unknown Source)
      	at java.lang.Thread.run(Thread.java:748)
      

      Attachments

        1. stack.txt
          231 kB
          Sergey Lapukhov
        2. create.cql
          0.8 kB
          Sergey Lapukhov
        3. CAS-14069.patch
          5 kB
          Sergey Lapukhov
        4. beforePatch.svg
          611 kB
          Sergey Lapukhov
        5. afterPatch.svg
          613 kB
          Sergey Lapukhov

        Issue Links

        Activity

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

          People

            slapukhov Sergey Lapukhov Assign to me
            slapukhov Sergey Lapukhov
            Sergey Lapukhov
            Marcus Eriksson
            Votes:
            0 Vote for this issue
            Watchers:
            8 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment