You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Alan Liang (JIRA)" <ji...@apache.org> on 2011/06/03 18:56:48 UTC

[jira] [Commented] (CASSANDRA-1610) Pluggable Compaction

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

Alan Liang commented on CASSANDRA-1610:
---------------------------------------

Wanted to add a little bit more context. This ticket now only addresses pluggable compaction only, I've moved the implementation of a timestamp based compaction to https://issues.apache.org/jira/browse/CASSANDRA-2735. 

This patch makes compaction pluggable in the sense that, you can implement your own AbstractCompactionStrategy. An AbstractCompactionStrategy is responsible for selecting the sstables for minor and major compaction. The strategy returns a list of AbstractCompactionTasks that are to be executed by the CompactionManager. These tasks can be regular compaction, expiration of sstables (see #2735), cleanup tasks, etc. For compaction, a strategy returns a list of CompactionTask's.

> Pluggable Compaction
> --------------------
>
>                 Key: CASSANDRA-1610
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1610
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Core
>            Reporter: Chris Goffinet
>            Assignee: Alan Liang
>            Priority: Minor
>              Labels: compaction
>             Fix For: 1.0
>
>         Attachments: 0001-move-compaction-code-into-own-package.patch, 0001-move-compaction-code-into-own-package.patch, 0001-move-compaction-code-into-own-package.patch, 0002-Pluggable-Compaction-and-Expiration.patch, 0002-pluggable-compaction.patch, 0002-pluggable-compaction.patch
>
>
> In CASSANDRA-1608, I proposed some changes on how compaction works. I think it also makes sense to allow the ability to have pluggable compaction per CF. There could be many types of workloads where this makes sense. One example we had at Digg was to completely throw away certain SSTables after N days.
> This ticket addresses making compaction pluggable only.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira