[jira] [Updated] (CASSANDRA-2864) Alternative Row Cache Implementation
[ https://issues.apache.org/jira/browse/CASSANDRA-2864?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gavin updated CASSANDRA-2864: - Tester: brandon.williams 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: 2.0 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 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
[ 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
[ 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] [Updated] (CASSANDRA-2864) Alternative Row Cache Implementation
[ 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] [Updated] (CASSANDRA-2864) Alternative Row Cache Implementation
[ 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] [Updated] (CASSANDRA-2864) Alternative Row Cache Implementation
[ 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
[ 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] [Updated] (CASSANDRA-2864) Alternative Row Cache Implementation
[ 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
[ 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
[ 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] [Updated] (CASSANDRA-2864) Alternative Row Cache Implementation
[ https://issues.apache.org/jira/browse/CASSANDRA-2864?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-2864: -- Reviewer: slebresne Priority: Major (was: Minor) Fix Version/s: 1.2 Labels: cache (was: ) 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
[ 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
[ 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
[ 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] [Updated] (CASSANDRA-2864) Alternative Row Cache Implementation
[ https://issues.apache.org/jira/browse/CASSANDRA-2864?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vijay updated CASSANDRA-2864: - Comment: was deleted (was: Wrote comments thinking it was a diffrent ticket hence removed the comments...) Alternative Row Cache Implementation Key: CASSANDRA-2864 URL: https://issues.apache.org/jira/browse/CASSANDRA-2864 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Daniel Doubleday Assignee: Daniel Doubleday Priority: Minor we have been working on an alternative implementation to the existing row cache(s) We have 2 main goals: - Decrease memory - get more rows in the cache without suffering a huge performance penalty - Reduce gc pressure This sounds a lot like we should be using the new serializing cache in 0.8. Unfortunately our workload consists of loads of updates which would invalidate the cache all the time. The second unfortunate thing is that the idea we came up with doesn't fit the new cache provider api... It looks like this: Like the serializing cache we basically only cache the serialized byte buffer. we don't serialize the bloom filter and try to do some other minor compression tricks (var ints etc not done yet). The main difference is that we don't deserialize but use the normal sstable iterators and filters as in the regular uncached case. So the read path looks like this: return filter.collectCollatedColumns(memtable iter, cached row iter) The write path is not affected. It does not update the cache During flush we merge all memtable updates with the cached rows. The attached patch is based on 0.8 branch r1143352 It does not replace the existing row cache but sits aside it. Theres environment switch to choose the implementation. This way it is easy to benchmark performance differences. -DuseSSTableCache=true enables the alternative cache. It shares its configuration with the standard row cache. So the cache capacity is shared. We have duplicated a fair amount of code. First we actually refactored the existing sstable filter / reader but than decided to minimize dependencies. Also this way it is easy to customize serialization for in memory sstable rows. We have also experimented a little with compression but since this task at this stage is mainly to kick off discussion we wanted to keep things simple. But there is certainly room for optimizations. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (CASSANDRA-2864) Alternative Row Cache Implementation
[ https://issues.apache.org/jira/browse/CASSANDRA-2864?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vijay updated CASSANDRA-2864: - Comment: was deleted (was: Hi Jonathan, When there is a write for X3 we invalidate/update the cache and the next fetch does the FS scan and populates the cache after it is out of the cache (it is similar to the page cache and if there is a write on the block the whole block is marked dirty and next fetch will go to the FS). there is a configurable block size when set high enough will cache the whole row (like the existing cache). The logic around it is kind of what the patch has I think you might need to write that book, because the commit history is tough to follow Yeah just wrote a prototype hence... :) I can it up if we agree on that approach.) Alternative Row Cache Implementation Key: CASSANDRA-2864 URL: https://issues.apache.org/jira/browse/CASSANDRA-2864 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Daniel Doubleday Assignee: Daniel Doubleday Priority: Minor we have been working on an alternative implementation to the existing row cache(s) We have 2 main goals: - Decrease memory - get more rows in the cache without suffering a huge performance penalty - Reduce gc pressure This sounds a lot like we should be using the new serializing cache in 0.8. Unfortunately our workload consists of loads of updates which would invalidate the cache all the time. The second unfortunate thing is that the idea we came up with doesn't fit the new cache provider api... It looks like this: Like the serializing cache we basically only cache the serialized byte buffer. we don't serialize the bloom filter and try to do some other minor compression tricks (var ints etc not done yet). The main difference is that we don't deserialize but use the normal sstable iterators and filters as in the regular uncached case. So the read path looks like this: return filter.collectCollatedColumns(memtable iter, cached row iter) The write path is not affected. It does not update the cache During flush we merge all memtable updates with the cached rows. The attached patch is based on 0.8 branch r1143352 It does not replace the existing row cache but sits aside it. Theres environment switch to choose the implementation. This way it is easy to benchmark performance differences. -DuseSSTableCache=true enables the alternative cache. It shares its configuration with the standard row cache. So the cache capacity is shared. We have duplicated a fair amount of code. First we actually refactored the existing sstable filter / reader but than decided to minimize dependencies. Also this way it is easy to customize serialization for in memory sstable rows. We have also experimented a little with compression but since this task at this stage is mainly to kick off discussion we wanted to keep things simple. But there is certainly room for optimizations. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (CASSANDRA-2864) Alternative Row Cache Implementation
[ https://issues.apache.org/jira/browse/CASSANDRA-2864?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Daniel Doubleday updated CASSANDRA-2864: Attachment: (was: 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 Reporter: Daniel Doubleday Assignee: Daniel Doubleday Priority: Minor we have been working on an alternative implementation to the existing row cache(s) We have 2 main goals: - Decrease memory - get more rows in the cache without suffering a huge performance penalty - Reduce gc pressure This sounds a lot like we should be using the new serializing cache in 0.8. Unfortunately our workload consists of loads of updates which would invalidate the cache all the time. The second unfortunate thing is that the idea we came up with doesn't fit the new cache provider api... It looks like this: Like the serializing cache we basically only cache the serialized byte buffer. we don't serialize the bloom filter and try to do some other minor compression tricks (var ints etc not done yet). The main difference is that we don't deserialize but use the normal sstable iterators and filters as in the regular uncached case. So the read path looks like this: return filter.collectCollatedColumns(memtable iter, cached row iter) The write path is not affected. It does not update the cache During flush we merge all memtable updates with the cached rows. The attached patch is based on 0.8 branch r1143352 It does not replace the existing row cache but sits aside it. Theres environment switch to choose the implementation. This way it is easy to benchmark performance differences. -DuseSSTableCache=true enables the alternative cache. It shares its configuration with the standard row cache. So the cache capacity is shared. We have duplicated a fair amount of code. First we actually refactored the existing sstable filter / reader but than decided to minimize dependencies. Also this way it is easy to customize serialization for in memory sstable rows. We have also experimented a little with compression but since this task at this stage is mainly to kick off discussion we wanted to keep things simple. But there is certainly room for optimizations. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (CASSANDRA-2864) Alternative Row Cache Implementation
[ 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] [Updated] (CASSANDRA-2864) Alternative Row Cache Implementation
[ 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