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

2012-08-02 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-2864:
-

I looked a bit into doing this 'skip the cache on read' for counters, but I 
realized that there is some complications.

The way the current patch works, when we do a read through cache and there is 
nothing cached, we do a "normal" read and cache the result. And by normal read, 
I mean reading in all the memtables in particular. Which, I think, has 2 
problems:
* this is racy, even in the non counter case. Suppose we start a read. Maybe 
the current memtable is empty and we start reading the sstables. While that 
happens, you could have updates coming in the memtable and, if you are unlucky, 
have this memtable flushed almost right away. If it gets merged to the cache 
before our read finishes and cache it's result, we have a problem similar to 
CASSANDRA-3862. Granted this is very unlikely but it is possible, at least in 
theory. Which means we probably need to have some sentinel business even with 
this patch to be safe.
* this doesn't work for counters at all, because that means that when we cache 
stuff, we have all the data that is currently in any current memtable. It means 
that for counters, the read-to-cache should only read from the sstables, not 
the memtables. But now that racy with flush and become quite subtle to 
synchronize correctly. It's probably doable but makes me wonder if keeping the 
current code path for counter is not way simpler.

> 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, 
> rowcache-with-snaptree-sketch.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 C

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

2012-07-20 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

bq. or to skip cache during the memtable merge for counters

Just a thought: it might be easy enough to only skip the cache if the row is in 
one of the memtables. As in tryCache. When the controller reads a CF from a 
memtable it bails out and the read could be re-performed uncached

> 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, 
> rowcache-with-snaptree-sketch.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,

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

2012-07-10 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-2864:
-

bq. I modified the stuff I did so that it applies on your branch.

Thanks.

bq. But I honestly doubt that this is going somewhere.

I agree.

I guess so far the simple options are probably to either keep the current cache 
path in for counters, or to skip cache during the memtable merge for counters. 
I'll have to think a bit more about this, weighting the pros and cons, or 
trying to come up with a better option.

