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

2012-10-15 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday edited comment on CASSANDRA-2864 at 10/15/12 6:55 AM:
---

Ok - next attempt:

Basic idea is optimistic locking. The DataTracker.View gets a generation 
number. The cache miss read compares the current generation number with the one 
the read was created with. If it finds that they dont match it doesn't write 
the row to the cache. Also there is some double checking on read.

I think the getThroughCacheMethod documents the idea mostly so I paste it here:

{noformat}
private ColumnFamily getThroughCache(UUID cfId, QueryFilter filter, int 
gcBefore)
{
assert isRowCacheEnabled()
   : String.format(Row cache is not enabled on column family [ + 
getColumnFamilyName() + ]);

RowCacheKey key = new RowCacheKey(cfId, filter.key);

CachedRow cachedRow = (CachedRow) 
CacheService.instance.rowCache.get(key);
if (cachedRow != null)
{
if (cachedRow.isValid())
{
RowCacheCollationController collationController = new 
RowCacheCollationController(this, memtables(), cachedRow, filter, gcBefore);
ColumnFamily returnCF = collationController.getColumnFamily();
if (!metadata.getDefaultValidator().isCommutative() || 
collationController.getView().generation == data.getView().generation)
return returnCF;
else
return getIgnoreCache(filter, gcBefore);
}
else
return getIgnoreCache(filter, gcBefore);

}
else
{
// for cache = false: we dont cache the cf itself
CollationController controller = 
collateTopLevelColumns(QueryFilter.getIdentityFilter(filter.key, new 
QueryPath(columnFamily)), gcBefore, false);
ColumnFamily cf = controller.getColumnFamily();
if (cf != null)
{
cachedRow = CachedRow.serialize(cf);
if (controller.getView().generation == 
data.getView().generation)
{
// we can try to set the row in the cache but if 
mergeRowCache runs before the putIfAbsent
// it wont see the row and we'll loose the update
boolean setInCache = 
CacheService.instance.rowCache.putIfAbsent(key, cachedRow);
if (setInCache)
{
// before flush switchMemtable is called which 
increments the view generation
// so only when the generation re-check is ok we can 
mark the cached row as valid
if (controller.getView().generation == 
data.getView().generation)
cachedRow.setValid(true);
else
CacheService.instance.rowCache.remove(key);
}
}
return filterColumnFamily(cf, filter, gcBefore);
}

return null;
}
}
{noformat}

I created a patch based on your branch. -I *think* that it would be also safe 
to call getThroughCache for counters now-. But I haven't done any testing so 
far but wanted to get your opinion first if that could work.

EDIT: Scratch the counters thing. Doesn't take flushing memtables into account 
yet.

  was (Author: doubleday):
Ok - next attempt:

Basic idea is optimistic locking. The DataTracker.View gets a generation 
number. The cache miss read compares the current generation number with the one 
the read was created with. If it finds that they dont match it doesn't write 
the row to the cache. Also there is some double checking on read.

I think the getThroughCacheMethod documents the idea mostly so I paste it here:

