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

2015-06-01 Thread Etienne Adam (JIRA)

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

Etienne Adam commented on CASSANDRA-8340:
-

I didn't know DTCS and read repair were incompatible.. But I can still run 
nodetool repair, isn'it ?

From my understanding of Cassandra, read repair or nodetool repair are 
mandatory to have a sane cluster over time.. Am I wrong ?

 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-v2.diff, 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)


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

2015-06-01 Thread Etienne Adam (JIRA)

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

Etienne Adam commented on CASSANDRA-8340:
-

Marcus, we are having this issue migrating from STCS to DTCS, one big sstable 
was always compacting.

 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-v2.diff, 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)


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

2015-06-01 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-8340:


[~eadam] I guess the old timestamp could be read repair for example, with DTCS 
you should disable read repair

 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-v2.diff, 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)


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

2015-06-01 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-8340:


yes you will still need to run repairs

problem with read repair is that it can bring in old data to a current memtable 
which then gets flushed to disk, this will create sstables with a big timestamp 
range, which can confuse DTCS

 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-v2.diff, 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)


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

2015-06-01 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-8340:


I doubt we should introduce this now - having many hard-to-explain 
configuration options is bad thing.

For your use case, you should probably just run a custom compaction strategy as 
I really hope it is not a common problem. For the people migrating from STCS, 
we should probably build a major compaction for DTCS that splits data based on 
its timestamps, but I have not heard any complaints about issues when migrating 
to DTCS from STCS since DTCS was introduced.

 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-v2.diff, 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)


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

2015-05-29 Thread Jeff Jirsa (JIRA)

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

Jeff Jirsa commented on CASSANDRA-8340:
---

Also: Patch is against 2.1, as that's where our immediate need is, and I hope 
we can get it in (since it doesn't impact anyone who doesn't explicitly ask for 
this change, and it has the potential to help us and others coming from STCS). 
Will provide versions for 2.2 and 3.0 if it's not a clean merge.



 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)


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

2015-05-14 Thread Etienne Adam (JIRA)

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

Etienne Adam commented on CASSANDRA-8340:
-

We have the same issue in 2.1.5, converting a CF from STCS to DTCS: one huge 
sstable keeps getting compacted... The compaction takes more than 10 hours and 
runs very often (at least every day), causing a constant load on all nodes.
I think that having a max_window_size would solve this problem, as this 
sstable would immediately stop compacting, even with a high setting of 1 month.

 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)


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

2015-05-14 Thread JIRA

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

Björn Hegerfors commented on CASSANDRA-8340:


[~eadam] That sounds suspicious. I don't see how that could happen. Not even if 
you uncapped max_sstable_age. The frequency of recompaction of an SSTable under 
normal circumstances decreases exponentially. If the time between the last two 
compactions was 1 day, you should expect the next compaction to happen roughly 
min_threshold days after the latter one.

This is governed by the min timestamps of all SSTables involved. The first 
place I would look is whether your writes come in with the timestamps that DTCS 
expects. For example, if you have configured DTCS with 
timestamp_resolution='MICROSECONDS' (default) and your clients write 
milliseconds, then DTCS will behave as if base_time_seconds was 1000 times its 
set value. That would mean that the size of the incoming window would be 1000 
times larger than intended. And everything in there compacts repeatedly.

An alternative theory to the one above is that you have at one point written 
something with a microsecond timestamp (a manual insert via cqlsh would do 
that), but now some or all of your writes have millisecond timestamps. In that 
case, DTCS will think that every new arriving SSTable has a min timestamp that 
is insanely much older than the newest written value (the highest timestamp 
of microsecond format, which may in reality be old). After all, microsecond 
timestamps have 3 more digits than millisecond timestamps. Then all SSTables 
could very well fall into the same particular time window.

Highly unsynced client clocks (like months of years off) would be another 
variation on this, but on a smaller scale.

Anyway, that seems like abnormal behavior. On one of your nodes, could you take 
a look at the large SSTable and some of the newest SSTables using 
tools/sstablemetadata (I think it shows both max and min timestamp) and make 
sure that all timestamps have the same amount of digits? And that the timestamp 
size matches the timestamp_resolution setting on you table? Do the very newest 
SSTables have a reasonable difference between max and min timestamp? A Memtable 
should probably not have timestamps ranging multiple hours when it's flushed.

 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)


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

