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

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

I've pushed a few modifications over Daniel's patch at 
https://github.com/pcmanus/cassandra/commits/2864-1.

The first change is the idea of reusing the ColumnSerializer instead of 
serializing columns manually. I actually think that this simplify the patch 
quite a bit and even slightly decrease the serialized size (by 2 bytes per 
columns) compared to the original patch. I'll note that this version don't do 
more copying or basically more work than the original version so while I 
haven't made any proper performance comparison, there shouldn't be any 
noticeable difference.

I've also refactor the code a little bit to move all that code into the 
CachedRow class directly, which avoids having loads of static method and is imo 
a little bit cleaner. It also allowed to move most of what was in the header 
(namely the maxTimestamp and column count) from the serialized format into 
fields of CachedRow. I think this is a bit cleaner and since we typically 
access the columnCount often, I don't see a point to deserialize it each time.  
And since those are primitive types, having them as fields of CachedRow don't 
change the number of object we have in memory nor the size occupied (at least 
not noticeably).

The patches also add support for #3708, which was fairly since, being in 
memory, it just serialize the whole deletionInfo which is likely good enough 
for the forseable future.

I did also refactor the slice iterator a bit, which should be faster because we 
do much less comparison.

Also, turned out that merging the two collationIterator is not so easy, but 
I've still extracted the common parts into an AbstractCollationController class.

I didn't start work on deailing with counters yet however. I think the idea of 
using the SnapTree copy-on-write facilities is a good idea. But since Daniel 
you said you've starting lookding at it, I don't wan to duplicate efforts. So 
are you still working on that? If you've started but don't have much time right 
now, feel free to put what you've started on github and I'll tried advance it. 
Could be worth at least seeing with Ben if he is still up for making the 
backing map of CLHM configurable, I don't want to maintain a fork of the thing.
                
> 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-support.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

        

Reply via email to