cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Carl Yeksigian (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-7409) Allow multiple overlapping sstables in L1
Date Thu, 15 Jan 2015 16:08:35 GMT

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

Carl Yeksigian commented on CASSANDRA-7409:
-------------------------------------------

It works in much the same way as was described before; except that:

- When promoting to a level which is not overlapping, the selection will take all overlapping
sstables to compact, rather than just a single one. This is how it was supposed to work initially,
but was actually hitting a different case instead.
- When selecting the level which needs to be compacted, it starts from 0 going up to MOLO
- 1, then from the max level down to MOLO. This means that we try to compact any overlapping
levels first, and if we don't find anything, then we'll compact according to our previous
ways of compacting.

I'm a little suspicious of the MOLO=0 results, now that I've specified the changes; it should
more closely mirror the results I got from LCS w/o STCS.

It is sensitive to the length of the test; L5 allows for as many overlaps as necessary in
this test. The levels shouldn't have any overlap, though; it won't stop compactions until
none of the levels have overlap. The L2 runs slower because it is doing more I/O because of
the non-overlapping.

> 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
>              Labels: compaction
>             Fix For: 3.0
>
>
> 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)

Mime
View raw message