{noformat}
private ColumnFamily getThroughCache(UUID cfId, QueryFilter filter, int 
gcBefore)
{
assert isRowCacheEnabled()
   : String.format(Row cache is not enabled on column family [ + 
getColumnFamilyName() + ]);

RowCacheKey key = new RowCacheKey(cfId, filter.key);

CachedRow cachedRow = (CachedRow) 
CacheService.instance.rowCache.get(key);
if (cachedRow != null)
{
if (cachedRow.isValid())
{
RowCacheCollationController collationController = new 
RowCacheCollationController(this, memtables(), cachedRow, filter, gcBefore);
ColumnFamily returnCF = collationController.getColumnFamily();
if (!metadata.getDefaultValidator().isCommutative() || 
collationController.getView().generation == data.getView().generation)
return returnCF;
else
return getIgnoreCache(filter, gcBefore);
  

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

2012-10-15 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2864:


Attachment: (was: optimistic-locking.patch)

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

 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, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


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

2012-10-15 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2864:


Attachment: optimistic-locking.patch

Second shot ...

This one is special casing cached reads for counters.

The relevant change in CFS looks like this:

{noformat}

ViewFragment viewFragment = memtables();

// cant use the cache for counters when key is in one of the flushing memtables
boolean commutative = metadata.getDefaultValidator().isCommutative();
if (commutative  viewFragment.keyIsFlushing(filter.key))
return getIgnoreCache(filter, gcBefore);

RowCacheCollationController collationController = new 
RowCacheCollationController(this, viewFragment, cachedRow, filter, gcBefore);
ColumnFamily returnCF = collationController.getColumnFamily();

// for counters we must make sure that flushing didnt start during this read
if (!commutative || collationController.getView().generation == 
data.getView().generation)
return returnCF;
else
return getIgnoreCache(filter, gcBefore);

{noformat}

One issue is that cache hit ratios will not reflect the edge cases.

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

 Attachments: 0001-CASSANDRA-2864-w-out-direct-counter-support.patch, 
 optimistic-locking.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 
 

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

2012-10-15 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday edited comment on CASSANDRA-2864 at 10/15/12 10:41 AM:


Second shot ...

This one is special casing cached reads for counters.

The relevant change in CFS (cache hit case) looks like this:

{noformat}

ViewFragment viewFragment = memtables();

// cant use the cache for counters when key is in one of the flushing memtables
boolean commutative = metadata.getDefaultValidator().isCommutative();
if (commutative  viewFragment.keyIsFlushing(filter.key))
return getIgnoreCache(filter, gcBefore);

RowCacheCollationController collationController = new 
RowCacheCollationController(this, viewFragment, cachedRow, filter, gcBefore);
ColumnFamily returnCF = collationController.getColumnFamily();

// for counters we must make sure that flushing didnt start during this read
if (!commutative || collationController.getView().generation == 
data.getView().generation)
return returnCF;
else
return getIgnoreCache(filter, gcBefore);

{noformat}

One issue is that cache hit ratios will not reflect the edge cases.

  was (Author: doubleday):
Second shot ...

This one is special casing cached reads for counters.

The relevant change in CFS looks like this:

{noformat}

ViewFragment viewFragment = memtables();

// cant use the cache for counters when key is in one of the flushing memtables
boolean commutative = metadata.getDefaultValidator().isCommutative();
if (commutative  viewFragment.keyIsFlushing(filter.key))
return getIgnoreCache(filter, gcBefore);

RowCacheCollationController collationController = new 
RowCacheCollationController(this, viewFragment, cachedRow, filter, gcBefore);
ColumnFamily returnCF = collationController.getColumnFamily();

// for counters we must make sure that flushing didnt start during this read
if (!commutative || collationController.getView().generation == 
data.getView().generation)
return returnCF;
else
return getIgnoreCache(filter, gcBefore);

{noformat}

One issue is that cache hit ratios will not reflect the edge cases.
  
 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.3

 Attachments: 0001-CASSANDRA-2864-w-out-direct-counter-support.patch, 
 optimistic-locking.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  
 ---

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

2012-10-13 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2864:


Attachment: optimistic-locking.patch

Ok - next attempt:

Basic idea is optimistic locking. The DataTracker.View gets a generation 
number. The cache miss read compares the current generation number with the one 
the read was created with. If it finds that they dont match it doesn't write 
the row to the cache. Also there is some double checking on read.

I think the getThroughCacheMethod documents the idea mostly so I paste it here:

{noformat}
private ColumnFamily getThroughCache(UUID cfId, QueryFilter filter, int 
gcBefore)
{
assert isRowCacheEnabled()
   : String.format(Row cache is not enabled on column family [ + 
getColumnFamilyName() + ]);

RowCacheKey key = new RowCacheKey(cfId, filter.key);

CachedRow cachedRow = (CachedRow) 
CacheService.instance.rowCache.get(key);
if (cachedRow != null)
{
if (cachedRow.isValid())
{
RowCacheCollationController collationController = new 
RowCacheCollationController(this, memtables(), cachedRow, filter, gcBefore);
ColumnFamily returnCF = collationController.getColumnFamily();
if (!metadata.getDefaultValidator().isCommutative() || 
collationController.getView().generation == data.getView().generation)
return returnCF;
else
return getIgnoreCache(filter, gcBefore);
}
else
return getIgnoreCache(filter, gcBefore);

}
else
{
// for cache = false: we dont cache the cf itself
CollationController controller = 
collateTopLevelColumns(QueryFilter.getIdentityFilter(filter.key, new 
QueryPath(columnFamily)), gcBefore, false);
ColumnFamily cf = controller.getColumnFamily();
if (cf != null)
{
cachedRow = CachedRow.serialize(cf);
if (controller.getView().generation == 
data.getView().generation)
{
// we can try to set the row in the cache but if 
mergeRowCache runs before the putIfAbsent
// it wont see the row and we'll loose the update
boolean setInCache = 
CacheService.instance.rowCache.putIfAbsent(key, cachedRow);
if (setInCache)
{
// before flush switchMemtable is called which 
increments the view generation
// so only when the generation re-check is ok we can 
mark the cached row as valid
if (controller.getView().generation == 
data.getView().generation)
cachedRow.setValid(true);
else
CacheService.instance.rowCache.remove(key);
}
}
return filterColumnFamily(cf, filter, gcBefore);
}

return null;
}
}
{noformat}

I created a patch based on your branch. I *think* that it would be also safe to 
call getThroughCache for counters now. But I haven't done any testing so far 
but wanted to get your opinion first if that could 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.3

 Attachments: 0001-CASSANDRA-2864-w-out-direct-counter-support.patch, 
 optimistic-locking.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 

[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-tabpanelfocusedCommentId=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, please contact your JIRA administrators: 

[jira] [Updated] (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:all-tabpanel
 ]

Daniel Doubleday updated CASSANDRA-2864:


Attachment: rowcache-with-snaptree-sketch.patch

 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, 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-07-10 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2864?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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 CollationController is used to access and collate via cache and 
 said new iterators. It also 

[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-tabpanelfocusedCommentId=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 the cached row by max update 
 timestamp
 h4. Writes
 - Writes dont update or 

[jira] [Commented] (CASSANDRA-4009) Increase usage of Metrics and flesh out o.a.c.metrics

2012-06-29 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4009?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13403757#comment-13403757
 ] 

Daniel Doubleday commented on CASSANDRA-4009:
-

We're using Metrics with Graphite which is quite nice. Any chance to add an 
option for reporters?

 Increase usage of Metrics and flesh out o.a.c.metrics
 -

 Key: CASSANDRA-4009
 URL: https://issues.apache.org/jira/browse/CASSANDRA-4009
 Project: Cassandra
  Issue Type: Improvement
Reporter: Brandon Williams
Assignee: Yuki Morishita
Priority: Minor
 Fix For: 1.2

 Attachments: 4009.txt


 With CASSANDRA-3671 we have begun using the Metrics packages to expose stats 
 in a new JMX structure, intended to be more user-friendly (for example, you 
 don't need to know what a StorageProxy is or does.)  This ticket serves as a 
 parent for subtasks to finish fleshing out the rest of the enhanced metrics.

--
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-06-29 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2864?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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 data view is 
 switched. I fear that this is killing counters because they would be 
 

[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-tabpanelfocusedCommentId=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 (sstCache.getCapacity()  0) {
 mergeSSTCache(memtable);
 }
 

[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-tabpanelfocusedCommentId=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   
 ===
 Column Index (num cols * 12)  
 

[jira] [Updated] (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:all-tabpanel
 ]

Daniel Doubleday updated CASSANDRA-2864:


Attachment: 0001-CASSANDRA-2864-w-out-direct-counter-support.patch

 Alternative Row Cache Implementation
 

 Key: CASSANDRA-2864
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2864
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Daniel Doubleday
Assignee: Daniel Doubleday
  Labels: cache
 Fix For: 1.2

 Attachments: 0001-CASSANDRA-2864-w-out-direct-counter-support.patch


 we have been working on an alternative implementation to the existing row 
 cache(s)
 We have 2 main goals:
 - Decrease memory - get more rows in the cache without suffering a huge 
 performance penalty
 - Reduce gc pressure
 This sounds a lot like we should be using the new serializing cache in 0.8. 
 Unfortunately our workload consists of loads of updates which would 
 invalidate the cache all the time.
 *Note: Updated Patch Description (Please check history if you're interested 
 where this was comming from)*
 h3. Rough Idea
 - Keep serialized row (ByteBuffer) in mem which represents unfiltered but 
 collated columns of all ssts but not memtable columns
 - Writes dont affect the cache at all. They go only to the memtables
 - Reads collect columns from memtables and row cache
 - Serialized Row is re-written (merged) with mem tables when flushed
 h3. Some Implementation Details
 h4. Reads
 - Basically the read logic differ from regular uncached reads only in that a 
 special CollationController which is deserializing columns from in memory 
 bytes
 - In the first version of this cache the serialized in memory format was the 
 same as the fs format but test showed that performance sufferd because a lot 
 of unnecessary deserialization takes place and that columns seeks are O( n ) 
 whithin one block
 - To improve on that a different in memory format was used. It splits length 
 meta info and data of columns so that the names can be binary searched. 
 {noformat}
 ===
 Header (24)
 ===
 MaxTimestamp:long  
 LocalDeletionTime:   int   
 MarkedForDeleteAt:   long  
 NumColumns:  int   
 ===
 Column Index (num cols * 12)  
 ===
 NameOffset:  int   
 ValueOffset: int   
 ValueLength: int   
 ===
 Column Data
 ===
 Name:byte[]
 Value:   byte[]
 SerializationFlags:  byte  
 Misc:? 
 Timestamp:   long  
 ---
 Misc Counter Column
 ---
 TSOfLastDelete:  long  
 ---
 Misc Expiring Column   
 ---
 TimeToLive:  int   
 LocalDeletionTime:   int   
 ===
 {noformat}
 - These rows are read by 2 new column interators which correspond to 
 SSTableNamesIterator and SSTableSliceIterator. During filtering only columns 
 that actually match are constructed. The searching / skipping is performed on 
 the raw ByteBuffer and does not create any objects.
 - A special CollationController is used to access and collate via cache and 
 said new iterators. It also supports skipping the cached row by max update 
 timestamp
 h4. Writes
 - Writes dont update or invalidate the cache.
 - In CFS.replaceFlushed memtables are merged before the data view is 
 switched. I fear that this is killing counters because they would be 
 overcounted but my understading of counters is somewhere between weak and 
 non-existing. I guess that counters if one wants to support them here would 
 need an additional unique local identifier in memory and in serialized cache 
 to be able to filter duplicates or something like that.
 {noformat}
 void replaceFlushed(Memtable memtable, SSTableReader sstable)
 {
 if (sstCache.getCapacity()  0) {
 mergeSSTCache(memtable);
 }
 data.replaceFlushed(memtable, sstable);
 CompactionManager.instance.submitBackground(this);
 }
 {noformat}
 Test Results: See comments below

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (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:all-tabpanel
 ]

Daniel Doubleday updated CASSANDRA-2864:


Attachment: (was: 0001-CASSANDRA-2864-w-out-direct-counter-su.patch)

 Alternative Row Cache Implementation
 

 Key: CASSANDRA-2864
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2864
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Daniel Doubleday
Assignee: Daniel Doubleday
  Labels: cache
 Fix For: 1.2

 Attachments: 0001-CASSANDRA-2864-w-out-direct-counter-support.patch


 we have been working on an alternative implementation to the existing row 
 cache(s)
 We have 2 main goals:
 - Decrease memory - get more rows in the cache without suffering a huge 
 performance penalty
 - Reduce gc pressure
 This sounds a lot like we should be using the new serializing cache in 0.8. 
 Unfortunately our workload consists of loads of updates which would 
 invalidate the cache all the time.
 *Note: Updated Patch Description (Please check history if you're interested 
 where this was comming from)*
 h3. Rough Idea
 - Keep serialized row (ByteBuffer) in mem which represents unfiltered but 
 collated columns of all ssts but not memtable columns
 - Writes dont affect the cache at all. They go only to the memtables
 - Reads collect columns from memtables and row cache
 - Serialized Row is re-written (merged) with mem tables when flushed
 h3. Some Implementation Details
 h4. Reads
 - Basically the read logic differ from regular uncached reads only in that a 
 special CollationController which is deserializing columns from in memory 
 bytes
 - In the first version of this cache the serialized in memory format was the 
 same as the fs format but test showed that performance sufferd because a lot 
 of unnecessary deserialization takes place and that columns seeks are O( n ) 
 whithin one block
 - To improve on that a different in memory format was used. It splits length 
 meta info and data of columns so that the names can be binary searched. 
 {noformat}
 ===
 Header (24)
 ===
 MaxTimestamp:long  
 LocalDeletionTime:   int   
 MarkedForDeleteAt:   long  
 NumColumns:  int   
 ===
 Column Index (num cols * 12)  
 ===
 NameOffset:  int   
 ValueOffset: int   
 ValueLength: int   
 ===
 Column Data
 ===
 Name:byte[]
 Value:   byte[]
 SerializationFlags:  byte  
 Misc:? 
 Timestamp:   long  
 ---
 Misc Counter Column
 ---
 TSOfLastDelete:  long  
 ---
 Misc Expiring Column   
 ---
 TimeToLive:  int   
 LocalDeletionTime:   int   
 ===
 {noformat}
 - These rows are read by 2 new column interators which correspond to 
 SSTableNamesIterator and SSTableSliceIterator. During filtering only columns 
 that actually match are constructed. The searching / skipping is performed on 
 the raw ByteBuffer and does not create any objects.
 - A special CollationController is used to access and collate via cache and 
 said new iterators. It also supports skipping the cached row by max update 
 timestamp
 h4. Writes
 - Writes dont update or invalidate the cache.
 - In CFS.replaceFlushed memtables are merged before the data view is 
 switched. I fear that this is killing counters because they would be 
 overcounted but my understading of counters is somewhere between weak and 
 non-existing. I guess that counters if one wants to support them here would 
 need an additional unique local identifier in memory and in serialized cache 
 to be able to filter duplicates or something like that.
 {noformat}
 void replaceFlushed(Memtable memtable, SSTableReader sstable)
 {
 if (sstCache.getCapacity()  0) {
 mergeSSTCache(memtable);
 }
 data.replaceFlushed(memtable, sstable);
 CompactionManager.instance.submitBackground(this);
 }
 {noformat}
 Test Results: See comments below

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Reopened] (CASSANDRA-3708) Support composite prefix tombstones

2012-06-20 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday reopened CASSANDRA-3708:
-


IntervalTree:254

{code}
if (comparator == null)
  Collections.sort(allEndpoints, comparator);
else
  Collections.sort((ListComparable)allEndpoints);
{code}

 Support composite prefix tombstones
 -

 Key: CASSANDRA-3708
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3708
 Project: Cassandra
  Issue Type: Sub-task
Reporter: Jonathan Ellis
Assignee: Sylvain Lebresne
 Fix For: 1.2




--
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-3708) Support composite prefix tombstones

2012-06-20 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-3708?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13397630#comment-13397630
 ] 

Daniel Doubleday commented on CASSANDRA-3708:
-

Dont want to reopen again because its not a bug but maybe an improvement ...

It seems that searchInternal is always descending into the tree even when there 
are no chances to find a result anymore.
Unless I missing something a break early test could help (unit tests pass and 
hit the break early test):

{code}
void searchInternal(IntervalC, D searchInterval, ListD results)
{
if (comparePoints(searchInterval.max, low)  0 ||
comparePoints(searchInterval.min, high)  0) return;
{code}


 Support composite prefix tombstones
 -

 Key: CASSANDRA-3708
 URL: https://issues.apache.org/jira/browse/CASSANDRA-3708
 Project: Cassandra
  Issue Type: Sub-task
Reporter: Jonathan Ellis
Assignee: Sylvain Lebresne
 Fix For: 1.2




--
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-4303) Compressed bloomfilters

2012-06-03 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4303?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13288173#comment-13288173
 ] 

Daniel Doubleday commented on CASSANDRA-4303:
-

I totally understand / agree with what's being said: You don't want any portion 
of the bf not im memory for a cf that's under any significant read load.

My point was that for any significant read load and reasonable amount of memory 
the bloom filter will be in memory due to its random access nature. I did spend 
some time on page cache related tests and found it pretty hard to out smart. 
Its generational design doesn't just page out stuff because you are running 
through some large files once. So my theory was that if a bloom filter is hot 
(and it's hot pretty fast) it will stay in memory or you are so under-equipped 
with RAM that it doesn't matter.

But I guess you are right that it doesn't really help the underlying issue that 
bloom filters get too large for a large amount of rows. They need to be in 
memory one way or the other ...

It might be useful to be able to reduce the bf size dynamically though. So 
instead of reducing FP and rewriting the filters on disc you could leave it at 
minimum and just do one more mod operation to map bit pos to buckets while 
deserializing. 
   

 Compressed bloomfilters
 ---

 Key: CASSANDRA-4303
 URL: https://issues.apache.org/jira/browse/CASSANDRA-4303
 Project: Cassandra
  Issue Type: Improvement
Reporter: Brandon Williams
 Fix For: 1.2


 Very commonly, people encountering an OOM need to increase their bloom filter 
 false positive ratio to reduce memory pressure, since BFs tend to be the 
 largest shareholder.  It would make sense if we could alleviate the memory 
 pressure from BFs with compression while maintaining the FP ratio (at the 
 cost of a bit of cpu) that some users have come to expect.  One possible 
 implementation is at http://code.google.com/p/javaewah/

--
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-4303) Compressed bloomfilters

2012-06-01 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4303?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13287349#comment-13287349
 ] 

Daniel Doubleday commented on CASSANDRA-4303:
-

Out of curiosity because I already started experimenting in this direction:

Has anybody benchmarked the option to keep bloom filters on disk. I did some 
simple tests and could read around 6M random ints / sec from a paged-in 
SegmentedFile. My line of thinking was: why not letting the os decide where the 
memory should be allocated because its a tradeoff sst vs other data kept in mem 
anyways.

I'm not close to a patch to do some real world testing so maybe you can save me 
from the effort because someone else tried and learned that it does not make 
sense.

 Compressed bloomfilters
 ---

 Key: CASSANDRA-4303
 URL: https://issues.apache.org/jira/browse/CASSANDRA-4303
 Project: Cassandra
  Issue Type: Improvement
Reporter: Brandon Williams
 Fix For: 1.2


 Very commonly, people encountering an OOM need to increase their bloom filter 
 false positive ratio to reduce memory pressure, since BFs tend to be the 
 largest shareholder.  It would make sense if we could alleviate the memory 
 pressure from BFs with compression while maintaining the FP ratio (at the 
 cost of a bit of cpu) that some users have come to expect.  One possible 
 implementation is at http://code.google.com/p/javaewah/

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

2012-05-14 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2864:


Attachment: 0001-First-take-w-out-direct-counter-su.patch

 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-First-take-w-out-direct-counter-su.patch


 we have been working on an alternative implementation to the existing row 
 cache(s)
 We have 2 main goals:
 - Decrease memory - get more rows in the cache without suffering a huge 
 performance penalty
 - Reduce gc pressure
 This sounds a lot like we should be using the new serializing cache in 0.8. 
 Unfortunately our workload consists of loads of updates which would 
 invalidate the cache all the time.
 *Note: Updated Patch Description (Please check history if you're interested 
 where this was comming from)*
 h3. Rough Idea
 - Keep serialized row (ByteBuffer) in mem which represents unfiltered but 
 collated columns of all ssts but not memtable columns
 - Writes dont affect the cache at all. They go only to the memtables
 - Reads collect columns from memtables and row cache
 - Serialized Row is re-written (merged) with mem tables when flushed
 h3. Some Implementation Details
 h4. Reads
 - Basically the read logic differ from regular uncached reads only in that a 
 special CollationController which is deserializing columns from in memory 
 bytes
 - In the first version of this cache the serialized in memory format was the 
 same as the fs format but test showed that performance sufferd because a lot 
 of unnecessary deserialization takes place and that columns seeks are O( n ) 
 whithin one block
 - To improve on that a different in memory format was used. It splits length 
 meta info and data of columns so that the names can be binary searched. 
 {noformat}
 ===
 Header (24)
 ===
 MaxTimestamp:long  
 LocalDeletionTime:   int   
 MarkedForDeleteAt:   long  
 NumColumns:  int   
 ===
 Column Index (num cols * 12)  
 ===
 NameOffset:  int   
 ValueOffset: int   
 ValueLength: int   
 ===
 Column Data
 ===
 Name:byte[]
 Value:   byte[]
 SerializationFlags:  byte  
 Misc:? 
 Timestamp:   long  
 ---
 Misc Counter Column
 ---
 TSOfLastDelete:  long  
 ---
 Misc Expiring Column   
 ---
 TimeToLive:  int   
 LocalDeletionTime:   int   
 ===
 {noformat}
 - These rows are read by 2 new column interators which correspond to 
 SSTableNamesIterator and SSTableSliceIterator. During filtering only columns 
 that actually match are constructed. The searching / skipping is performed on 
 the raw ByteBuffer and does not create any objects.
 - A special CollationController is used to access and collate via cache and 
 said new iterators. It also supports skipping the cached row by max update 
 timestamp
 h4. Writes
 - Writes dont update or invalidate the cache.
 - In CFS.replaceFlushed memtables are merged before the data view is 
 switched. I fear that this is killing counters because they would be 
 overcounted but my understading of counters is somewhere between weak and 
 non-existing. I guess that counters if one wants to support them here would 
 need an additional unique local identifier in memory and in serialized cache 
 to be able to filter duplicates or something like that.
 {noformat}
 void replaceFlushed(Memtable memtable, SSTableReader sstable)
 {
 if (sstCache.getCapacity()  0) {
 mergeSSTCache(memtable);
 }
 data.replaceFlushed(memtable, sstable);
 CompactionManager.instance.submitBackground(this);
 }
 {noformat}
 Test Results: See comments below

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




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

2012-05-14 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2864:


Attachment: (was: 0001-First-take-w-out-direct-counter-su.patch)

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

2012-05-14 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2864:


Attachment: 0001-CASSANDRA-2864-w-out-direct-counter-su.patch

Uups - sorry. First patch did not include cache merging.

 Alternative Row Cache Implementation
 

 Key: CASSANDRA-2864
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2864
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Daniel Doubleday
Assignee: Daniel Doubleday
  Labels: cache
 Fix For: 1.2

 Attachments: 0001-CASSANDRA-2864-w-out-direct-counter-su.patch


 we have been working on an alternative implementation to the existing row 
 cache(s)
 We have 2 main goals:
 - Decrease memory - get more rows in the cache without suffering a huge 
 performance penalty
 - Reduce gc pressure
 This sounds a lot like we should be using the new serializing cache in 0.8. 
 Unfortunately our workload consists of loads of updates which would 
 invalidate the cache all the time.
 *Note: Updated Patch Description (Please check history if you're interested 
 where this was comming from)*
 h3. Rough Idea
 - Keep serialized row (ByteBuffer) in mem which represents unfiltered but 
 collated columns of all ssts but not memtable columns
 - Writes dont affect the cache at all. They go only to the memtables
 - Reads collect columns from memtables and row cache
 - Serialized Row is re-written (merged) with mem tables when flushed
 h3. Some Implementation Details
 h4. Reads
 - Basically the read logic differ from regular uncached reads only in that a 
 special CollationController which is deserializing columns from in memory 
 bytes
 - In the first version of this cache the serialized in memory format was the 
 same as the fs format but test showed that performance sufferd because a lot 
 of unnecessary deserialization takes place and that columns seeks are O( n ) 
 whithin one block
 - To improve on that a different in memory format was used. It splits length 
 meta info and data of columns so that the names can be binary searched. 
 {noformat}
 ===
 Header (24)
 ===
 MaxTimestamp:long  
 LocalDeletionTime:   int   
 MarkedForDeleteAt:   long  
 NumColumns:  int   
 ===
 Column Index (num cols * 12)  
 ===
 NameOffset:  int   
 ValueOffset: int   
 ValueLength: int   
 ===
 Column Data
 ===
 Name:byte[]
 Value:   byte[]
 SerializationFlags:  byte  
 Misc:? 
 Timestamp:   long  
 ---
 Misc Counter Column
 ---
 TSOfLastDelete:  long  
 ---
 Misc Expiring Column   
 ---
 TimeToLive:  int   
 LocalDeletionTime:   int   
 ===
 {noformat}
 - These rows are read by 2 new column interators which correspond to 
 SSTableNamesIterator and SSTableSliceIterator. During filtering only columns 
 that actually match are constructed. The searching / skipping is performed on 
 the raw ByteBuffer and does not create any objects.
 - A special CollationController is used to access and collate via cache and 
 said new iterators. It also supports skipping the cached row by max update 
 timestamp
 h4. Writes
 - Writes dont update or invalidate the cache.
 - In CFS.replaceFlushed memtables are merged before the data view is 
 switched. I fear that this is killing counters because they would be 
 overcounted but my understading of counters is somewhere between weak and 
 non-existing. I guess that counters if one wants to support them here would 
 need an additional unique local identifier in memory and in serialized cache 
 to be able to filter duplicates or something like that.
 {noformat}
 void replaceFlushed(Memtable memtable, SSTableReader sstable)
 {
 if (sstCache.getCapacity()  0) {
 mergeSSTCache(memtable);
 }
 data.replaceFlushed(memtable, sstable);
 CompactionManager.instance.submitBackground(this);
 }
 {noformat}
 Test Results: See comments below

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[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-tabpanelfocusedCommentId=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-10 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2864?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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);
 }
 data.replaceFlushed(memtable, sstable);
 

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

2012-05-07 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2864:


Description: 
we have been working on an alternative implementation to the existing row 
cache(s)

We have 2 main goals:

- Decrease memory - get more rows in the cache without suffering a huge 
performance penalty
- Reduce gc pressure

This sounds a lot like we should be using the new serializing cache in 0.8. 
Unfortunately our workload consists of loads of updates which would invalidate 
the cache all the time.

Updated Patch Description (Please check history if you're interested where this 
was comming from)

h3. Rough Idea

- Keep serialized row (ByteBuffer) in mem which represents unfiltered but 
collated columns of all ssts but not memtable columns
- Writes dont affect the cache at all. They go only to the memtables
- Reads collect columns from memtables and row cache
- Serialized Row is re-written (merged) with mem tables when flushed


h3. Some Implementation Details

h4. Reads

- Basically the read logic differ from regular uncached reads only in that a 
special CollationController which is deserializing columns from in memory bytes
- In the first version of this cache the serialized in memory format was the 
same as the fs format but test showed that performance sufferd because a lot of 
unnecessary deserialization takes place and that columns seeks are O( n ) 
whithin one block
- To improve on that a different in memory format was used. It splits names and 
data of columns so that the names portion byte layout is constant and can be 
binary searched. 

{noformat}

===
Header (48)
===
MaxTimestamp:long  
LocalDeletionTime:   int   
MarkedForDeleteAt:   long  
NumColumns:  int   
===
Column Index (num cols * 24)  
===
NameOffset:  int   
ValueOffset: int   
ValueLength: int   
===
Column Data
===
Name:byte[]
Value:   byte[]
SerializationFlags:  byte  
Misc:? 
Timestamp:   long  
---
Misc Counter Column
---
TSOfLastDelete:  long  
---
Misc Expiring Column   
---
TimeToLive:  int   
LocalDeletionTime:   int   
===

{noformat}

- These rows are read by 2 new column interators which correspond to 
SSTableNamesIterator and SSTableSliceIterator. During filtering only columns 
that actually match are constructed. The searching / skipping is performed on 
the raw ByteBuffer and does not create any objects.
- A special CollationController is used to access and collate via cache and 
said new iterators. It also supports skipping the cached row by max update 
timestamp

h4. Writes

- Writes dont update or invalidate the cache.
- In CFS.replaceFlushed memtables are merged before the data view is switched. 
I fear that this is killing counters because they would be overcounted but my 
understading of counters is somewhere between weak and non-existing. I guess 
that counters if one wants to support them here would need an additional unique 
local identifier in memory and in serialized cache to be able to filter 
duplicates or something like that.

{noformat}
void replaceFlushed(Memtable memtable, SSTableReader sstable)
{
if (sstCache.getCapacity()  0) {
mergeSSTCache(memtable);
}
data.replaceFlushed(memtable, sstable);
CompactionManager.instance.submitBackground(this);
}
{noformat}


Test Results: See comments below



  was:
we have been working on an alternative implementation to the existing row 
cache(s)

We have 2 main goals:

- Decrease memory - get more rows in the cache without suffering a huge 
performance penalty
- Reduce gc pressure

This sounds a lot like we should be using the new serializing cache in 0.8. 
Unfortunately our workload consists of loads of updates which would invalidate 
the cache all the time.

The second unfortunate thing is that the idea we came up with doesn't fit the 
new cache provider api...

It looks like this:

Like the serializing cache we basically only cache the serialized byte buffer. 
we don't serialize the bloom filter and try to do some other minor compression 
tricks (var ints etc not done yet). The main difference is that we don't 
deserialize but use the normal sstable iterators and filters as in the regular 
uncached case.

So the read path looks like this:

return filter.collectCollatedColumns(memtable iter, cached row iter)

The write path is not affected. It does not update the cache

During flush we merge all memtable updates with the cached rows.

The attached 

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

2012-05-07 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2864:


Description: 
we have been working on an alternative implementation to the existing row 
cache(s)

We have 2 main goals:

- Decrease memory - get more rows in the cache without suffering a huge 
performance penalty
- Reduce gc pressure

This sounds a lot like we should be using the new serializing cache in 0.8. 
Unfortunately our workload consists of loads of updates which would invalidate 
the cache all the time.

*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 (48)
===
MaxTimestamp:long  
LocalDeletionTime:   int   
MarkedForDeleteAt:   long  
NumColumns:  int   
===
Column Index (num cols * 24)  
===
NameOffset:  int   
ValueOffset: int   
ValueLength: int   
===
Column Data
===
Name:byte[]
Value:   byte[]
SerializationFlags:  byte  
Misc:? 
Timestamp:   long  
---
Misc Counter Column
---
TSOfLastDelete:  long  
---
Misc Expiring Column   
---
TimeToLive:  int   
LocalDeletionTime:   int   
===

{noformat}

- These rows are read by 2 new column interators which correspond to 
SSTableNamesIterator and SSTableSliceIterator. During filtering only columns 
that actually match are constructed. The searching / skipping is performed on 
the raw ByteBuffer and does not create any objects.
- A special CollationController is used to access and collate via cache and 
said new iterators. It also supports skipping the cached row by max update 
timestamp

h4. Writes

- Writes dont update or invalidate the cache.
- In CFS.replaceFlushed memtables are merged before the data view is switched. 
I fear that this is killing counters because they would be overcounted but my 
understading of counters is somewhere between weak and non-existing. I guess 
that counters if one wants to support them here would need an additional unique 
local identifier in memory and in serialized cache to be able to filter 
duplicates or something like that.

{noformat}
void replaceFlushed(Memtable memtable, SSTableReader sstable)
{
if (sstCache.getCapacity()  0) {
mergeSSTCache(memtable);
}
data.replaceFlushed(memtable, sstable);
CompactionManager.instance.submitBackground(this);
}
{noformat}


Test Results: See comments below



  was:
we have been working on an alternative implementation to the existing row 
cache(s)

We have 2 main goals:

- Decrease memory - get more rows in the cache without suffering a huge 
performance penalty
- Reduce gc pressure

This sounds a lot like we should be using the new serializing cache in 0.8. 
Unfortunately our workload consists of loads of updates which would invalidate 
the cache all the time.

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 

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

2012-05-07 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2864:


Description: 
we have been working on an alternative implementation to the existing row 
cache(s)

We have 2 main goals:

- Decrease memory - get more rows in the cache without suffering a huge 
performance penalty
- Reduce gc pressure

This sounds a lot like we should be using the new serializing cache in 0.8. 
Unfortunately our workload consists of loads of updates which would invalidate 
the cache all the time.

*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



  was:
we have been working on an alternative implementation to the existing row 
cache(s)

We have 2 main goals:

- Decrease memory - get more rows in the cache without suffering a huge 
performance penalty
- Reduce gc pressure

This sounds a lot like we should be using the new serializing cache in 0.8. 
Unfortunately our workload consists of loads of updates which would invalidate 
the cache all the time.

*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 

[jira] [Assigned] (CASSANDRA-2498) Improve read performance in update-intensive workload

2011-08-19 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday reassigned CASSANDRA-2498:
---

Assignee: Jonathan Ellis  (was: Daniel Doubleday)

-1 on me for not even seeing that v3 wasn't collection memtables

 Improve read performance in update-intensive workload
 -

 Key: CASSANDRA-2498
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2498
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Minor
  Labels: ponies
 Fix For: 1.0

 Attachments: 2498-v2.txt, 2498-v3.txt, 2498-v4.txt, 
 supersede-name-filter-collations.patch


 Read performance in an update-heavy environment relies heavily on compaction 
 to maintain good throughput. (This is not the case for workloads where rows 
 are only inserted once, because the bloom filter keeps us from having to 
 check sstables unnecessarily.)
 Very early versions of Cassandra attempted to mitigate this by checking 
 sstables in descending generation order (mostly equivalent to descending 
 mtime): once all the requested columns were found, it would not check any 
 older sstables.
 This was incorrect, because data timestamp will not correspond to sstable 
 timestamp, both because compaction has the side effect of refreshing data 
 to a newer sstable, and because hintead handoff may send us data older than 
 what we already have.
 Instead, we could create a per-sstable piece of metadata containing the most 
 recent (client-specified) timestamp for any column in the sstable.  We could 
 then sort sstables by this timestamp instead, and perform a similar 
 optimization (if the remaining sstable client-timestamps are older than the 
 oldest column found in the desired result set so far, we don't need to look 
 further). Since under almost every workload, client timestamps of data in a 
 given sstable will tend to be similar, we expect this to cut the number of 
 sstables down proportionally to how frequently each column in the row is 
 updated. (If each column is updated with each write, we only have to check a 
 single sstable.)
 This may also be useful information when deciding which SSTables to compact.
 (Note that this optimization is only appropriate for named-column queries, 
 not slice queries, since we don't know what non-overlapping columns may exist 
 in older sstables.)

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




[jira] [Commented] (CASSANDRA-2498) Improve read performance in update-intensive workload

2011-08-19 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2498?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13087961#comment-13087961
 ] 

