[
https://issues.apache.org/jira/browse/CASSANDRA-2864?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Daniel Doubleday updated CASSANDRA-2864:
----------------------------------------
Attachment: 0001-CASSANDRA-2864-w-out-direct-counter-su.patch
Uups - sorry. First patch did not include cache merging.
> Alternative Row Cache Implementation
> ------------------------------------
>
> Key: CASSANDRA-2864
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2864
> Project: Cassandra
> Issue Type: Improvement
> Components: Core
> Reporter: Daniel Doubleday
> Assignee: Daniel Doubleday
> Labels: cache
> Fix For: 1.2
>
> Attachments: 0001-CASSANDRA-2864-w-out-direct-counter-su.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.
> *Note: Updated Patch Description (Please check history if you're interested
> where this was comming from)*
> h3. Rough Idea
> - Keep serialized row (ByteBuffer) in mem which represents unfiltered but
> collated columns of all ssts but not memtable columns
> - Writes dont affect the cache at all. They go only to the memtables
> - Reads collect columns from memtables and row cache
> - Serialized Row is re-written (merged) with mem tables when flushed
> h3. Some Implementation Details
> h4. Reads
> - Basically the read logic differ from regular uncached reads only in that a
> special CollationController which is deserializing columns from in memory
> bytes
> - In the first version of this cache the serialized in memory format was the
> same as the fs format but test showed that performance sufferd because a lot
> of unnecessary deserialization takes place and that columns seeks are O( n )
> whithin one block
> - To improve on that a different in memory format was used. It splits length
> meta info and data of columns so that the names can be binary searched.
> {noformat}
> ===========================
> Header (24)
> ===========================
> MaxTimestamp: long
> LocalDeletionTime: int
> MarkedForDeleteAt: long
> NumColumns: int
> ===========================
> Column Index (num cols * 12)
> ===========================
> NameOffset: int
> ValueOffset: int
> ValueLength: int
> ===========================
> Column Data
> ===========================
> Name: byte[]
> Value: byte[]
> SerializationFlags: byte
> Misc: ?
> Timestamp: long
> ---------------------------
> Misc Counter Column
> ---------------------------
> TSOfLastDelete: long
> ---------------------------
> Misc Expiring Column
> ---------------------------
> TimeToLive: int
> LocalDeletionTime: int
> ===========================
> {noformat}
> - These rows are read by 2 new column interators which correspond to
> SSTableNamesIterator and SSTableSliceIterator. During filtering only columns
> that actually match are constructed. The searching / skipping is performed on
> the raw ByteBuffer and does not create any objects.
> - A special CollationController is used to access and collate via cache and
> said new iterators. It also supports skipping the cached row by max update
> timestamp
> h4. Writes
> - Writes dont update or invalidate the cache.
> - In CFS.replaceFlushed memtables are merged before the data view is
> switched. I fear that this is killing counters because they would be
> overcounted but my understading of counters is somewhere between weak and
> non-existing. I guess that counters if one wants to support them here would
> need an additional unique local identifier in memory and in serialized cache
> to be able to filter duplicates or something like that.
> {noformat}
> void replaceFlushed(Memtable memtable, SSTableReader sstable)
> {
> if (sstCache.getCapacity() > 0) {
> mergeSSTCache(memtable);
> }
> data.replaceFlushed(memtable, sstable);
> CompactionManager.instance.submitBackground(this);
> }
> {noformat}
> Test Results: See comments below
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators:
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira