[ 
https://issues.apache.org/jira/browse/CASSANDRA-7019?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15117536#comment-15117536
 ] 

Branimir Lambov commented on CASSANDRA-7019:
--------------------------------------------

Updated [the 
patch|https://github.com/blambov/cassandra/tree/7019-tombstone-compaction], it 
is now ready for review.

Adds a compaction option called {{provide_overlapping_tombstones}} which 
enables the use of tombstones from overlapping sstables in all compactions. 
Adds tests and a benchmark of compaction performance with and without the flag. 
This version will not use sstables whose max timestamp mean they cannot include 
relevant tombstones (which solves the higher-levels in leveled compaction 
problem).

Performance appears to be very acceptable:
{code}
{"class":"org.apache.cassandra.db.compaction.LeveledCompactionStrategy"}
Copy compactions completed in 3.302s
Operations completed in 288.778s, out of which 37.088 for ongoing background 
compactions
At start:            7 tables    455928060 bytes       432373 rows       165634 
deleted rows        65872 tombstone markers
At end:              7 tables    455932906 bytes       432373 rows       165634 
deleted rows        65872 tombstone markers

{"class":"org.apache.cassandra.db.compaction.LeveledCompactionStrategy"}
GC compactions completed in 3.106s
Operations completed in 314.313s, out of which 38.003 for ongoing GC compactions
At start:            7 tables    456105855 bytes       432373 rows       165634 
deleted rows        65872 tombstone markers
At end:              7 tables    362819985 bytes       343661 rows       142714 
deleted rows        63222 tombstone markers

{"max_threshold":"32","min_threshold":"4","class":"org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy"}
Copy compactions completed in 7.924s
Operations completed in 481.763s, out of which 16.680 for ongoing background 
compactions
At start:           11 tables   1387890116 bytes      1318812 rows       358724 
deleted rows       190539 tombstone markers
At end:             11 tables   1387887287 bytes      1318812 rows       358724 
deleted rows       190539 tombstone markers

{"max_threshold":"32","min_threshold":"4","class":"org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy"}
GC compactions completed in 5.551s
Operations completed in 416.796s, out of which 19.210 for ongoing GC background 
compactions
At start:           11 tables   1176568240 bytes      1117556 rows       323352 
deleted rows       178182 tombstone markers
At end:             11 tables    362865780 bytes       343728 rows       142714 
deleted rows        88709 tombstone markers
{code}
Benchmark adds and deletes data in a table, does compaction at predefined 
times, and finally does a one-by-one compaction on all sstables to remove 
deleted data. Turning the flag on does not have a visible effect on the 
performance of the ongoing compactions, and even the GC at the end is not much 
slower than just copying.

Because it removes deleted data, it can cause compaction to run faster with the 
flag enabled in workloads that include deletions.

My plan is create several sub-tickets next:
- To further improve performance by reshuffling {{BigTableScanner}} code to 
implement tombstone iteration using a shared skipping mechanism with slicing.
- To add a GC compaction operation as done by the benchmark to improve the 
state of existing tables.

> Improve tombstone compactions
> -----------------------------
>
>                 Key: CASSANDRA-7019
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7019
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Compaction
>            Reporter: Marcus Eriksson
>            Assignee: Branimir Lambov
>              Labels: compaction
>             Fix For: 3.x
>
>
> When there are no other compactions to do, we trigger a single-sstable 
> compaction if there is more than X% droppable tombstones in the sstable.
> In this ticket we should try to include overlapping sstables in those 
> compactions to be able to actually drop the tombstones. Might only be doable 
> with LCS (with STCS we would probably end up including all sstables)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to