Daniel Doubleday commented on CASSANDRA-2498:
-

That would be a +1 on v4

nit: the test doesn't look right to me (can't really check right now cause I'm 
away form a real keyboard) but

the last assertEquals tests:

{noformat}
+assertEquals(0, cfs.getRecentSSTablesPerReadHistogram()[0]);
{noformat}

but this should surely read

{noformat}
+assertEquals(1, cfs.getRecentSSTablesPerReadHistogram()[0]);
{noformat}

I guess you wanted to do one more write without flushing and do the '0' test ...


 Improve read performance in update-intensive workload
 -

 Key: CASSANDRA-2498
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2498
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Jonathan Ellis
Assignee: Jonathan Ellis
Priority: Minor
  Labels: ponies
 Fix For: 1.0

 Attachments: 2498-v2.txt, 2498-v3.txt, 2498-v4.txt, 
 supersede-name-filter-collations.patch


 Read performance in an update-heavy environment relies heavily on compaction 
 to maintain good throughput. (This is not the case for workloads where rows 
 are only inserted once, because the bloom filter keeps us from having to 
 check sstables unnecessarily.)
 Very early versions of Cassandra attempted to mitigate this by checking 
 sstables in descending generation order (mostly equivalent to descending 
 mtime): once all the requested columns were found, it would not check any 
 older sstables.
 This was incorrect, because data timestamp will not correspond to sstable 
 timestamp, both because compaction has the side effect of refreshing data 
 to a newer sstable, and because hintead handoff may send us data older than 
 what we already have.
 Instead, we could create a per-sstable piece of metadata containing the most 
 recent (client-specified) timestamp for any column in the sstable.  We could 
 then sort sstables by this timestamp instead, and perform a similar 
 optimization (if the remaining sstable client-timestamps are older than the 
 oldest column found in the desired result set so far, we don't need to look 
 further). Since under almost every workload, client timestamps of data in a 
 given sstable will tend to be similar, we expect this to cut the number of 
 sstables down proportionally to how frequently each column in the row is 
 updated. (If each column is updated with each write, we only have to check a 
 single sstable.)
 This may also be useful information when deciding which SSTables to compact.
 (Note that this optimization is only appropriate for named-column queries, 
 not slice queries, since we don't know what non-overlapping columns may exist 
 in older sstables.)

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




