You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Marcus Eriksson (JIRA)" <ji...@apache.org> on 2014/09/03 14:45:52 UTC

[jira] [Commented] (CASSANDRA-7409) Allow multiple overlapping sstables in L1

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

Marcus Eriksson commented on CASSANDRA-7409:
--------------------------------------------

I think the within-level compaction could be reworked a bit, now we find which sstables are overlapping the most, and compact those into the next level.

Since sstables that overlap a lot are bound to contain the same keys, it means we are compacting hot partitions into the next level, where it is likely to be overlapping with a bunch of sstables again causing another compaction into the next level.

If we instead try to guess how much overlap a set of sstables would cause in the next level, and pick the ones that would cause the *least* overlap, i think we could save ourselves a bunch of write amplification by keeping the hot keys low in the leveling.

I guess we need to have a threshold that the most overlapping sstables should eventually be compacted together as well, but maybe they should stay in the same level they are?

Approach (say MOLO = 2);
* L0 -> L1 is kept the same, flushed L0 sstables are bound to overlap everything in L1 anyway
* Find a set of sstables in L1 that would create the least overlap in L2 and compact those together
* If we have "many" (4?) highly overlapping sstables in L1, compact those together, but make them stay in L1

We probably need a better metric for overlappiness here as well (CASSANDRA-6474 or by using the HyperLogLog component in CompactionMetadata)

WDYT, does it make sense? Very likely I'm overthinking this and due to the fact that each sstable contain a wide range of keys (atleast in L1) it might not make a difference.


> Allow multiple overlapping sstables in L1
> -----------------------------------------
>
>                 Key: CASSANDRA-7409
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7409
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Carl Yeksigian
>            Assignee: Carl Yeksigian
>
> Currently, when a normal L0 compaction takes place (not STCS), we take up to MAX_COMPACTING_L0 L0 sstables and all of the overlapping L1 sstables and compact them together. If we didn't have to deal with the overlapping L1 tables, we could compact a higher number of L0 sstables together into a set of non-overlapping L1 sstables.
> This could be done by delaying the invariant that L1 has no overlapping sstables. Going from L1 to L2, we would be compacting fewer sstables together which overlap.
> When reading, we will not have the same one sstable per level (except L0) guarantee, but this can be bounded (once we have too many sets of sstables, either compact them back into the same level, or compact them up to the next level).
> This could be generalized to allow any level to be the maximum for this overlapping strategy.



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