2015-05-14 Thread Etienne Adam (JIRA)

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

Etienne Adam commented on CASSANDRA-8340:
-

All timestamps are inserted with microseconds (but not with USING TIMESTAMP, we 
let cassandra set the default current timestamp), we never used milliseconds.

Metadata from the big sstable:
Minimum timestamp: 1418249984079000 (Wed Dec 10 23:19:44 CET 2014
Maximum timestamp: 1431479047084000 (Wed May 13 03:04:07 CEST 2015
Metadata from 2 of smaller, recent sstables:
Minimum timestamp: 1431487800237000 (Wed May 13 05:30:00 CEST 2015)
Maximum timestamp: 1431615168636000 (Thu May 14 16:52:48 CEST 2015)
Minimum timestamp: 1427621080007000 (Sun Mar 29 11:24:40 CEST 2015)
Maximum timestamp: 1431616482164000 (Thu May 14 17:14:42 CEST 2015)

I do not understand how the timestamp 1427621080007000 (March 29) could be 
flushed just now.. We never force timestamp and all nodes are synced with ntp. 
The only difference in the logs are RMI TCP Connection(42243)-192.168.96.31 
instead of the usual NativePoolCleaner when flushing:

INFO  [RMI TCP Connection(42243)-192.168.96.x] 2015-05-14 17:14:42,320 
ColumnFamilyStore.java:877 - Enqueuing flush of xxx: 14940262 (1%) on-heap, 
18910037 (1%) off-heap
INFO  [MemtableFlushWriter:14583] 2015-05-14 17:14:42,803 Memtable.java:378 - 
Completed flushing 
/var/lib/cassandra/data/xxx/xxx-b7d729907fbf11e4ab6615203fafe427/xxx-xxx-ka-101593-Data.db
 (3640771 bytes) for commitlog position ReplayPosition(segmentId=1430810039198, 
position=18118938)

We have a flush rate at about 1 sstable every 2 hours, which does not sound 
elevated given the insertion rate.

 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)


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

2014-11-20 Thread JIRA

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

Björn Hegerfors commented on CASSANDRA-8340:


No drawback, really. It doesn't make a big difference. Whatever is easiest to 
reason about would be best. It's true that in your repair example, it would 
have some effect, but only when the repair SSTables are not older than 
max_sstable_age_days while the big one is. I would imagine that repair would be 
likely to bring in a bunch of files that are older than max_sstable_age_days, 
which will stay scattered anyway.

I suppose using min timestamp would align more with that the rest of the 
strategy uses to determine age. In fact, something that would work even more 
consistently with the strategy would be to specify maximum window size. Perhaps 
in terms of initial window size. We have
* up to min_threshold windows of size 1, followed by
* up to min_threshold windows of size min_threshold, followed by
* up to min_threshold windows of size min_threshold^2, followed by
* up to min_threshold windows of size min_threshold^3, followed by
* etc.

And then we can simply stop generating more windows after some point. The 
simplest, yet perhaps least intuitive, option would be max_window_exponent. 
If we set max_window_exponent=n, then we would stop after windows of size 
min_threshold^n. Example: max_window_exponent=3, min_threshold=4. The last few 
windows would be 64*base_time_seconds in size, no 256 window is every created. 
Other option alternatives are max_window or max_window_seconds.

WDYT [~krummas]?

 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)


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

2014-11-19 Thread JIRA

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

Björn Hegerfors commented on CASSANDRA-8340:


Actually, it's already implemented like that. The SSTables are paired with 
their age in the createSSTableAndMinTimestampPairs method. Max timestamps are 
only used in getNow and filterOldSSTables.

I used min timestamps based on the same reasoning as yours. I agree that major 
compaction is the way to go when switching.

 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)


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

2014-11-19 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-8340:


I meant using max timestamp in filterOldSSTables to avoid compacting the big 
sstable at all

 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)


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

2014-11-19 Thread JIRA

[ 
https://issues.apache.org/jira/browse/CASSANDRA-8340?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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)