[jira] [Commented] (CASSANDRA-2498) Improve read performance in update-intensive workload

2011-08-17 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2498?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13086653#comment-13086653
 ] 

Daniel Doubleday commented on CASSANDRA-2498:
-

bq. avoid the tombstone collection by avoiding full collateColumns until the 
end.

Very nice and clean and solves the tombstone problem.

+1 Looks all good to me

 Improve read performance in update-intensive workload
 -

 Key: CASSANDRA-2498
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2498
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Jonathan Ellis
Assignee: Daniel Doubleday
Priority: Minor
  Labels: ponies
 Fix For: 1.0

 Attachments: 2498-v2.txt, 2498-v3.txt, 
 supersede-name-filter-collations.patch


 Read performance in an update-heavy environment relies heavily on compaction 
 to maintain good throughput. (This is not the case for workloads where rows 
 are only inserted once, because the bloom filter keeps us from having to 
 check sstables unnecessarily.)
 Very early versions of Cassandra attempted to mitigate this by checking 
 sstables in descending generation order (mostly equivalent to descending 
 mtime): once all the requested columns were found, it would not check any 
 older sstables.
 This was incorrect, because data timestamp will not correspond to sstable 
 timestamp, both because compaction has the side effect of refreshing data 
 to a newer sstable, and because hintead handoff may send us data older than 
 what we already have.
 Instead, we could create a per-sstable piece of metadata containing the most 
 recent (client-specified) timestamp for any column in the sstable.  We could 
 then sort sstables by this timestamp instead, and perform a similar 
 optimization (if the remaining sstable client-timestamps are older than the 
 oldest column found in the desired result set so far, we don't need to look 
 further). Since under almost every workload, client timestamps of data in a 
 given sstable will tend to be similar, we expect this to cut the number of 
 sstables down proportionally to how frequently each column in the row is 
 updated. (If each column is updated with each write, we only have to check a 
 single sstable.)
 This may also be useful information when deciding which SSTables to compact.
 (Note that this optimization is only appropriate for named-column queries, 
 not slice queries, since we don't know what non-overlapping columns may exist 
 in older sstables.)

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




