[ 
https://issues.apache.org/jira/browse/CASSANDRA-2864?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Daniel Doubleday updated CASSANDRA-2864:
----------------------------------------

    Description: 
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.

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 names and 
data of columns so that the names portion byte layout is constant and can be 
binary searched. 

{noformat}

===========================
Header (48)                    
===========================
MaxTimestamp:        long  
LocalDeletionTime:   int   
MarkedForDeleteAt:   long  
NumColumns:          int   
===========================
Column Index (num cols * 24)              
===========================
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



  was:
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 compression 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.


    
> 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
>            Priority: Minor
>
> 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.
> 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 names 
> and data of columns so that the names portion byte layout is constant and can 
> be binary searched. 
> {noformat}
> ===========================
> Header (48)                    
> ===========================
> MaxTimestamp:        long  
> LocalDeletionTime:   int   
> MarkedForDeleteAt:   long  
> NumColumns:          int   
> ===========================
> Column Index (num cols * 24)              
> ===========================
> 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