> 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, 
> rowcache-with-snaptree-sketch.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);
> }
> {no

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

2012-07-10 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

Hmokay.

I modified the stuff I did so that it applies on your branch. But I honestly 
doubt that this is going somewhere.

It's a really rough sketch to see if a slightly modified CLHM will work. At 
least in the limited time I spent on this: I dont think so. 
I didn't actually removed the cache switch for the counter case. I just pushed 
it so far that I could do some basic tests. 

As said before its fragile and ugly.

Rough idea is:

- Use a snap tree container (SnapTreeHolder) that contains the current and in 
case of an ongoing merge future version of the snaptree
- Make sure that only one merge is performed
- Get a view of the cache (SnapTreeHolder.State) and add it to ViewFragment to 
make sure that reads get a consistent snapshot of memtables and cache
- Use a thread local (ViewContext) to get hold of that snapshot from within the 
cache (cough - that was the only hack I could think of without changing CLHM)

Cache misses during merge don't get applied to the cache during merge for all 
CFs (not only the merging one). This could be optimized in a way the other CFs 
cache misses are applied to both snaptrees or only the future tree which should 
be used for reads than. 

Well in short there are many optimizations possible but the main problem is: 
without changing CLHM in a more fundamental way I don't think it makes sense at 
all.



> 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, 
> rowcache-with-snaptree-sketch.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 CollationControl

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

2012-07-09 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

Sorry was away ... Just had a quick look and changes look great so far.

As for my experiments: not sure if what I have is really helpful. I did try to 
get something going without mayor modifications to CLHM by only allowing to 
specify the backing map in the builder. I havent't finished integration but I 
can already see that even if it works it will be super fragile because the 
implementation relies on the the way CLHM uses the backing map and will fall 
apart as soon as some implementation detail will change.

My gut feeling is that if you want to take that route it would be probably 
cleaner to fork and specialize CLHM instead of using it as a lib. It's 
basically one file after all ...

Another thing you might consider (although it smells a little like surrender):

A little test showed me that in mem serialization of 1M columns took around 1s. 
Maybe it would be good enough to either
- block on read while merging (after all gc pauses can be worse) (but I dont 
think that this is a good idea)
- just omit the cache entirely for counters while merging (just lose a couple 
of cache hits for 1..2 secs)

That way we wouldn't need to mess with the cache at all ...

Anyways: I'll attach my stuff later when I get home (dont have it here) and you 
can see if there's anything in it for you

> 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 

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

2012-07-05 Thread Sylvain Lebresne (JIRA)

[ 
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 

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

2012-06-29 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-2864:
-

bq. Cache merging should be pretty fast so it might be an acceptable 
inefficiency

Hum, I though initially you wanted to do the merge of the memtable with the 
cache during the flush, but I guess if you do it in a separate iteration after 
the flush then yes, cache merging might be fast enough. 

> 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 a

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

2012-06-29 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

bq. One of the slight complication is that when reading to cache we'd have to 
add the new read to both the cache and the clone and synchronize that properly.

Actually I thought that we might be able to live with losing those reads. Cache 
merging should be pretty fast so it might be an acceptable inefficiency

bq. I'm not looking forward to maintaining a fork of CLHM backed by SnapTree 
either

That's a bummer yes. But some time ago I asked if the backing map could be made 
configurable via the builder and it seems that Ben would consider extensions: 
http://code.google.com/p/concurrentlinkedhashmap/issues/detail?id=26 (I wanted 
to cache partial rows)

> 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 (

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

2012-06-29 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2864:
---

Size of heap also affects young gen GC, since old-gen references have to be 
treated as roots for the young gen collection: 
http://blog.ragozin.info/2011/06/understanding-gc-pauses-in-jvm-hotspots.html

> 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 i

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

2012-06-29 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-2864:
-

bq. What I meant to say is that you will have one object per row in both cases: 
it's either the on-heap or the off-heap structure

Oh, ok, I though you were comparing with the current non-serializing cache, my 
bad. But yeah, the advantage of being off-heap is that you can big caches 
without having a big heap, and for the old gen (more precisely the compaction 
part), the sheer size of the heap does matter if I'm not mistaken.

bq. I started experimenting with a patched version of CLHM backed by a SnapTree 
to get snapshot capabilities in the cache.

Yeah that's probably a solution. One of the slight complication is that when 
reading to cache we'd have to add the new read to both the cache and the clone 
and synchronize that properly.

I have though of another solution though who would basically consist in keeping 
some cursor of where we are of the currently flushed memtable to basically 
allow read to tell if a given row in the mt we flush has been merge to the 
cache already or not. However the synchronization of this wouldn't necessarily 
be much easier and handling things like range iterators might be a pain. On the 
other side, I'm not looking forward to maintaining a fork of CLHM backed by 
SnapTree either, so overall I'm not sure yet what is the less painful solution 
:)

> 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 

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

2012-06-29 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

bq. I'm not sure I follow, while serialized we'll just have one ByteBuffer 
per-row, won't we?

Yes. What I meant to say is that you will have one object per row in both 
cases: it's either the on-heap or the off-heap structure. The 'old' serializing 
cache had the added advantage of reducing the number of objects compared to 
CLHM. But this wont be case here

bq. That, I'm not sure what makes you thing that. It certainly shouldn't be the 
case and at least on some basic tests it works as it should

My bad. I was misguided by the getName() of RangeTombstone and missed the 
serialization logic in ColumnIndex

bq. That's ok, I'll try to add support for CASSANDRA-3708 and for counters.

Cool.

I started experimenting with a patched version of CLHM backed by a SnapTree to 
get snapshot capabilities in the cache.
I thought of something like:

- get switch lock
- clone cache
- merge memtables in cloned cache
- add sstable to view
- switch cache

Im memory size of SnapTree doesn't seem to differ a lot from ConcurrentHashMap 
but it seems to a little slower (quick test was around 25% slower).

But maybe you have a better solution ...

> 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 da

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

2012-06-29 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-2864:
-

bq. The main thing is that searching should not copy bytes or construct columns 
while comparing column names.

On a first quick read it looked like this would be reasonably simple to do but 
maybe you're right. I'll have a closer look.

bq. But the point is that you wont have less objects in this case.

I'm not sure I follow, while serialized we'll just have one ByteBuffer per-row, 
won't we?

bq. it seems that name queries will not return a range tomb stone [1..3] when 
asked for column 2

You're right, that's a bug and I've opened CASSANDRA-4395 to fix it.

bq. Also a range tomb stone [1..3] seems to overwrite [1..4]

That, I'm not sure what makes you thing that. It certainly shouldn't be the 
case and at least on some basic tests it works as it should.

bq. My time is pretty limited unfortunately so Sylvain if you want to take it 
from here ...

That's ok, I'll try to add support for CASSANDRA-3708 and for counters.

> 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 want

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

2012-06-25 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

So - same here - so mutch to do, so little time ...

bq. The serialization format for columns seems only marginaly different from 
our internal one. Maybe it would be worth reusing ColumnSerializer

Yes I guess it would make the code a bit cleaner. CachedRow could become a 
DataInput and BBU can special case read(). The main thing is that searching 
should not copy bytes or construct columns while comparing column names. At 
least my test showed my that this is where I lost most of the performance. You 
will add a byte or so in serialized size because the column index needs to be 
fixed width and offset information must be an int and is redundant if you use 
standard serialization.

bq. it would make sense to allow serializing off-heap

I thought about that but did not experiment. Actually I'm not so sure that it 
will make sense because the main idea of off-heap serialization here is 
reducing GC pressure. But the point is that you wont have less objects in this 
case. So it would only help when the ByteBuffers cause fragmentation problems. 
Since malloc is not a silver bullet either well ... But it should be so ease to 
implement that experimenting wouldn't hurt

bq. What is the point of collectTimeOrderedData in RowCacheCollationController

I had a test with many writes which showed that the overhead of testing the 
cached row for a column that is superseded by a memtable value is significant 
and I wanted to avoid that in this case. And at that point I still hoped that I 
can support counters out of the box.

bq. What's the goal of noMergeNecessary

Removed. 

bq. instead of having two collation controllers

Definitely. It was just easier to maintain for us as a patch

Now: I updated the patch.

- Should apply cleanly on trunk
- Support for CASSANDRA-3885 (untested)
- No support counters
- No support for CASSANDRA-3708

I tried to update the patch for CASSANDRA-3708 but failed because I couldn't 
get a firm understanding in the limited time. I.e. it seems that name queries 
will not return a range tomb stone [1..3] when asked for column 2. Also a range 
tomb stone [1..3] seems to overwrite [1..4]. Both seems strange but might be a 
misunderstanding or it's supposed to work that way.

In short: I gave up on this one for the time being.

My time is pretty limited unfortunately so Sylvain if you want to take it from 
here ...
Otherwise I would need some support regarding CASSANDRA-3708.






> 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   
> ===
> C

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

2012-06-18 Thread Sylvain Lebresne (JIRA)

[ 
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 col

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

2012-05-10 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

bq. If you could use the occasion to squash/regroup the patches too, that would 
be awesome

Yeah sure. I wont admit my lack of git voodoo powers publicly but due to the 
new global cache design in > 1.0 I have to port rather than rebase anyway. And 
I'll try to make it a more digestible patch

bq. For counters, we need to ensure that the same column is not read twice.

I was thinking that since this is only a local problem it might be possible / 
easier to just serialize the System.identityHashCode of the counter column in 
the serialized cached row and filter during collation but I would rather port 
the patch and let you have a look first before getting into that.


> 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
>
>
> 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

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

2012-05-10 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2864:
---

This is going to be an entirely different code path though, not just a new 
CacheProvider.  (So we'd still have SCP and CLHCP, but they'd both use this new 
design.)  I'd like to drop the old code path entirely instead of leaving both 
around, if we can make that work.

> 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
>
>
> 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] [Commented] (CASSANDRA-2864) Alternative Row Cache Implementation

2012-05-10 Thread Sylvain Lebresne (JIRA)

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

Sylvain Lebresne commented on CASSANDRA-2864:
-

If you could use the occasion to squash/regroup the patches too, that would be 
awesome :D

For counters, we need to ensure that the same column is not read twice. I 
haven't really looked at the patches yet, but a priori this means that for a 
flushed memtable mt, "merging mt into the cache" and "stop merging mt content 
to the cache on reads" would have to happen atomically, which will probably be 
tricky to do efficiently (it's probably possible to do this row-per-row with a 
row cursor on the memtable that is being merged to the cache. We would 1) 
read/deserialize the row from the cache and merge it to the row in the memtable 
and then atomically "add the new content to the cache" and "move the memtable 
cursor to the next row", and reads would only merge the cache with those rows 
that are after the cursor; but anyway, this is way more complicated that what 
we want for a first version here).

