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