Leveled Compaction Strategy (LCS)
The Unified Compaction Strategy (UCS) is the recommended compaction strategy for most workloads starting with Cassandra 5.0. If you are creating new tables, use this strategy. |
The LeveledCompactionStrategy (LCS)
is recommended for read-heavy workloads, although UCS is the best recommendation for all workloads today.
It alleviates some of the read operation issues with STCS, while providing reasonable write operations.
This strategy works with a series of levels, where each level contains a set of SSTables.
When data in memtables is flushed, SSTables are written in the first level (L0), where SSTables are not guaranteed to be non-overlapping.
LCS compaction merges these first SSTables with larger SSTables in level L1.
Each level is by default 10x the size of the previous one.
Once an SSTable is written to L1 or higher, the SSTable is guaranteed to be non-overlapping with other SSTables in the same level.
If a read operation needs to access a row, it will only need to look at one SSTable per level.
To accomplish compaction, all overlapping SSTables are merged into a new SSTable in the next level. For L0 → L1 compactions, we almost always need to include all L1 SSTables since most L0 SSTables cover the full range of partitions. LCS compacts SSTables from one level to the next, writing partitions to fit a defined SSTable size. In addition, each level has a prescribed size, so that compaction will be triggered when a level reaches its size limit. Creating new SSTables in one level can trigger compaction in the next level, and so on, until all levels have been compacted based on the settings.
There is a failsafe if too many SSTables reads are being done in the L0 level. An STCS compaction will be triggered in L0 if there are more than 32 SSTables in L0. This compaction quickly merges SSTables out of L0, and into L1, where they will be compacted to non-overlapping SSTables.
LCS is not as disk hungry as STCS, needing only approximately 10% of disk to execute, but it is more IO and CPU intensive. For ongoing minor compactions in a read-heavy workload, the amount of compaction is reasonable. It is not a good choice for write-heavy workloads, though, because it will cause a lot of disk IO and CPU usage. Major compactions are not recommended for LCS.
Bootstrapping
During bootstrapping, SSTables are streamed from other nodes. Because many SSTables will be both flushed from the new writes to memtables, as well as streaming from a remote note, the new node will have many SSTables in L0. To avoid a collision of the flushing and streaming SSTables, only STCS in L0 is executed until the bootstrapping is complete.
Starved sstables
If the leveling is not optimal, LCS can end up with starved sstables.
High level SSTables can be stranded and not compacted, because SSTables in lower levels are not getting merged and compacted.
For example, this situation can make it impossible for lower levels to drop tombstones.
If these starved SSTables are not resolved within a defined number of compaction rounds, they will be included in other compactions.
This situation generally occurs if a user lowers the sstable_size
setting.
The |
LCS options
Subproperty | Description |
---|---|
enabled |
Enables background compaction. Default value: true |
tombstone_compaction_interval |
The minimum number of seconds after which an SSTable is created before Cassandra considers the SSTable for tombstone compaction.
An SSTable is eligible for tombstone compaction if the table exceeds the |
tombstone_threshold |
The ratio of garbage-collectable tombstones to all contained columns. If the ratio exceeds this limit, Cassandra starts compaction on that table alone, to purge the tombstones. Default value: 0.2 |
unchecked_tombstone_compaction |
If set to |
log_all |
Activates advanced logging for the entire cluster. Default value: false |
sstable_size_in_mb |
The target size for SSTables. Although SSTable sizes should be less or equal to sstable_size_in_mb, it is possible that compaction could produce a larger SSTable during compaction. This occurs when data for a given partition key is exceptionally large. The Cassandra database does not split the data into two SSTables. Default: 160 |
fanout_size |
The target size of levels increases by this |
single_sstable_uplevel |
??? Default: true |
LCS also supports a startup option, -Dcassandra.disable_stcs_in_l0=true
which disables STCS in L0.