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

Jonathan Ellis commented on CASSANDRA-1610:
-------------------------------------------

- I think Ben's selection of methods for the CompactionStrategy is an 
improvement, but I do like having an abstract class so it's obvious what the 
contract is for us vs having to inject parameters post-construction.
- I'd like to move away from minor/major terms as too tied to the old 
compaction internals. Perhaps background/maximal instead?
- We should also make user defined compactions part of ACS -- for some 
strategies (e.g. leveldb) we want to be able to reject user requests that would 
break strategy invariants.  Note that this should probably return a single 
Task, rather than a list.  ("Maximal" will also usually return a single task, 
but it's cleaner to represent "nothing to do" as an empty list, than as null.)
- handleInsufficientSpaceForCompaction is a bad encapsulation; it means both it 
and its caller have to deal with "find a place for an sstable."  suggest 
leaving it up to CT.execute to deal with.

Here's what I think ACS should end up looking like with these changes:
{code}
/**
 * Puggable compaction strategy determines how SSTables get merged.
 *
 * There are two main goals:
 *  - perform background compaction constantly as needed; this typically makes 
a tradeoff between
 *    i/o done by compaction, and merging done at read time.
 *  - perform a full (maximum possible) compaction if requested by the user
 */
public abstract class AbstractCompactionStrategy
{
    protected final ColumnFamilyStore cfs;
    protected final Map<String, String> options;

    protected AbstractCompactionStrategy(ColumnFamilyStore cfs, Map<String, 
String> options)
    {
        this.cfs = cfs;
        this.options = options;
    }

    /**
     * @return a list of compaction tasks that should run in the background to 
get the sstable
     * count down to desired parameters.
     * @param gcBefore throw away tombstones older than this
     */
    public abstract List<AbstractCompactionTask> getBackgroundTasks(final int 
gcBefore);

    /**
     * @return the number of background tasks estimated to still be needed for 
this columnfamilystore
     */
    public abstract int getEstimatedRemainingTasks();

    /**
     * @return a list of compaction tasks that should be run to compact this 
columnfamilystore
     * as much as possible.
     * @param gcBefore throw away tombstones older than this
     */
    public abstract List<AbstractCompactionTask> getMaximalTasks(final int 
gcBefore);

    /**
     * @return a compaction task corresponding to the requested sstables
     * @param gcBefore throw away tombstones older than this
     */
    public abstract AbstractCompactionTask 
getUserDefinedTasks(List<SSTableReader> sstables, final int gcBefore);
}
{code}

- Finally, can you update to conform with 
http://wiki.apache.org/cassandra/CodeStyle, especially the part about multiline 
statements?

> 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, 
> 0001-move-compaction-code-into-own-package.patch, 
> 0001-move-compaction-code-into-own-package.patch, 
> 0001-move-compaction-code-into-own-package.patch, 
> 0001-pluggable-compaction.patch, 
> 0002-Pluggable-Compaction-and-Expiration.patch, 
> 0002-pluggable-compaction.patch, 0002-pluggable-compaction.patch, 
> 0002-pluggable-compaction.patch, 0002-pluggable-compaction.patch, 
> 0002-pluggable-compaction.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

Reply via email to