In any case, given that we currently only support whole CF of counters, the 
trivial solution is to disallow this new alternative cache for said CF.

> 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
>
>
> 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 
>

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

2012-05-10 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

bq. Can you rebase to 1.2?

Yes. Will do.

> 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
>
>
> 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




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

2012-05-08 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2864:
---

Okay, so this is complementary wrt to CASSANDRA-1956 -- 1956 addresses caching 
different kinds of queries, and this is strictly about not throwing away a 
[serialized] cache row in the face of updates.

+1 from me in theory.  Can you rebase to 1.2?

Will need Sylvain's input on counters, 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
>Priority: Minor
>  Labels: cache
> Fix For: 1.2
>
>
> 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/C

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

2012-04-18 Thread Vijay (Commented) (JIRA)

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

Vijay commented on CASSANDRA-2864:
--

Wrote comments thinking it was a diffrent ticket hence removed the comments...

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

--
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




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

2012-04-18 Thread Vijay (Commented) (JIRA)

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

Vijay commented on CASSANDRA-2864:
--

Hi Jonathan, When there is a write for X3 we invalidate/update the cache and 
the next fetch does the FS scan and populates the cache after it is out of the 
cache (it is similar to the page cache and if there is a write on the block the 
whole block is marked dirty and next fetch will go to the FS). there is a 
configurable block size when set high enough will cache the whole row (like the 
existing cache). The logic around it is kind of what the patch has

