You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Carl Yeksigian (JIRA)" <ji...@apache.org> on 2015/01/14 19:22:35 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=14277372#comment-14277372 ] 

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

I've pushed up an updated branch at: https://github.com/carlyeks/cassandra/tree/overlapping-better-selection

The compaction selection has been updated so that it performs much better now. The biggest issue was selecting a single sstable for the overlapping compactions, instead of the that one plus the overlapping ones. It performs much better now.

|| || Operation Time || Compaction Time ||
| MOLO=0 | 3:52:46 | 0:21:04 |
| MOLO=2 | 3:45:52 | 0:37:50 | 
| MOLO=5 | 3:42:59 | 0:03:17 |
| LCS w/ STCS | 3:48:14 | 0:50:24 |
| LCS w/o STCS | 3:50:38 | 1:05:02 |

The performance on spinning disk is also improved by allowing overlapping; here are the results of a read operation after running a large mixed read/write workload: http://cstar.datastax.com/graph?stats=e113f706-9b54-11e4-9f2c-42010af0688f&metric=op_rate&operation=2_user&smoothing=1&show_aggregates=true&xmin=0&xmax=121.88&ymin=0&ymax=113984.2

> 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)