[jira] [Commented] (CASSANDRA-2868) Native Memory Leak

2011-07-18 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2868?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13066835#comment-13066835
 ] 

Daniel Doubleday commented on CASSANDRA-2868:
-

Looks good to me. Guess cassandra should just disable the inspector for now 
(probably make it jmx'able to start it manually)

Thu Jul 14 09:39:26 CEST 2011: [anon]: 3234068
Thu Jul 14 17:22:45 CEST 2011: [anon]: 3266888
Fri Jul 15 09:33:53 CEST 2011: [anon]: 3269160
Mon Jul 18 09:54:29 CEST 2011: [anon]: 3270188

 Native Memory Leak
 --

 Key: CASSANDRA-2868
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2868
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.6
Reporter: Daniel Doubleday
Priority: Minor
 Attachments: 2868-v1.txt, low-load-36-hours-initial-results.png


 We have memory issues with long running servers. These have been confirmed by 
 several users in the user list. That's why I report.
 The memory consumption of the cassandra java process increases steadily until 
 it's killed by the os because of oom (with no swap)
 Our server is started with -Xmx3000M and running for around 23 days.
 pmap -x shows
 Total SST: 1961616 (mem mapped data and index files)
 Anon  RSS: 6499640
 Total RSS: 8478376
 This shows that  3G are 'overallocated'.
 We will use BRAF on one of our less important nodes to check wether it is 
 related to mmap and report back.

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




[jira] [Commented] (CASSANDRA-2868) Native Memory Leak

2011-07-16 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2868?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13066372#comment-13066372
 ] 

Daniel Doubleday commented on CASSANDRA-2868:
-

Yes - we did disable the GCInspector.

 Native Memory Leak
 --

 Key: CASSANDRA-2868
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2868
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.6
Reporter: Daniel Doubleday
Priority: Minor
 Attachments: 2868-v1.txt, low-load-36-hours-initial-results.png


 We have memory issues with long running servers. These have been confirmed by 
 several users in the user list. That's why I report.
 The memory consumption of the cassandra java process increases steadily until 
 it's killed by the os because of oom (with no swap)
 Our server is started with -Xmx3000M and running for around 23 days.
 pmap -x shows
 Total SST: 1961616 (mem mapped data and index files)
 Anon  RSS: 6499640
 Total RSS: 8478376
 This shows that  3G are 'overallocated'.
 We will use BRAF on one of our less important nodes to check wether it is 
 related to mmap and report back.

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




