cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Björn Hegerfors (JIRA) <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-8340) Use sstable min timestamp when deciding if an sstable should be included in DTCS compactions
Date Wed, 19 Nov 2014 22:16:33 GMT

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

Björn Hegerfors commented on CASSANDRA-8340:
--------------------------------------------

OK, let's see. This is a big SSTable with a timestamp span of [t0, t1]. Since it came out
of a major compaction, t1 is close to the current time. DTCS would never generate an SSTable
that large with t1 that close to current time. But as time passes, [t0, t1] eventually becomes
a timestamp span that even DTCS could have generated. Only beyond that point in time would
DTCS actually consider compacting it, because it's t0 that governs when it compacts next,
not t1. This is because t0 is so old and so far away from the min timestamp of any other SSTable.
I'm certain of this. I haven't got a formula for this (I wish to make one), but I think that
the major compacted SSTable may even have to double its age before next compaction will happen,
so if the min timestamp was older than max_sstable_age_days when switching strategies, the
max timestamp will be older than that before any compaction was ever considered.

In other words, your scenario is not in any way a particular reason to change the max_sstable_age_days
behavior. There may still be other reasons.

Did you get that? I had a hard time figuring out a sensible way to formulate my reasoning
here. Rewrote this 3 times :P

> Use sstable min timestamp when deciding if an sstable should be included in DTCS compactions
> --------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-8340
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8340
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Marcus Eriksson
>            Priority: Minor
>
> Currently we check how old the newest data (max timestamp) in an sstable is when we check
if it should be compacted.
> If we instead switch to using min timestamp for this we have a pretty clean migration
path from STCS/LCS to DTCS. 
> My thinking is that before migrating, the user does a major compaction, which creates
a huge sstable containing all data, with min timestamp very far back in time, then switching
to DTCS, we will have a big sstable that we never compact (ie, min timestamp of this big sstable
is before max_sstable_age_days), and all newer data will be after that, and that new data
will be properly compacted
> WDYT [~Bj0rn] ?



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

Mime
View raw message