> We added a bunch of new nodes to a cluster (2.1.13) and everything went fine, 
> except for the number of pending compactions that is staying quite high on a 
> subset of the new nodes. Over the past 3 days, the pending compactions have 
> never been less than ~130 on such nodes, with peaks of ~200.

When you bootstrap with Vnodes, you end up with thousands (or tens of 
thousands) of sstables – with 256 Vnodes (default) * 20 sstables per node, your 
resulting node will have 5k sstables. It takes quite a while for compaction to 
chew through that. If you added a bunch of nodes in sequence, you’d have 5k on 
the first node, then potentailly 10k on the next, and could potentially keep 
increasing as you start streaming from nodes that have way too many sstables.  
This is one of the reasons that many people who have to grow their clusters 
frequently try not to use vnodes.

>From your other email:

> Also related to this point, now I'm seeing something even more odd: some 
> compactions are way bigger than the size of the column family itself, such as:

The size reported by compactionstats is the uncompressed size – if you’re using 
compression, it’s perfectly reasonable for 30G of data to show up as 118G of 
data during compaction.

- Jeff

From:  Gianluca Borello
Reply-To:  "user@cassandra.apache.org"
Date:  Monday, March 21, 2016 at 12:50 PM
To:  "user@cassandra.apache.org"
Subject:  Pending compactions not going down on some nodes of the cluster

Hi,

We added a bunch of new nodes to a cluster (2.1.13) and everything went fine, 
except for the number of pending compactions that is staying quite high on a 
subset of the new nodes. Over the past 3 days, the pending compactions have 
never been less than ~130 on such nodes, with peaks of ~200. On the other 
nodes, they correctly fluctuate between 0 and ~20, which has been our norm for 
a long time.

We are quite paranoid about pending compactions because in the past such high 
number caused a lot of data being brought in memory during some reads and that 
triggered a chain reaction of full GCs that brought down our cluster, so we try 
to monitor them closely.

Some data points that should let the situation speak for itself:

- We use LCS for all our column families

- The cluster is operating absolutely fine and seems healthy, and every node is 
handling pretty much the same load in terms of reads and writes. Also, these 
nodes with higher pending compactions don't seem in any way performing worse 
than the others

- The pending compactions don't go down even when setting the compaction 
throughput to unlimited for a very long time

- This is the typical output of compactionstats and tpstats:

$ nodetool compactionstats
pending tasks: 137
   compaction type   keyspace            table     completed         total    
unit   progress
        Compaction     draios   message_data60    6111208394    6939536890   
bytes     88.06%
        Compaction     draios    message_data1   26473390790   37243294809   
bytes     71.08%
Active compaction remaining time :        n/a

$ nodetool tpstats
Pool Name                    Active   Pending      Completed   Blocked  All 
time blocked
CounterMutationStage              0         0              0         0          
       0
ReadStage                         1         0      111766844         0          
       0
RequestResponseStage              0         0      244259493         0          
       0
MutationStage                     0         0      163268653         0          
       0
ReadRepairStage                   0         0        8933323         0          
       0
GossipStage                       0         0         363003         0          
       0
CacheCleanupExecutor              0         0              0         0          
       0
AntiEntropyStage                  0         0              0         0          
       0
MigrationStage                    0         0              2         0          
       0
Sampler                           0         0              0         0          
       0
ValidationExecutor                0         0              0         0          
       0
CommitLogArchiver                 0         0              0         0          
       0
MiscStage                         0         0              0         0          
       0
MemtableFlushWriter               0         0          32644         0          
       0
MemtableReclaimMemory             0         0          32644         0          
       0
PendingRangeCalculator            0         0            527         0          
       0
MemtablePostFlush                 0         0          36565         0          
       0
CompactionExecutor                2        70         108621         0          
       0
InternalResponseStage             0         0              0         0          
       0
HintedHandoff                     0         0             10         0          
       0
Native-Transport-Requests         6         0      188996929         0          
   79122

Message type           Dropped
RANGE_SLICE                  0
READ_REPAIR                  0
PAGED_RANGE                  0
BINARY                       0
READ                         0
MUTATION                     0
_TRACE                       0
REQUEST_RESPONSE             0
COUNTER_MUTATION             0

- If I do a nodetool drain on such nodes, and then wait for a while, the number 
of pending compactions stays high even if there are no compactions being 
executed anymore and the node is completely idle: 

$ nodetool compactionstats
pending tasks: 128

- It's also interesting to notice how the compaction in the previous example is 
trying to compact ~37 GB, which is essentially the whole size of the column 
family message_data1 as reported by cfstats:

$ nodetool cfstats -H draios.message_data1
Keyspace: draios
Read Count: 208168
Read Latency: 2.4791508685292647 ms.
Write Count: 502529
Write Latency: 0.20701542000561163 ms.
Pending Flushes: 0
Table: message_data1
SSTable count: 261
SSTables in each level: [43/4, 92/10, 125/100, 0, 0, 0, 0, 0, 0]
Space used (live): 36.98 GB
Space used (total): 36.98 GB
Space used by snapshots (total): 0 bytes
Off heap memory used (total): 36.21 MB
SSTable Compression Ratio: 0.15461126176169512
Number of keys (estimate): 101025
Memtable cell count: 229344
Memtable data size: 82.4 MB
Memtable off heap memory used: 0 bytes
Memtable switch count: 83
Local read count: 208225
Local read latency: 2.479 ms
Local write count: 502581
Local write latency: 0.208 ms
Pending flushes: 0
Bloom filter false positives: 11497
Bloom filter false ratio: 0.04307
Bloom filter space used: 94.97 KB
Bloom filter off heap memory used: 92.93 KB
Index summary off heap memory used: 57.88 KB
Compression metadata off heap memory used: 36.06 MB
Compacted partition minimum bytes: 447 bytes
Compacted partition maximum bytes: 34.48 MB
Compacted partition mean bytes: 1.51 MB
Average live cells per slice (last five minutes): 26.269698643294515
Maximum live cells per slice (last five minutes): 100.0
Average tombstones per slice (last five minutes): 0.0
Maximum tombstones per slice (last five minutes): 0.0

- There are no warnings or errors in the log, even after a clean restart

- Restarting the node doesn't seem to have any effect on the number of pending 
compactions

Any help would be very appreciated.

Thank you for reading


Attachment: smime.p7s
Description: S/MIME cryptographic signature

Reply via email to