[jira] [Commented] (CASSANDRA-2868) Native Memory Leak

2011-07-16 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2868?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13066392#comment-13066392
 ] 

Daniel Doubleday commented on CASSANDRA-2868:
-

Well either it's environment specific or (more likely) others didn't notice / 
care because they have enough memory and/or restart the nodes often enough.

We have 16GB of RAM and run Cassandra with 3GB. Within one month we loose ~3GB 
(13GB - 10GB) files system cache because of the mem leak. Looking at our 
graphs I can't really tell a difference performance wise. So I guess only 
people with weaker servers (less memory headroom) will really notice. We 
noticed only because we got the system oom on a cluster that's not critical and 
which we didn't really monitor.

 Native Memory Leak
 --

 Key: CASSANDRA-2868
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2868
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.6
Reporter: Daniel Doubleday
Priority: Minor
 Attachments: 2868-v1.txt, low-load-36-hours-initial-results.png


 We have memory issues with long running servers. These have been confirmed by 
 several users in the user list. That's why I report.
 The memory consumption of the cassandra java process increases steadily until 
 it's killed by the os because of oom (with no swap)
 Our server is started with -Xmx3000M and running for around 23 days.
 pmap -x shows
 Total SST: 1961616 (mem mapped data and index files)
 Anon  RSS: 6499640
 Total RSS: 8478376
 This shows that  3G are 'overallocated'.
 We will use BRAF on one of our less important nodes to check wether it is 
 related to mmap and report back.

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




[jira] [Commented] (CASSANDRA-2868) Native Memory Leak

2011-07-15 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2868?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13065867#comment-13065867
 ] 

Daniel Doubleday commented on CASSANDRA-2868:
-

It's indeed promising. We have been running this in production for 3 days now 
and rss increased only insignificantly by ~5MB a day. 

 Native Memory Leak
 --

 Key: CASSANDRA-2868
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2868
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.6
Reporter: Daniel Doubleday
Priority: Minor
 Attachments: 2868-v1.txt, low-load-36-hours-initial-results.png


 We have memory issues with long running servers. These have been confirmed by 
 several users in the user list. That's why I report.
 The memory consumption of the cassandra java process increases steadily until 
 it's killed by the os because of oom (with no swap)
 Our server is started with -Xmx3000M and running for around 23 days.
 pmap -x shows
 Total SST: 1961616 (mem mapped data and index files)
 Anon  RSS: 6499640
 Total RSS: 8478376
 This shows that  3G are 'overallocated'.
 We will use BRAF on one of our less important nodes to check wether it is 
 related to mmap and report back.

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




[jira] [Updated] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

2011-07-14 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2753:


Attachment: SSTableWriterTest.patch

Dunno if SSTableWriterTest is the right place but the added test would break.

 Capture the max client timestamp for an SSTable
 ---

 Key: CASSANDRA-2753
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Alan Liang
Assignee: Alan Liang
Priority: Minor
 Fix For: 1.0

 Attachments: 
 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 
 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 
 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 
 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch, 
 SSTableWriterTest.patch, supercolumn.patch




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




[jira] [Updated] (CASSANDRA-2498) Improve read performance in update-intensive workload

2011-07-13 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2498:


Attachment: supersede-name-filter-collations.patch

Took a shot at this one.

I saw 2 ways of doing this:

- Implementing lazy versions of column iterators
- Doing multiple collations while reducing the filter columns when one can be 
sure that the column will supersede

Problem with second choice is that everything in the query filter code assumes 
that all column iterators are collated at once but since first choice seemed to 
be a lot of effort I tried multiple collations anyway.

So that's the plan:

- collect and collate mem tables
- while there are columns in the filter that are not known to supersede any 
other iterate over sorted sstabled and remove all cols that supersede. 
- stop if no cols are left in the filter

Everything takes place in a new CollationController.

So far I found one ugly edge case that comes up with system table that have 0 
grace time.

If you guys think that's a worthwhile approach I'll provide tests. Standard 
test suite obviously runs.

I guess it would be easy to extend that approach to slice filters and skinny 
rows when implementing CASSANDRA-2503. The only thing that would be needed is a 
superseding timestamp in the header of a row / CF same as DeletionInfo and 
probably a configuration option per CF. If too many sstable are read than one 
could compact that row and put it in memtable.

Also with a little (or more) work it might be interesting to see if superseding 
range information could be stored on block level (row index).


 Improve read performance in update-intensive workload
 -

 Key: CASSANDRA-2498
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2498
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Jonathan Ellis
Assignee: Sylvain Lebresne
Priority: Minor
  Labels: ponies
 Fix For: 1.0

 Attachments: supersede-name-filter-collations.patch


 Read performance in an update-heavy environment relies heavily on compaction 
 to maintain good throughput. (This is not the case for workloads where rows 
 are only inserted once, because the bloom filter keeps us from having to 
 check sstables unnecessarily.)
 Very early versions of Cassandra attempted to mitigate this by checking 
 sstables in descending generation order (mostly equivalent to descending 
 mtime): once all the requested columns were found, it would not check any 
 older sstables.
 This was incorrect, because data timestamp will not correspond to sstable 
 timestamp, both because compaction has the side effect of refreshing data 
 to a newer sstable, and because hintead handoff may send us data older than 
 what we already have.
 Instead, we could create a per-sstable piece of metadata containing the most 
 recent (client-specified) timestamp for any column in the sstable.  We could 
 then sort sstables by this timestamp instead, and perform a similar 
 optimization (if the remaining sstable client-timestamps are older than the 
 oldest column found in the desired result set so far, we don't need to look 
 further). Since under almost every workload, client timestamps of data in a 
 given sstable will tend to be similar, we expect this to cut the number of 
 sstables down proportionally to how frequently each column in the row is 
 updated. (If each column is updated with each write, we only have to check a 
 single sstable.)
 This may also be useful information when deciding which SSTables to compact.
 (Note that this optimization is only appropriate for named-column queries, 
 not slice queries, since we don't know what non-overlapping columns may exist 
 in older sstables.)

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




[jira] [Commented] (CASSANDRA-2868) Native Memory Leak

2011-07-12 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2868?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13063770#comment-13063770
 ] 

Daniel Doubleday commented on CASSANDRA-2868:
-

Next: [anon]: 3675224 (+47616KB in 1 day)

bq. Is your data size constant? If not you are probably seeing growth in the 
index samples and bloom filters.

Well no - the data size is increasing. But I thought that index and bf is good 
old plain java heap no? JVM heap stats are really relaxed. Yet I think that 
doesn't really matter because what we are seeing is an ever increasing rss mem 
consumption even though we have -Xmx3G and -Xms3G and mlockall (pmap shows 
these 3G as one block). So something seems to be constantly allocating native 
mem that has nothing to do with java heap.

 Native Memory Leak
 --

 Key: CASSANDRA-2868
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2868
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.6
Reporter: Daniel Doubleday
Priority: Minor

 We have memory issues with long running servers. These have been confirmed by 
 several users in the user list. That's why I report.
 The memory consumption of the cassandra java process increases steadily until 
 it's killed by the os because of oom (with no swap)
 Our server is started with -Xmx3000M and running for around 23 days.
 pmap -x shows
 Total SST: 1961616 (mem mapped data and index files)
 Anon  RSS: 6499640
 Total RSS: 8478376
 This shows that  3G are 'overallocated'.
 We will use BRAF on one of our less important nodes to check wether it is 
 related to mmap and report back.

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




[jira] [Reopened] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

2011-07-12 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday reopened CASSANDRA-2753:
-


While looking into CASSANDRA-2498 I think I found a problem in 
SuperColumn.maxTimestamp.

One of the tests is writing a single deleted SC in a SST and the maxTimestamp 
is negativ which broke my superseding code.

See attached patch

 Capture the max client timestamp for an SSTable
 ---

 Key: CASSANDRA-2753
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Alan Liang
Assignee: Alan Liang
Priority: Minor
 Fix For: 1.0

 Attachments: 
 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 
 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 
 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 
 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch, 
 supercolumn.patch




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




[jira] [Updated] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

2011-07-12 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2753:


Attachment: supercolumn.patch

 Capture the max client timestamp for an SSTable
 ---

 Key: CASSANDRA-2753
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
 Project: Cassandra
  Issue Type: New Feature
  Components: Core
Reporter: Alan Liang
Assignee: Alan Liang
Priority: Minor
 Fix For: 1.0

 Attachments: 
 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 
 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 
 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 
 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch, 
 supercolumn.patch




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




[jira] [Issue Comment Edited] (CASSANDRA-2868) Native Memory Leak

2011-07-11 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2868?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13063280#comment-13063280
 ] 

Daniel Doubleday edited comment on CASSANDRA-2868 at 7/11/11 10:45 AM:
---

Hm after 3 days checking a node that does not use mmaped files it looks like 
this:

nativelib: 14128
locale-archive: 1492
ffiSwFShY(deleted): 8
javajar: 2292
[anon]: 3609388
[stack]: 132
java: 44
7008: 32
jna534482390478104336.tmp: 92

