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

Marcus Eriksson commented on CASSANDRA-11035:
---------------------------------------------

I've been working on this a bit and I have something that works pretty well 
now: https://github.com/krummas/cassandra/commits/marcuse/11035

It does the following
* tracks row cardinality using HyperLogLogPlus to be able to better estimate 
how much we will gain by compacting the given sstables together
* uses an off-heap version of stream-lib: 
https://github.com/krummas/stream-lib/commits/marcuse/offheapregisterset
* picks sstable candidates using the following heuristic;
** use the standard STCS bucketing for size (reasoning being that we should 
never compact together a tiny sstable with a huge one, even if the big one 
overlaps the tiny one 100%)
** in each bucket, pick the oldest (by modification date) sstable and add it to 
the compaction set. Pick the oldest one to avoid having sstables getting starved
** * find the sstable in the bucket that gives the best compaction gain if it 
was added to the compaction set
** add that sstable to the compaction set if it improves the total compaction 
gain for the compaction set or if the number of sstables in the compaction set 
is < min_threshold, otherwise we are done
** if size of the compaction set is smaller than {{max_threshold}} goto *

Results in my small tests are promising;
* compaction gain estimation is very accurate, always within 1% of the actual 
result
* I created 10G of highly overlapping data in 100 sstables using a stress 
profile and with autocompaction disabled
** Old STCS takes this down to about 6G
** New STCS takes it down to 3.1G
** Major compaction takes it to 2.8G

should note though that we have a 'perfect' view of the data we are about to 
compact here, this wont be the case when we run with autocompaction enabled 
etc, so we definitely need real-world tests

Todo:
* Submit a pull request for the stream-lib changes - it currently uses a 
finalizer to clear the off heap memory to avoid changing all the ICardinality 
APIs etc, but we could use a version without a finalizer if we think it could 
be a problem
* Find a way to gracefully degrade if the number of sstables is huge - the 
heuristic above runs in O(n^2) where n = number of sstables (or, I guess it 
runs in O(n * max_threshold))
* find an incremental 64bit murmur hash for hashing the clustering values - I 
currently use the old hash value as the seed for the next and it seems to be 
working ok, but I have no idea if it is correct.

> Use cardinality estimation to pick better compaction candidates for STCS 
> (SizeTieredCompactionStrategy)
> -------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-11035
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-11035
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Compaction
>            Reporter: Wei Deng
>            Assignee: Marcus Eriksson
>
> This was initially mentioned in this blog post 
> http://www.datastax.com/dev/blog/improving-compaction-in-cassandra-with-cardinality-estimation
>  but I couldn't find any existing JIRA for it. As stated by [~jbellis], 
> "Potentially even more useful would be using cardinality estimation to pick 
> better compaction candidates. Instead of blindly merging sstables of a 
> similar size a la SizeTieredCompactionStrategy." The L0 STCS in LCS should 
> benefit as well.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to