You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Sylvain Lebresne (JIRA)" <ji...@apache.org> on 2012/06/18 13:53:43 UTC

[jira] [Commented] (CASSANDRA-2864) Alternative Row Cache Implementation

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

Sylvain Lebresne commented on CASSANDRA-2864:
---------------------------------------------

Sorry for taking so long to get to this. But as a consequence, the patch will need some rebase (the hardest part will probably be to account for CASSANDRA-3708). But a few remarks based on the current patch:
* The serialization format for columns seems only marginaly different from our internal one. Maybe it would be worth reusing ColumnSerializer? A priori, this seem to make it more difficult to avoid copying on deserialize, but we could thenuse ByteBufferUtil.inputStream and specialize ByteBufferUtil.read() to recognize that specific input stream and avoid copy.
* Could be worth making it easier to use variable length int encoding (i.e. hardcode less the TypeSizes.NATIVE). Could give a nice benefit.
* This is all serialized in heap. But it would make sense to allow serializing off-heap (Did you experimented with that?). That's even the strength of this idea I think: the in-heap and off-heap cache could be almost identical, except for the ByteBuffer.allocate that would become an allocateDirect in CachedRowSerializer.serialize(). With the big advantage that this off-heap cache wouldn't have to deserialize everything everytime of course.
* What is the point of collectTimeOrderedData in RowCacheCollationController?
* What's the goal of noMergeNecessary in CachedRowSliceIterator? Feels like the merge necessay path is not really much slower than the other one. Or rather, it feels like CachedRowSliceIterator.appendRow() can easily be turn into an iterator, which would pretty much be CachedRowSliceIterator.
* If we're going to replace the current cache path by this patch, we may want to refactor code a bit. For instance, instead of having two collation controllers, we may just want one and have it decide if it uses sstables iterators or the cache iterator based on whether the row is cached.

And some nits:
* In RowCacheCollationController: we don't use underscores in front of variables :)
* There is a few places where the code style is not respected (not a big deal at that point, just mentioning it fyi).
* In CachedRowSerializer, I'd avoid names like deserializeFromSSTableNoColumns.

Now the main problem is counters. As said previously, we will need to be able to distinguish during read between data that has been merged in the cache, and what hasn't been merge yet (the difficulty being to do that during the merge of a memtable). This is probably doable though.

                
> 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