Total RSS: 3627608
Total SST: 0


Compared to start RSS increased by ~400MB. So it seems that this is not related 
to mem mapping.

We will deploy CASSANDRA-2654 this week. Will see if that changes anything but 
I suspect not ...

  was (Author: doubleday):
Hm after 3 days checking a node that does not use mmaped files it looks 
like this:

nativelib: 14128
locale-archive: 1492
ffiSwFShY(deleted): 8
javajar: 2292
[anon]: 3609388
[stack]: 132
java: 44
7008: 32
jna534482390478104336.tmp: 92

Total RSS: 3627608
Total SST: 0


Compared to start RSS increased by ~400MM. So it seems that this is not related 
to mem mapping.

We will deploy CASSANDRA-2654 this week. Will see if that changes anything but 
I suspect not ...
  
 Native Memory Leak
 --

 Key: CASSANDRA-2868
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2868
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.6
Reporter: Daniel Doubleday
Priority: Minor

 We have memory issues with long running servers. These have been confirmed by 
 several users in the user list. That's why I report.
 The memory consumption of the cassandra java process increases steadily until 
 it's killed by the os because of oom (with no swap)
 Our server is started with -Xmx3000M and running for around 23 days.
 pmap -x shows
 Total SST: 1961616 (mem mapped data and index files)
 Anon  RSS: 6499640
 Total RSS: 8478376
 This shows that  3G are 'overallocated'.
 We will use BRAF on one of our less important nodes to check wether it is 
 related to mmap and report back.

--
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-tabpanelfocusedCommentId=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-07 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-2864?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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] [Updated] (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:all-tabpanel
 ]

Daniel Doubleday updated CASSANDRA-2864:


Description: 
we have been working on an alternative implementation to the existing row 
cache(s)

We have 2 main goals:

- Decrease memory - get more rows in the cache without suffering a huge 
performance penalty
- Reduce gc pressure

This sounds a lot like we should be using the new serializing cache in 0.8. 
Unfortunately our workload consists of loads of updates which would invalidate 
the cache all the time.

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.


  was:
we have been working on an alternative implementation to the existing row 
cache(s)

We have 2 main goals:

- Decrease memory - get more rows in the cache without suffering a huge 
performance penalty
- Reduce gc pressure

This sounds a lot like we should be using the new serializing cache in 0.8. 
Unfortunately our workload consists of loads of updates which would invalidate 
the cache all the time.

The second unfortunate thing is that the idea we came up with doesn't fit the 
new cache provider api...

It looks like this:

Like the serializing cache we basically only cache the serialized byte buffer. 
we don't serialize the bloom filter and try to do some other minor compression 
tricks (var ints etc not done yet). The main difference is that we don't 
deserialize but use the normal sstable iterators and filters as in the regular 
uncached case.

So the read path looks like this:

return filter.collectCollatedColumns(memtable iter, cached row iter)

The write path is not affected. It does not update the cache

During flush we merge all memtable updates with the cached rows.

The attached patch is based on 0.8 branch r1143352

It does not replace the existing row cache but sits aside it. Theres 
environment switch to choose the implementation. This way it is easy to 
benchmark performance differences.

-DuseSSTableCache=true enables the alternative cache. It shares its 
configuration with the standard row cache. So the cache capacity is shared. 

We have duplicated a fair amount of code. First we actually refactored the 
existing sstable filter / reader but than decided to minimize dependencies. 
Also this way it is easy to customize serialization for in memory sstable rows. 

We have also experimented a little with 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.



 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 

[jira] [Created] (CASSANDRA-2868) Native Memory Leak

2011-07-07 Thread Daniel Doubleday (JIRA)
Native Memory Leak
--

 Key: CASSANDRA-2868
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2868
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.6
Reporter: Daniel Doubleday
Priority: Minor


We have memory issues with long running servers. These have been confirmed by 
several users in the user list. That's why I report.

The memory consumption of the cassandra java process increases steadily until 
it's killed by the os because of oom (with no swap)

Our server is started with -Xmx3000M and running for around 23 days.

pmap -x shows

Total SST: 1961616 (mem mapped data and index files)
Anon  RSS: 6499640
Total RSS: 8478376

This shows that  3G are 'overallocated'.

We will use BRAF on one of our less important nodes to check wether it is 
related to mmap and report back.

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




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

2011-07-06 Thread Daniel Doubleday (JIRA)
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


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

2011-07-06 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2864:


Attachment: rowcache.patch

 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] [Created] (CASSANDRA-2669) Scrub does not close files

2011-05-19 Thread Daniel Doubleday (JIRA)
Scrub does not close files
--

 Key: CASSANDRA-2669
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2669
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.6
Reporter: Daniel Doubleday
Priority: Minor


After scrubbing I find that cassandra process still holds file handles to the 
deleted sstables:

{noformat}
root@blnrzh047:/mnt/cassandra# jps
6932 Jps
32359 CassandraDaemon
32398 CassandraJmxHttpServer

root@blnrzh047:/mnt/cassandra# du -sh .
315G.

root@blnrzh047:/mnt/cassandra# df -h .
FilesystemSize  Used Avail Use% Mounted on
/dev/md0  1.1T  626G  420G  60% /mnt/cassandra


root@blnrzh047:/mnt/cassandra# lsof | grep /mnt
java  32359root  356r  REG9,0   24
4194599 /mnt/cassandra/data/system/Migrations-f-13-Index.db (deleted)
java  32359root  357r  REG9,0   329451
4194547 /mnt/cassandra/data/system/HintsColumnFamily-f-588-Data.db (deleted)
java  32359root  358r  REG9,0   22
4194546 /mnt/cassandra/data/system/HintsColumnFamily-f-588-Index.db (deleted)
java  32359root  359r  REG9,0   313225
4194534 /mnt/cassandra/data/system/HintsColumnFamily-f-587-Data.db (deleted)
java  32359root  360r  REG9,0   22
4194494 /mnt/cassandra/data/system/HintsColumnFamily-f-587-Index.db (deleted)
java  32359root  361r  REG9,030452
4194636 /mnt/cassandra/data/system/Schema-f-13-Data.db (deleted)
java  32359root  362r  REG9,0  484
4194635 /mnt/cassandra/data/system/Schema-f-13-Index.db (deleted)
{noformat}

I guess there's a missing dataFile.close() in CompactionManager:648


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


[jira] [Created] (CASSANDRA-2670) CF restriction not respected during repair

2011-05-19 Thread Daniel Doubleday (JIRA)
CF restriction not respected during repair
--

 Key: CASSANDRA-2670
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2670
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.6
Reporter: Daniel Doubleday


 I see this:

- Validation compaction runs on nodes 2,3,4 for CF_A only (expected)
- Node 3 streams SSTables from CF_A only to nodes 2 and 4 (expected)
- Nodes 2 and 4 stream SSTables from ALL column families in the keyspace to 
node 2 (VERY unexpected)

This is a quote from:

http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Repair-on-single-CF-not-working-0-7-td5956845.html

only difference is that this description seems to be a rf=2 cluster and ours is 
rf=3

Seems that AES.performStreamingRepair just sends a StreamRequestMessage without 
CF info and the peer nodes will simply send all data from every CF in that 
table they have for that range. But I must be missing something since that 
doesn't make any sense at all.

Fact is that after minor compactions the node on which the repair was triggered 
basically contained everything twice. 

Good news is that while our 0.6 cluster would never have survived this it 
almost didn't affect read latencies. That whole page cache optimization thing 
really seems to work. Very very cool!


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


[jira] [Created] (CASSANDRA-2662) Nodes get ignored by dynamic snitch when read repair chance is zero

2011-05-18 Thread Daniel Doubleday (JIRA)
Nodes get ignored by dynamic snitch when read repair chance is zero
---

 Key: CASSANDRA-2662
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2662
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.6
Reporter: Daniel Doubleday
Priority: Trivial


DynamicEndpointSnitch falls back to subsnitch when one of the scores of the 
endpoints being compared is missing.

