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

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

Ok, a few comments (pushed 
[here|https://github.com/krummas/cassandra/commits/branimir/7019])

* tombstone source iterator is not closed
* we can never bring in data from the tombstone sources into the new compacted 
sstable (it brings in the partition level deletion) - this could mess up the 
timestamps for DTCS for example
* if we are running with "only_purge_repaired_tombstones" and are compacting 
unrepaired data we can't drop any tombstones

thoughts/questions;
* did you compare performance when dropping overwritten data?
* we should probably run this for a while on a large data set (unless you did 
that already?)
* cell level tombstones are not handled, is that for performance reasons? (do 
we avoid deserializing the row?)
* should we always enable this for the [single-sstable tombstone 
compactions|https://github.com/krummas/cassandra/blob/trunk/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java#L231]?

> 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