>>> I think you might need to write that book, because the commit history is 
>>> tough to follow
Yeah just wrote a prototype hence... :) I can it up if we agree on that 
approach.

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

--
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




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

2012-04-18 Thread Jonathan Ellis (Commented) (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2864:
---

I think you might need to write that book, because the commit history is tough 
to follow. :)

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

--
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




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

2012-04-18 Thread Jonathan Ellis (Commented) (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2864:
---

If so, how do you avoid scanning the sstables?  Does this only work on 
named-column queries?  That is, if I ask for a slice from X to Y, if you have 
data in your cache for X1 X2, how do you know there is not also an X3 on disk 
somewhere?

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

--
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




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

2012-04-18 Thread Jonathan Ellis (Commented) (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2864:
---

Is the original description here still an accurate guide to the approach taken?

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

--
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




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

2012-03-29 Thread Daniel Doubleday (Commented) (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

Removed old obsolete patch to prevent confusion


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

--
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




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

2012-03-29 Thread Daniel Doubleday (Commented) (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

So ... some final remarks

We have this in production and it's looking good so far.
Our cached real world cfs are pretty skinny so far so the reduction in memsize 
is only ~ 3.5 - 4x.

Latency wise there's no difference (compared to CLHC) keeping the max number of 
items equal. So the improvement comes from being able to keep more rows in 
memory and therefor increase hit ratio or leave more mem for page cache.

If there's any interest in this: the fork we are running lives here:

https://github.com/Smeet/cassandra/tree/cassandra-1.0

This is still work in progress which works for us (counters and supercolumns 
are untested) and allows to switch implementation via startup params.

Again the intention of this patch is to replace both CLHC and SC. 
Silvain expressed concerns that this might not work for counters. Since we 
don't use them I didn't bother to much (at least until it's clear whether this 
is interesting for you or not)

Next step for me is to port to 1.1 and look at the key cache.

Please comment if you want to follow up or close otherwise.

Thanks

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

--
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




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

2012-02-20 Thread Daniel Doubleday (Commented) (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

Ok for the curious I just wanted to report some findings 

Disclaimer: I ignored counters and super cols for the time being.

I did some testing on various machines with different CPU / Mem profiles.
I tried different read / write / overwrite scenarios with reads being 
normal-distributed so I could configure cache hit ratios while keeping 
everything else constant.
I also tried to test the impact of different io loads by controlled stressing 
of discs.
One of my initial mayor concerns was also memory footprint: how much rows can 
we fit into memory without getting in real trouble with gc.

Hm results are well ... difficult. In a way I tend to think that we (or maybe 
its just me) are looking in the wrong direction. Right now I believe that in 
the long run caching doesn't make sense at all but for now I just want to 
report some figures:

After the first real testing results looked ambivalent:

# The alternative cache is way superior in terms fo memory usage and gc. In 
general I found that I can put around 10x rows in the cache.
# On the other side performance on rather cpu restricted machines was worse 
than I hoped. In general it didn't really made a big difference whether I was 
using the cache or had only a few memtables fully cached in page cache

Since this sucked I looked where all that cpu was burned and decided to change 
the serialized row format and write custom name and slice filters. I figured 
that the problem was that lots of objects are deserialized right now and theres 
to much search scanning going on.

So now a row in mem loks like that:

|| Header || Column Data ||

Column offsets are encoded in the header. This way I can do binary searches and 
don't need to scan.
Also the filters only ever deserialize anything when it's really returned as 
relevant column.

Before I write a book... Below are some figures. These are only ment to give a 
broad idea. The total performance numbers dont mean anything. This was a 4-core 
server with the tester threads running on the same machine. Machine was CPU 
bound in all tests.

CPU bound? Yes - right now I still can't deliver anything really conclusive in 
terms of what all this means for throughput (other than that I think caching is 
the wrong answer). It's all about isolated cache performance so far.

h2. Memory Footprint

Note, the memory vals are from JProfiler. I'm not sure if they are bullet proof 
but should be in the right ball park.
Payload estimate derived as name (variable), nameLength (2) value (variable), 
valueLength (4), timestamp (8), local delete (4), type (1)

10k Rows, 500 Columns, 4byte names, 32byte value
Payload: 5M Columns: 275M

|| Cache || Retained Size || Num Objects ||
| Standard   | 1,280 MB   | 10M  |
| ByteBuffer |   277 MB   | 20k  |

10k Rows, 50 Columns, 4byte names, 1byte value
Payload: 500k Columns: 28M

|| Cache || Retained Size || Num Objects ||
| Standard   |   112 MB   | 900k |
| ByteBuffer |30 MB   |  20k |


h2. Performance

All rows had 500 cols with 32bytes values and int names/keys.

For a starter the following are simple 'as fast as you can' stress tests.
Performance indicator is pages / sec.

Name Filter: Random get of one col
Slice Filter: Random slice of 10 cols

Comparisons: 

- No row cache but everything in page cache
- Alternative Cache File System Layout (V1)
- Standard Map Cache
- Alternative Cache New Layout (V2)

h3. No row cache, Non compacted, (average 2,5 SST reads for slices)

Get: 12k
Slice: 6.5k

h3. No row cache, Compacted

Get: 12k
Slice: 9.2k

h3. Alternative Cache V1

Get: 15.9k
Slice: 14.6k

h3. Good old non serializing row cache

Get: 25.4k
Slice: 23k

h3. Alternative Cache V2

Get: 25.5k
Slice: 24k

We still plan to take this live, but since I wrote more code than initially 
thought I need to write more unit tests.

So long.


> 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
> 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 
> in

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

2012-01-27 Thread Jonathan Ellis (Commented) (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2864:
---

bq. Not sure what you mean by 'read-time row defragmentation'. Sounds like 
superseding but I thought that would be 1.1. as well?

You're right (CASSANDRA-2503).  It was long enough ago that I thought it made 
it into 1.0, but it did not.

bq. I guess this one could be closed as 'wont fix'?

I'm fine with leaving it as In Progress if you're testing it soon.

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

--
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




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

2012-01-27 Thread Daniel Doubleday (Commented) (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

Well last couple of months we had to few problems with cassandra (thanks) to 
move on with this but we actually just started to upgrade to 1.0 using this 
cache implementation. We hope to deploy within 3-4 weeks. We can then compare 
the different approaches.

I'm pretty confident that this makes sense for our upcoming use case: an 
average of 5k small columns per row. Many updates, random access queries. We 
basically get the footprint of the serializing cache but with update support 
and no need to deserialize the entire row. But we'll see.

If successful we would make this a global cache as well.

Not sure what you mean by 'read-time row defragmentation'. Sounds like 
superseding but I thought that would be 1.1. as well?

I guess this one could be closed as 'wont fix'? 

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

--
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




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

2011-07-11 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2864:
---

bq. one shared row-cache for all CFs that is only configured with max size in MB

That would be a big improvement, I agree.  It would be awesome to make that the 
default.

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

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




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

2011-07-11 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

Agreed to some extend. 

I guess my point is that while I certainly think that the said tasks will help 
and in most cases probably mitigate the need for row caching I assume that 
there still will be other cases such as relatively small hot data sets with 
loads of reads and writes that you would normally put in memcache or such. For 
those I think you get more bang.
 
But we will see. Can always do a shoot-out after the improvements are in.

Also I think that one shared row-cache for all CFs that is only configured with 
max size in MB that can do CASSANDRA-1956 (dont cache row but row + filter) and 
read through BRAF that skips cache could be worth testing. But this will 
probably end in the big 'Would be cool if I hade the time'-box.


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

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




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

2011-07-09 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2864:
---

bq. I was more thinking of replacing the old row cache

That does make more sense than having both, but it's not clear to me that a new 
container that has some properties of both memtable and sstable, is better than 
building something out of those primitives.

Taking that (2498) approach, you get all the benefits of the sstable 
infrastructure (persistence, stat tracking, even streaming to new nodes) for 
free, as well as playing nicely with the OS's page cache instead of being a 
separate memory area.

bq. implementing a variation of CASSANDRA-1956 will be pretty easy since we can 
work with the standard filters now

True, but you could do the same kind of IColumnIterator for the existing cache 
api just as easily, no?

bq. it seems that they dont help for slicing

Not without extra metadata, no.  But I'm okay with adding 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 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.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira




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

2011-07-07 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday commented on CASSANDRA-2864:
-

bq. However, I'm reluctant to add more special cases to the read path

Well I was more thinking of replacing the old row cache :-) 
In terms of "throughput over latency" this might be a winner. It seems read 
latencies increase only moderately combined with lowered mem usage ...

Also I think that implementing a variation of CASSANDRA-1956 will be pretty 
easy since we can work with the standard filters now.
So instead of putting toplevel columns back in the cache one could just cache 
the filtered columns. Plus a little logic that decides wether the cache can 
handle the request.

But I understand that this is quite a change and the patch is easy to maintain 
so we can always patch.

bq. It looks like the CASSANDRA-2498 + CASSANDRA-2503 approach might offer 
similar benefits

These look promising but it seems that they dont help for slicing 
(CASSANDRA-2503 might make the slicing case even worse) and FWIW we do slice a 
lot even in skinny cached rows. Looks like we have the worst case scenario 
there: lots of random updates (in terms of ranges) so even if there were cached 
range meta infos for sstables somewhere I doubt that it would really work for 
us.

Anyways I will look at CASSANDRA-2498 this weekend and check if I think that I 
can come up with a patch. Or else report that I cant.


> 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




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

2011-07-06 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-2864:
---

(Just committed CASSANDRA-2753, which adds the timestamp tracking needed for 
2498.)

> 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




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

2011-07-06 Thread Jonathan Ellis (JIRA)

[ 
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