This leads to a stable order of hosts until reads will lead to recorded scores. 
If setting read repair chance to 0 and reads are performed with quorum then (rf 
- # quorum nodes) will never get reads.

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


[jira] [Updated] (CASSANDRA-2662) Nodes get ignored by dynamic snitch when read repair chance is zero

2011-05-18 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2662:


Attachment: dynsnitch.patch

One very simple fix is to initialize scores with 0 which forces at least one 
read. 

Dunno if thats a good idea when using multi dc snitches ...

 Nodes get ignored by dynamic snitch when read repair chance is zero
 ---

 Key: CASSANDRA-2662
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2662
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.6
Reporter: Daniel Doubleday
Priority: Trivial
 Attachments: dynsnitch.patch


 DynamicEndpointSnitch falls back to subsnitch when one of the scores of the 
 endpoints being compared is missing.
 This leads to a stable order of hosts until reads will lead to recorded 
 scores. 
 If setting read repair chance to 0 and reads are performed with quorum then 
 (rf - # quorum nodes) will never get reads.

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


[jira] [Created] (CASSANDRA-2595) Tame excessive logging during repairs

2011-05-03 Thread Daniel Doubleday (JIRA)
Tame excessive logging during repairs
-

 Key: CASSANDRA-2595
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2595
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.5
Reporter: Daniel Doubleday
Priority: Trivial


PendingFile.toString is called from logging (i.e. StreamOut:173) which lists 
all sections in the pending file.

This leads to (in our case multi mb ) ... 
(59352638,59354005),(59373477,59379520),(59381952,59385368) ... in the log.

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


[jira] [Updated] (CASSANDRA-2595) Tame excessive logging during repairs

2011-05-03 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-2595:


Comment: was deleted

(was: Dont log seections)

 Tame excessive logging during repairs
 -

 Key: CASSANDRA-2595
 URL: https://issues.apache.org/jira/browse/CASSANDRA-2595
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 0.7.5
Reporter: Daniel Doubleday
Priority: Trivial
 Attachments: PendingFileToString.patch


 PendingFile.toString is called from logging (i.e. StreamOut:173) which lists 
 all sections in the pending file.
 This leads to (in our case multi mb ) ... 
 (59352638,59354005),(59373477,59379520),(59381952,59385368) ... in the log.

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


[jira] [Commented] (CASSANDRA-1862) StorageProxy throws an InvalidRequestException in readProtocol during bootstrap

2011-04-20 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-1862?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13022179#comment-13022179
 ] 

Daniel Doubleday commented on CASSANDRA-1862:
-

Don't know wether commenting a closed jira makes sense but I think that this 
made matters worse because no bootstrapping is not distinguishable anymore.
I thought that UnavailableException would signal that a read cannot be served 
due to CL. And a bootstrapping coordinator does not imply this right? 

 StorageProxy throws an InvalidRequestException in readProtocol during 
 bootstrap
 ---

 Key: CASSANDRA-1862
 URL: https://issues.apache.org/jira/browse/CASSANDRA-1862
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 0.7.0 rc 2
Reporter: Nate McCall
Assignee: Nate McCall
Priority: Minor
 Fix For: 0.7.0 rc 3

 Attachments: 1862.txt, 1862_0.6.txt


 Though the error message provides details, IRE is supposed to signify poorly 
 formed API requests. In the context of a client request, an 
 UnavailableException is more appropriate. This would allow the client to take 
 action like removing the node from its host list. 

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


[jira] [Issue Comment Edited] (CASSANDRA-1862) StorageProxy throws an InvalidRequestException in readProtocol during bootstrap

2011-04-20 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-1862?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13022179#comment-13022179
 ] 

Daniel Doubleday edited comment on CASSANDRA-1862 at 4/20/11 3:41 PM:
--

Don't know wether commenting a closed jira makes sense but I think that this 
made matters worse because now bootstrapping is not distinguishable anymore.
I thought that UnavailableException would signal that a read cannot be served 
due to CL. And a bootstrapping coordinator does not imply this right? 

  was (Author: doubleday):
Don't know wether commenting a closed jira makes sense but I think that 
this made matters worse because no bootstrapping is not distinguishable anymore.
I thought that UnavailableException would signal that a read cannot be served 
due to CL. And a bootstrapping coordinator does not imply this right? 
  
 StorageProxy throws an InvalidRequestException in readProtocol during 
 bootstrap
 ---

 Key: CASSANDRA-1862
 URL: https://issues.apache.org/jira/browse/CASSANDRA-1862
 Project: Cassandra
  Issue Type: Bug
Affects Versions: 0.7.0 rc 2
Reporter: Nate McCall
Assignee: Nate McCall
Priority: Minor
 Fix For: 0.7.0 rc 3

 Attachments: 1862.txt, 1862_0.6.txt


 Though the error message provides details, IRE is supposed to signify poorly 
 formed API requests. In the context of a client request, an 
 UnavailableException is more appropriate. This would allow the client to take 
 action like removing the node from its host list. 

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


[jira] Commented: (CASSANDRA-1956) Convert row cache to row+filter cache

2011-01-28 Thread Daniel Doubleday (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-1956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12988023#action_12988023
 ] 

Daniel Doubleday commented on CASSANDRA-1956:
-

Now it's my ahh  :-) I think I understood your idea.

Instead of configuring cache rules you want to cache every filtered request 
like mysql query cache?

I dropped the idea because I thought it would be either very restricted to 
certain query patterns or very complicated to keep in sync with memtables 
and/or decide whether a query can be served by the cache. Also it might be hard 
to avoid that the cache is being polluted (analogous to the page cache eviction 
problem during compaction). It might force the developer to spread the data 
over multiple CFs according to access pattern which increases memory needs 
(more memtables, more rows).

But yes - if you can come up with an automagical cache management that just 
works that would be obviously nicer!

PS: If you wan to have a look at the patch: apply to 0.7 r1064192


 Convert row cache to row+filter cache
 -

 Key: CASSANDRA-1956
 URL: https://issues.apache.org/jira/browse/CASSANDRA-1956
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Affects Versions: 0.7.0
Reporter: Stu Hood
Assignee: Daniel Doubleday
 Fix For: 0.7.2

 Attachments: 0001-row-cache-filter.patch


 Changing the row cache to a row+filter cache would make it much more useful. 
 We currently have to warn against using the row cache with wide rows, where 
 the read pattern is typically a peek at the head, but this usecase would be 
 perfect supported by a cache that stored only columns matching the filter.
 Possible implementations:
 * (copout) Cache a single filter per row, and leave the cache key as is
 * Cache a list of filters per row, leaving the cache key as is: this is 
 likely to have some gotchas for weird usage patterns, and it requires the 
 list overheard
 * Change the cache key to rowkey+filterid: basically ideal, but you need a 
 secondary index to lookup cache entries by rowkey so that you can keep them 
 in sync with the memtable
 * others?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Updated: (CASSANDRA-1956) Convert row cache to row+filter cache

2011-01-27 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-1956:


Attachment: (was: 0001-row-cache-filter.patch)

 Convert row cache to row+filter cache
 -

 Key: CASSANDRA-1956
 URL: https://issues.apache.org/jira/browse/CASSANDRA-1956
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Affects Versions: 0.7.0
Reporter: Stu Hood
Assignee: Daniel Doubleday
 Fix For: 0.7.2


 Changing the row cache to a row+filter cache would make it much more useful. 
 We currently have to warn against using the row cache with wide rows, where 
 the read pattern is typically a peek at the head, but this usecase would be 
 perfect supported by a cache that stored only columns matching the filter.
 Possible implementations:
 * (copout) Cache a single filter per row, and leave the cache key as is
 * Cache a list of filters per row, leaving the cache key as is: this is 
 likely to have some gotchas for weird usage patterns, and it requires the 
 list overheard
 * Change the cache key to rowkey+filterid: basically ideal, but you need a 
 secondary index to lookup cache entries by rowkey so that you can keep them 
 in sync with the memtable
 * others?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Updated: (CASSANDRA-1956) Convert row cache to row+filter cache

2011-01-27 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-1956:


Attachment: 0001-row-cache-filter.patch

Allow filter to invalidate row

TailRowCacheFilter invalidates cache if a column within its range is deleted

 Convert row cache to row+filter cache
 -

 Key: CASSANDRA-1956
 URL: https://issues.apache.org/jira/browse/CASSANDRA-1956
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Affects Versions: 0.7.0
Reporter: Stu Hood
Assignee: Daniel Doubleday
 Fix For: 0.7.2

 Attachments: 0001-row-cache-filter.patch


 Changing the row cache to a row+filter cache would make it much more useful. 
 We currently have to warn against using the row cache with wide rows, where 
 the read pattern is typically a peek at the head, but this usecase would be 
 perfect supported by a cache that stored only columns matching the filter.
 Possible implementations:
 * (copout) Cache a single filter per row, and leave the cache key as is
 * Cache a list of filters per row, leaving the cache key as is: this is 
 likely to have some gotchas for weird usage patterns, and it requires the 
 list overheard
 * Change the cache key to rowkey+filterid: basically ideal, but you need a 
 secondary index to lookup cache entries by rowkey so that you can keep them 
 in sync with the memtable
 * others?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Updated: (CASSANDRA-1956) Convert row cache to row+filter cache

2011-01-21 Thread Daniel Doubleday (JIRA)

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

Daniel Doubleday updated CASSANDRA-1956:


Attachment: 0001-row-cache-filter.patch

 Convert row cache to row+filter cache
 -

 Key: CASSANDRA-1956
 URL: https://issues.apache.org/jira/browse/CASSANDRA-1956
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Affects Versions: 0.7.0
Reporter: Stu Hood
 Attachments: 0001-row-cache-filter.patch


 Changing the row cache to a row+filter cache would make it much more useful. 
 We currently have to warn against using the row cache with wide rows, where 
 the read pattern is typically a peek at the head, but this usecase would be 
 perfect supported by a cache that stored only columns matching the filter.
 Possible implementations:
 * (copout) Cache a single filter per row, and leave the cache key as is
 * Cache a list of filters per row, leaving the cache key as is: this is 
 likely to have some gotchas for weird usage patterns, and it requires the 
 list overheard
 * Change the cache key to rowkey+filterid: basically ideal, but you need a 
 secondary index to lookup cache entries by rowkey so that you can keep them 
 in sync with the memtable
 * others?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.