[
https://issues.apache.org/jira/browse/CASSANDRA-2864?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13060982#comment-13060982
]
Jonathan Ellis commented on CASSANDRA-2864:
-------------------------------------------
Thanks for the patch, Daniel.
If I understand correctly, this is not a full row cache per se, but a sort of
merge-cache for sstable data to deal with lots of overwrites (= lots of sstable
fragments to merge). So a cache hit becomes "merge memtable[s] with cached
value."
That's an innovative solution for a problem that is causing real pain. Nice
work.
However, I'm reluctant to add more special cases to the read path. It looks
like the CASSANDRA-2498 + CASSANDRA-2503 approach might offer similar benefits
(that is, at most one -- or a configurable number of -- non-memtable version in
memory), for less complexity as well as a more graceful degradation when your
hot data set doesn't quite fit in memory.
Since you've clearly dug into the read code path more than most, I wonder if
you'd like to take a stab at that?
> Alternative Row Cache Implementation
> ------------------------------------
>
> Key: CASSANDRA-2864
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2864
> Project: Cassandra
> Issue Type: Improvement
> Components: Core
> Affects Versions: 0.8.1
> Reporter: Daniel Doubleday
> Priority: Minor
> Attachments: rowcache.patch
>
>
> we have been working on an alternative implementation to the existing row
> cache(s)
> We have 2 main goals:
> - Decrease memory -> get more rows in the cache without suffering a huge
> performance penalty
> - Reduce gc pressure
> This sounds a lot like we should be using the new serializing cache in 0.8.
> Unfortunately our workload consists of loads of updates which would
> invalidate the cache all the time.
> The second unfortunate thing is that the idea we came up with doesn't fit the
> new cache provider api...
> It looks like this:
> Like the serializing cache we basically only cache the serialized byte
> buffer. we don't serialize the bloom filter and try to do some other minor
> compression tricks (var ints etc not done yet). The main difference is that
> we don't deserialize but use the normal sstable iterators and filters as in
> the regular uncached case.
> So the read path looks like this:
> return filter.collectCollatedColumns(memtable iter, cached row iter)
> The write path is not affected. It does not update the cache
> During flush we merge all memtable updates with the cached rows.
> The attached patch is based on 0.8 branch r1143352
> It does not replace the existing row cache but sits aside it. Theres
> environment switch to choose the implementation. This way it is easy to
> benchmark performance differences.
> -DuseSSTableCache=true enables the alternative cache. It shares its
> configuration with the standard row cache. So the cache capacity is shared.
> We have duplicated a fair amount of code. First we actually refactored the
> existing sstable filter / reader but than decided to minimize dependencies.
> Also this way it is easy to customize serialization for in memory sstable
> rows.
> We have also experimented a little with compaction but since this task at
> this stage is mainly to kick off discussion we wanted to keep things simple.
> But there is certainly room for optimizations.
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira