[
https://issues.apache.org/jira/browse/CASSANDRA-1610?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13033607#comment-13033607
]
Stu Hood commented on CASSANDRA-1610:
-------------------------------------
bq. The suggested alternative compaction strategies don't sound very generally
useful. We shouldn't maintain them in-tree.
Time bucketing and expiration (as implemented here) are very, very useful for
timeseries data, and are in fact a blocker for production use of our timeseries
systems. The requirement is that column families which store events at varying
resolutions need to decay at different rates: there is no point keeping minute
level resolution data indefinitely. Additionally, using TTLs is much, much too
fine grained, and requires extra storage for each value.
bq. We shouldn't have any provider-specific logic left in CompactionManager
itself, it should all be based on the pluggable Strategy.
Agreed, but one approach that I think would be good middle ground would be to
move doCompaction and doExpiration onto implementations of an
AbstractCompactionTask, to be returned by the strategies that Alan has
implemented. The 'task' concept already exists in this patch as an enum that
CompactionManager switches on.
The interesting methods on CompactionStrategy are
selectFor(Minor|Major)Compaction, which calculate the possible tasks to perform
during a minor or major compaction. For the SizeTieredStrategy (aka, the
strategy implemented in trunk), selectForMinor is the same as the previous
getBuckets method.
----
The configuration changes in the patch are distracting, but it boils down to:
# Record the max client timestamp for sstables (useful for CASSANDRA-2498)
# Allow for a compaction strategy to choose which files to perform a particular
"task" on (bucketing)
# Implement a "task" for expiration (N files in, 0 files out)
# Add a strategy that uses the max client timestamp to expire old files
> 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,
> 0002-Pluggable-Compaction-and-Expiration.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.
> The goal of this ticket is to make compaction pluggable enough to support
> compaction based on max timestamp ordering of the sstables while satisfying
> max sstable size, min and max compaction thresholds. Another goal is to allow
> expiration of sstables based on a timestamp.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira