You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Jeff Jirsa (JIRA)" <ji...@apache.org> on 2015/05/30 04:28:17 UTC

[jira] [Updated] (CASSANDRA-8340) Use sstable min timestamp when deciding if an sstable should be included in DTCS compactions

     [ https://issues.apache.org/jira/browse/CASSANDRA-8340?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jeff Jirsa updated CASSANDRA-8340:
----------------------------------
    Attachment: 8340.diff

We have a use case where we're unable to rely on the max timestamp because it CAN BE (in some partitions, but not all) modified to be pathologically high (but never pathologically low), causing the existing filtering code to work as designed, but not in a way that benefits us. While we could write a custom DTCS-like replacement, this proposed improvement  has a nice side effect of also solving our problem, so I've written a simple patch to enable it OPTIONALLY for users who may prefer to filter on {{sstable.getMinTimestamp()}} rather than {{sstable.getMaxTimestamp()}}

The proposed syntax is fairly straight forward. Standard behavior is unchanged:

{noformat}
create table test (id text primary key) 
with compaction={
'max_sstable_age_days': '1', 
'base_time_seconds': '3600', 
'min_threshold': '4', 'class': 
'DateTieredCompactionStrategy'
} ;
{noformat}

Optional min timestamp is available with a boolean flag:

{noformat}
create table test (id text primary key) 
with compaction={
'max_sstable_age_days': '1', 
'base_time_seconds': '3600', 
'min_threshold': '4', 
'class': 'DateTieredCompactionStrategy', 
'use_min_timestamp' : 'true' 
} ;
{noformat}

This decreases risk for existing users, but adds functionality for people who are migrating from STCS and/or have other reasons to prefer getMinTimestamp().

What do you guys think, [~krummas] and [~Bj0rn] ?  

> 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
>         Attachments: 8340.diff
>
>
> 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)