cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Marcus Eriksson (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-7019) Improve tombstone compactions
Date Thu, 05 Feb 2015 15:16:37 GMT

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

Marcus Eriksson commented on CASSANDRA-7019:
--------------------------------------------

What we want is to be able to drop more tombstones by doing a specific tombstone removal compaction.

To be able to drop as many tombstones as possible, we want to include as many overlapping
sstables as we can in this compaction. 

Currently we do this with a single sstable - we find one single sstable, estimate how many
droppable tombstones we have and if more than X% (20 iirc) of all keys in the sstables are
droppable tombstones, we trigger a single sstable compaction including that. This is often
quite ineffective as the tombstones can cover data in other sstables.

Start by reading up on SizeTieredCompactionStrategy#worthDroppingTombstones()

So, we need to
# Find a good candidate sstable
# Include all sstables that overlap that sstable and contain older data (a tombstone can only
cover older data in other sstables)
# Start a compaction
# Figure out a good way to write out the data to disk (for STCS for example, all sstables
might overlap eachother, which would cause a major compaction, for LCS we need to distribute
the result in the leveled hierarchy somehow). This is the trickiest part of the ticket. One
way I've though about is to track which sstable the data is coming from and map each input
sstable to an output sstable, and write all non-tombstone data to those. The result would
be the same number of input sstables, minus tombstones (and any covered data)

> Improve tombstone compactions
> -----------------------------
>
>                 Key: CASSANDRA-7019
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7019
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Marcus Eriksson
>            Assignee: Branimir Lambov
>              Labels: compaction
>             Fix For: 3.0
>
>
> 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)

Mime
View raw message