[ 
https://issues.apache.org/jira/browse/CASSANDRA-14069?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sergey Lapukhov updated CASSANDRA-14069:
----------------------------------------
    Description: 
Cluster was flooded with SSTables. A table had ~20000 sstables. Write requests 
started failing. 

Steps to reproduce:
* Create cluster with 3 nodes
* Specify 
   {noformat}
memtable_heap_space_in_mb: 10
{noformat}
 in cassandra.yaml
* Create table standard1 in keyspace1 (for the cassandra-stress tool) with the 
script [^create.cql]. Please note
{noformat} compaction = {'class': 'SizeTieredCompactionStrategy', 'enabled': 
'false'} {noformat}
  i.e. compaction will be turned off for now.
*  Populate node with data:
  {noformat} cassandra-stress write n=1000000000 -node 127.0.0.1 {noformat}
* After node was populated, put both read and write pressure on it:
  {noformat}  cassandra-stress read n=1000000000 -node 127.0.0.1 
  cassandra-stress write n=1000000000 -node 127.0.0.1 {noformat}
* While still under pressure, enable LeveledCompactionStrategy
{code:java}  echo "ALTER TABLE keyspace1.standard1 WITH compaction = { 'class' 
: 'LeveledCompactionStrategy', 'sstable_size_in_mb' : 1 }; DESC 
keyspace1.standard1; exit" | bin/cqlsh; {code}

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

If issued from another node:

{noformat}
 $ bin/nodetool cfstats
...
Table: standard1
                SSTable count: 22637
                SSTables in each level: [22651/4, 0, 0, 0, 0, 0, 0, 0, 0] 
...
{noformat}


{noformat}
$ 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% 
{noformat}

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 %).

The root of the problem maybe in this thread (see [^stack.txt]):

{noformat}
"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)
{noformat}

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. 

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

Steps to reproduce:
* Create cluster with 3 nodes
* Specify 
   {noformat}
memtable_heap_space_in_mb: 10
{noformat}
 in cassandra.yaml
* Create table standard1 in keyspace1 (for the cassandra-stress tool) with the 
script [^create.cql]. Please note
{noformat} compaction = {'class': 'SizeTieredCompactionStrategy', 'enabled': 
'false'} {noformat}
  i.e. node will not perform compaction.
*  Populate node with data:
  {noformat} cassandra-stress write n=1000000000 -node 127.0.0.1 {noformat}
* After node was populated, put both read and write pressure on it:
  {noformat}  cassandra-stress read n=1000000000 -node 127.0.0.1 
  cassandra-stress write n=1000000000 -node 127.0.0.1 {noformat}
* While still under pressure, enable LeveledCompactionStrategy
{code:java}  echo "ALTER TABLE keyspace1.standard1 WITH compaction = { 'class' 
: 'LeveledCompactionStrategy', 'sstable_size_in_mb' : 1 }; DESC 
keyspace1.standard1; exit" | bin/cqlsh; {code}

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

If issued from another node:

{noformat}
 $ bin/nodetool cfstats
...
Table: standard1
                SSTable count: 22637
                SSTables in each level: [22651/4, 0, 0, 0, 0, 0, 0, 0, 0] 
...
{noformat}


{noformat}
$ 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% 
{noformat}

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 %).

The root of the problem maybe in this thread (see [^stack.txt]):

{noformat}
"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)
{noformat}

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. 


> Node stopped serving write requests when a table has a lot of sstables
> ----------------------------------------------------------------------
>
>                 Key: CASSANDRA-14069
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-14069
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Sergey Lapukhov
>         Attachments: create.cql, stack.txt
>
>
> Cluster was flooded with SSTables. A table had ~20000 sstables. Write 
> requests started failing. 
> Steps to reproduce:
> * Create cluster with 3 nodes
> * Specify 
>    {noformat}
> memtable_heap_space_in_mb: 10
> {noformat}
>  in cassandra.yaml
> * Create table standard1 in keyspace1 (for the cassandra-stress tool) with 
> the script [^create.cql]. Please note
> {noformat} compaction = {'class': 'SizeTieredCompactionStrategy', 'enabled': 
> 'false'} {noformat}
>   i.e. compaction will be turned off for now.
> *  Populate node with data:
>   {noformat} cassandra-stress write n=1000000000 -node 127.0.0.1 {noformat}
> * After node was populated, put both read and write pressure on it:
>   {noformat}  cassandra-stress read n=1000000000 -node 127.0.0.1 
>   cassandra-stress write n=1000000000 -node 127.0.0.1 {noformat}
> * While still under pressure, enable LeveledCompactionStrategy
> {code:java}  echo "ALTER TABLE keyspace1.standard1 WITH compaction = { 
> 'class' : 'LeveledCompactionStrategy', 'sstable_size_in_mb' : 1 }; DESC 
> keyspace1.standard1; exit" | bin/cqlsh; {code}
> *Results:*
> Write requests failing, read requests still processed.
> 'bin/nodetool cfstats' and 'bin/nodetool compactionstats' commands hanging, 
> if issued from the node running cassandra-stress tool.
> If issued from another node:
> {noformat}
>  $ bin/nodetool cfstats
> ...
> Table: standard1
>                 SSTable count: 22637
>                 SSTables in each level: [22651/4, 0, 0, 0, 0, 0, 0, 0, 0] 
> ...
> {noformat}
> {noformat}
> $ 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% 
> {noformat}
> 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 %).
> The root of the problem maybe in this thread (see [^stack.txt]):
> {noformat}
> "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)
> {noformat}
> 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. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to