Details

    • Type: Improvement Improvement
    • Status: Resolved
    • Priority: Minor Minor
    • Resolution: Fixed
    • Fix Version/s: 1.1.0
    • Component/s: None
    • Labels:

      Description

      Caches are difficult to configure well as ColumnFamilies are added, similar to how memtables were difficult pre-CASSANDRA-2006.

      1. 0003-final-fixes.patch
        11 kB
        Pavel Yaskevich
      2. 0002-fixes.patch
        23 kB
        Pavel Yaskevich
      3. 0001-CASSANDRA-3143-squashed.patch
        172 kB
        Pavel Yaskevich

        Issue Links

          Activity

          Hide
          Hudson added a comment -

          Integrated in Cassandra #1267 (See https://builds.apache.org/job/Cassandra/1267/)
          global key/row caches
          patch by Pavel Yaskevich; reviewed by Sylvain Lebresne for CASSANDRA-3143

          xedin : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1222715
          Files :

          • /cassandra/trunk/CHANGES.txt
          • /cassandra/trunk/NEWS.txt
          • /cassandra/trunk/conf/cassandra.yaml
          • /cassandra/trunk/doc/cql/CQL.textile
          • /cassandra/trunk/interface/cassandra.thrift
          • /cassandra/trunk/src/avro/internode.genavro
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/AutoSavingCache.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/AutoSavingKeyCache.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/AutoSavingRowCache.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/CacheKey.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/ConcurrentLinkedHashCache.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/ConcurrentLinkedHashCacheProvider.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/ICache.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/IRowCacheProvider.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/InstrumentingCache.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/InstrumentingCacheMBean.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/KeyCacheKey.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/RowCacheKey.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/SerializingCache.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cache/SerializingCacheProvider.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java
          • /cassandra/trunk/src/java/org/apache/cassandra/config/CFMetaData.java
          • /cassandra/trunk/src/java/org/apache/cassandra/config/Config.java
          • /cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
          • /cassandra/trunk/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java
          • /cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
          • /cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
          • /cassandra/trunk/src/java/org/apache/cassandra/db/DataTracker.java
          • /cassandra/trunk/src/java/org/apache/cassandra/db/DecoratedKey.java
          • /cassandra/trunk/src/java/org/apache/cassandra/db/Table.java
          • /cassandra/trunk/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
          • /cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
          • /cassandra/trunk/src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java
          • /cassandra/trunk/src/java/org/apache/cassandra/service/CacheService.java
          • /cassandra/trunk/src/java/org/apache/cassandra/service/CacheServiceMBean.java
          • /cassandra/trunk/src/java/org/apache/cassandra/service/GCInspector.java
          • /cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
          • /cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java
          • /cassandra/trunk/src/java/org/apache/cassandra/tools/NodeCmd.java
          • /cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java
          • /cassandra/trunk/src/java/org/apache/cassandra/utils/StatusLogger.java
          • /cassandra/trunk/src/resources/org/apache/cassandra/cli/CliHelp.yaml
          • /cassandra/trunk/test/unit/org/apache/cassandra/CleanupHelper.java
          • /cassandra/trunk/test/unit/org/apache/cassandra/SchemaLoader.java
          • /cassandra/trunk/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
          • /cassandra/trunk/test/unit/org/apache/cassandra/db/DefsTest.java
          • /cassandra/trunk/test/unit/org/apache/cassandra/db/KeyCacheTest.java
          • /cassandra/trunk/test/unit/org/apache/cassandra/db/RowCacheTest.java
          • /cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
          Show
          Hudson added a comment - Integrated in Cassandra #1267 (See https://builds.apache.org/job/Cassandra/1267/ ) global key/row caches patch by Pavel Yaskevich; reviewed by Sylvain Lebresne for CASSANDRA-3143 xedin : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1222715 Files : /cassandra/trunk/CHANGES.txt /cassandra/trunk/NEWS.txt /cassandra/trunk/conf/cassandra.yaml /cassandra/trunk/doc/cql/CQL.textile /cassandra/trunk/interface/cassandra.thrift /cassandra/trunk/src/avro/internode.genavro /cassandra/trunk/src/java/org/apache/cassandra/cache/AutoSavingCache.java /cassandra/trunk/src/java/org/apache/cassandra/cache/AutoSavingKeyCache.java /cassandra/trunk/src/java/org/apache/cassandra/cache/AutoSavingRowCache.java /cassandra/trunk/src/java/org/apache/cassandra/cache/CacheKey.java /cassandra/trunk/src/java/org/apache/cassandra/cache/ConcurrentLinkedHashCache.java /cassandra/trunk/src/java/org/apache/cassandra/cache/ConcurrentLinkedHashCacheProvider.java /cassandra/trunk/src/java/org/apache/cassandra/cache/ICache.java /cassandra/trunk/src/java/org/apache/cassandra/cache/IRowCacheProvider.java /cassandra/trunk/src/java/org/apache/cassandra/cache/InstrumentingCache.java /cassandra/trunk/src/java/org/apache/cassandra/cache/InstrumentingCacheMBean.java /cassandra/trunk/src/java/org/apache/cassandra/cache/KeyCacheKey.java /cassandra/trunk/src/java/org/apache/cassandra/cache/RowCacheKey.java /cassandra/trunk/src/java/org/apache/cassandra/cache/SerializingCache.java /cassandra/trunk/src/java/org/apache/cassandra/cache/SerializingCacheProvider.java /cassandra/trunk/src/java/org/apache/cassandra/cli/CliClient.java /cassandra/trunk/src/java/org/apache/cassandra/config/CFMetaData.java /cassandra/trunk/src/java/org/apache/cassandra/config/Config.java /cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java /cassandra/trunk/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java /cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java /cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java /cassandra/trunk/src/java/org/apache/cassandra/db/DataTracker.java /cassandra/trunk/src/java/org/apache/cassandra/db/DecoratedKey.java /cassandra/trunk/src/java/org/apache/cassandra/db/Table.java /cassandra/trunk/src/java/org/apache/cassandra/db/compaction/CompactionManager.java /cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java /cassandra/trunk/src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java /cassandra/trunk/src/java/org/apache/cassandra/service/CacheService.java /cassandra/trunk/src/java/org/apache/cassandra/service/CacheServiceMBean.java /cassandra/trunk/src/java/org/apache/cassandra/service/GCInspector.java /cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java /cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java /cassandra/trunk/src/java/org/apache/cassandra/tools/NodeCmd.java /cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java /cassandra/trunk/src/java/org/apache/cassandra/utils/StatusLogger.java /cassandra/trunk/src/resources/org/apache/cassandra/cli/CliHelp.yaml /cassandra/trunk/test/unit/org/apache/cassandra/CleanupHelper.java /cassandra/trunk/test/unit/org/apache/cassandra/SchemaLoader.java /cassandra/trunk/test/unit/org/apache/cassandra/cache/CacheProviderTest.java /cassandra/trunk/test/unit/org/apache/cassandra/db/DefsTest.java /cassandra/trunk/test/unit/org/apache/cassandra/db/KeyCacheTest.java /cassandra/trunk/test/unit/org/apache/cassandra/db/RowCacheTest.java /cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
          Hide
          Pavel Yaskevich added a comment -

          Committed.

          Show
          Pavel Yaskevich added a comment - Committed.
          Hide
          Jonathan Ellis added a comment -

          +1

          Show
          Jonathan Ellis added a comment - +1
          Hide
          Pavel Yaskevich added a comment -

          Thanks, Sylvain! I have created CASSANDRA-3667, will get to it as soon as I commit this one.

          Show
          Pavel Yaskevich added a comment - Thanks, Sylvain! I have created CASSANDRA-3667 , will get to it as soon as I commit this one.
          Hide
          Sylvain Lebresne added a comment -

          Last version lgtm, +1 (nit: I don't think the getCacheCapacityInBytes methods are too necessary when we already have it in MB).

          Show
          Sylvain Lebresne added a comment - Last version lgtm, +1 (nit: I don't think the getCacheCapacityInBytes methods are too necessary when we already have it in MB).
          Hide
          Pavel Yaskevich added a comment -

          rebased 0001 and 0002 + 0003 with final fixes

          Show
          Pavel Yaskevich added a comment - rebased 0001 and 0002 + 0003 with final fixes
          Hide
          Sylvain Lebresne added a comment -

          Alright, patch lgtm, +1. Great work Pavel.

          Just a few minor details that would be nice to do before committing:

          • As mentioned in the previous comments, currently when a row need to be read to be put in cache, CFS.cacheRow() decorates the key, which can be avoided just by making cacheRow take the DK and create the RowCacheKey internally.
          • We should rename setRowCacheCapacity to setRowCacheCapacityMB to match the others
          • It would be nice to move the cache stats from nodetool cfstats to nodetool info, rather than purely removing them
          • The saveCaches method still does not respect the cacheKeysToSave options

          And of course there is the question of disabling row caching on per-cf basis which, as said previously, I think is a must have before we release this (because any user that have at least one CF with wide rows (or that just happens to be a bad candidate for caching) will need it). So ok to do that post commit but let's put it at the top of the todo list then.

          Show
          Sylvain Lebresne added a comment - Alright, patch lgtm, +1. Great work Pavel. Just a few minor details that would be nice to do before committing: As mentioned in the previous comments, currently when a row need to be read to be put in cache, CFS.cacheRow() decorates the key, which can be avoided just by making cacheRow take the DK and create the RowCacheKey internally. We should rename setRowCacheCapacity to setRowCacheCapacityMB to match the others It would be nice to move the cache stats from nodetool cfstats to nodetool info, rather than purely removing them The saveCaches method still does not respect the cacheKeysToSave options And of course there is the question of disabling row caching on per-cf basis which, as said previously, I think is a must have before we release this (because any user that have at least one CF with wide rows (or that just happens to be a bad candidate for caching) will need it). So ok to do that post commit but let's put it at the top of the todo list then.
          Hide
          Pavel Yaskevich added a comment -

          separate patch with fixes attached.

          Show
          Pavel Yaskevich added a comment - separate patch with fixes attached.
          Hide
          Sylvain Lebresne added a comment -

          Oups, you're right, I was looking of the case of storing a new row in the rowCache, where the patch does decorate the key, and got a bit carried away. Especially so since even if the case of storing a row in the cache we can easily avoid the recomputation, it's just an inefficiency of the patch (that should be fixed).

          Show
          Sylvain Lebresne added a comment - Oups, you're right, I was looking of the case of storing a new row in the rowCache, where the patch does decorate the key, and got a bit carried away. Especially so since even if the case of storing a row in the cache we can easily avoid the recomputation, it's just an inefficiency of the patch (that should be fixed).
          Hide
          Jonathan Ellis added a comment -

          I'll also note for posterity that by removing the DK from the cache keys, we're trading off memory for cpu (since we have to redecorate for each use)

          Why is that? Shouldn't we be looking up by (cfid, raw key)? We already have the DK in the getPosition / getCachedPosition call tree, we shouldn't need to store it in the cache to avoid recomputing it.

          Show
          Jonathan Ellis added a comment - I'll also note for posterity that by removing the DK from the cache keys, we're trading off memory for cpu (since we have to redecorate for each use) Why is that? Shouldn't we be looking up by (cfid, raw key)? We already have the DK in the getPosition / getCachedPosition call tree, we shouldn't need to store it in the cache to avoid recomputing it.
          Hide
          Sylvain Lebresne added a comment -

          We're getting there

          A few more comments however:

          • When saving the cache, the 'writeKeyQuietly' make it possible to write a corrupted file (maybe only part of a key has been written) and has the risk of flooding the log (if one key throw an IOException, chances are the next one on this file will too). I think we'd have less problem just stop all saving if an IOError occurs.
          • In CFS.initRowCache(), the test {{ if (cachedRowsRead++ > rowCache.getCapacity()) }} is now not correct since getCapacity is a memory size. For now, I'm fine removing the test and say that 'if you reduce the size, you may not get you hottest keys loaded on startup'. Though ultimately we'll probably need to fix that.

          On what we expose through MBean:

          • I think I was preferring the old way of using the InstrumentingCacheMBean, rather than to have lots of {get,set}RowCache, {get,set}

            KeyCache method that forward to the InstrumentingCache ones. Basically I think it's more clear to have Cache->RowCache->infos and Cache->KeyCache->infos, rather than CacheServices->allInfos. It's also more easily extensible if we ever add some new cache.

          • In any case, we don't register the InstrumentingCacheMBean anymore and the CacheService one doesn't expose the hit rate nor the number of requests processed (and we can remove InstrumentingCacheMBean if we're not going to use it).
          • Both set {Key,Row}CacheSavePeriodInSeconds and saveCaches disregard the {Key,Row}

            CacheKeysToSave setting, while they probably shouldn't.

          • I'd rename getRowCacheCapacity() to getRwoCacheCapacityInMB() to match the set method.

          And a bunch of very minor nitpicks that I just happened to gather during the review:

          • In AutoSavingCache.saveCache, I'd log the "Deleting old files" message at DEBUG.
          • In DatabaseDescriptor, there's a wrongly placed import
          • In CFS, we should probably remove the getKeyCache method, to emphasis it's now a global thing. Same in DataTracker.
          • The comment from {{ data.addSSTables(sstables); // this will call updateCacheSizes() for us }} in CFS.loadNewSSTables is outdated
          • DK.java uselessly import RowCacheKey (and only have a very gratuitous codeStyle change btw).
          • I would rename the CacheService MBean name to more simply "org.apache.cassandra.db:type=Caches".

          I'll also note for posterity that by removing the DK from the cache keys, we're trading off memory for cpu (since we have to redecorate for each use). Don't get me wrong, it's likely a good trade-off, just wanted to write it down.

          Show
          Sylvain Lebresne added a comment - We're getting there A few more comments however: When saving the cache, the 'writeKeyQuietly' make it possible to write a corrupted file (maybe only part of a key has been written) and has the risk of flooding the log (if one key throw an IOException, chances are the next one on this file will too). I think we'd have less problem just stop all saving if an IOError occurs. In CFS.initRowCache(), the test {{ if (cachedRowsRead++ > rowCache.getCapacity()) }} is now not correct since getCapacity is a memory size. For now, I'm fine removing the test and say that 'if you reduce the size, you may not get you hottest keys loaded on startup'. Though ultimately we'll probably need to fix that. On what we expose through MBean: I think I was preferring the old way of using the InstrumentingCacheMBean, rather than to have lots of {get,set}RowCache, {get,set} KeyCache method that forward to the InstrumentingCache ones. Basically I think it's more clear to have Cache->RowCache->infos and Cache->KeyCache->infos, rather than CacheServices->allInfos. It's also more easily extensible if we ever add some new cache. In any case, we don't register the InstrumentingCacheMBean anymore and the CacheService one doesn't expose the hit rate nor the number of requests processed (and we can remove InstrumentingCacheMBean if we're not going to use it). Both set {Key,Row}CacheSavePeriodInSeconds and saveCaches disregard the {Key,Row} CacheKeysToSave setting, while they probably shouldn't. I'd rename getRowCacheCapacity() to getRwoCacheCapacityInMB() to match the set method. And a bunch of very minor nitpicks that I just happened to gather during the review: In AutoSavingCache.saveCache, I'd log the "Deleting old files" message at DEBUG. In DatabaseDescriptor, there's a wrongly placed import In CFS, we should probably remove the getKeyCache method, to emphasis it's now a global thing. Same in DataTracker. The comment from {{ data.addSSTables(sstables); // this will call updateCacheSizes() for us }} in CFS.loadNewSSTables is outdated DK.java uselessly import RowCacheKey (and only have a very gratuitous codeStyle change btw). I would rename the CacheService MBean name to more simply "org.apache.cassandra.db:type=Caches". I'll also note for posterity that by removing the DK from the cache keys, we're trading off memory for cpu (since we have to redecorate for each use). Don't get me wrong, it's likely a good trade-off, just wanted to write it down.
          Hide
          Pavel Yaskevich added a comment -

          squashed commit properly rebased with trunk.

          Show
          Pavel Yaskevich added a comment - squashed commit properly rebased with trunk.
          Hide
          Pavel Yaskevich added a comment -

          rebased and squashed.

          Show
          Pavel Yaskevich added a comment - rebased and squashed.
          Hide
          Sylvain Lebresne added a comment -

          This needs to be rebased. I really wouldn't mind if you were to squash all patches into just one btw.

          Show
          Sylvain Lebresne added a comment - This needs to be rebased. I really wouldn't mind if you were to squash all patches into just one btw.
          Hide
          Pavel Yaskevich added a comment -

          CacheKey could have a serializeSize() method for use rather needlessly creating ByteBuffers just to get there size in estimateSizeToSave.

          added serializedSize() method to CacheKey interface.

          CacheKey.serialize() is unused.

          removed serialize() in favor of serializeForStorage()

          Since that for saving each cache, we do n+1 iterations through the whole cache where n is the number of column families. It seems rather inefficient, we could probably write all caches (for all CFs) simultaneously for a more efficient process.

          Changed write to O by keeping writers cached.

          When reading the cache, it seems we decorate each key just to "validate saved data" (we discard the DK object afterwards). But I don't think decorating a key entails any kind of validation of the key so this feel useless.

          Fixed

          Do we care about reloading the row cache? Jonathan was right that reloading a cache is probably pretty useless.

          Key and Row cache reloading is not dropped. (key cache reloading was dropped by patch 7).

          Show
          Pavel Yaskevich added a comment - CacheKey could have a serializeSize() method for use rather needlessly creating ByteBuffers just to get there size in estimateSizeToSave. added serializedSize() method to CacheKey interface. CacheKey.serialize() is unused. removed serialize() in favor of serializeForStorage() Since that for saving each cache, we do n+1 iterations through the whole cache where n is the number of column families. It seems rather inefficient, we could probably write all caches (for all CFs) simultaneously for a more efficient process. Changed write to O by keeping writers cached. When reading the cache, it seems we decorate each key just to "validate saved data" (we discard the DK object afterwards). But I don't think decorating a key entails any kind of validation of the key so this feel useless. Fixed Do we care about reloading the row cache? Jonathan was right that reloading a cache is probably pretty useless. Key and Row cache reloading is not dropped. (key cache reloading was dropped by patch 7).
          Hide
          Pavel Yaskevich added a comment -

          When reading the cache, it seems we decorate each key just to "validate saved data" (we discard the DK object afterwards). But I don't think decorating a key entails any kind of validation of the key so this feel useless.

          We need a DecoratedKey there for key cache as SSTableReader operates on the DecoratedKey instances in load method.

          Show
          Pavel Yaskevich added a comment - When reading the cache, it seems we decorate each key just to "validate saved data" (we discard the DK object afterwards). But I don't think decorating a key entails any kind of validation of the key so this feel useless. We need a DecoratedKey there for key cache as SSTableReader operates on the DecoratedKey instances in load method.
          Hide
          Pavel Yaskevich added a comment -

          If you say so, I was just concerned about additional amount of fds open. Will change that in upcoming patch #9.

          Show
          Pavel Yaskevich added a comment - If you say so, I was just concerned about additional amount of fds open. Will change that in upcoming patch #9.
          Hide
          Sylvain Lebresne added a comment -

          I was thinking about that - it would require to keep all of the files open, do we want that?

          Is that a big deal? I suppose if you have 1000 CFs it's starting to be a big number of fds open, but even then it doesn't sound like such a big deal, especially given it's for a very short time. Or did you had other concerns in mind?

          Show
          Sylvain Lebresne added a comment - I was thinking about that - it would require to keep all of the files open, do we want that? Is that a big deal? I suppose if you have 1000 CFs it's starting to be a big number of fds open, but even then it doesn't sound like such a big deal, especially given it's for a very short time. Or did you had other concerns in mind?
          Hide
          Pavel Yaskevich added a comment -

          Since that for saving each cache, we do n+1 iterations through the whole cache where n is the number of column families. It seems rather inefficient, we could probably write all caches (for all CFs) simultaneously for a more efficient process.

          I was thinking about that - it would require to keep all of the files open, do we want that?

          Do we care about reloading the row cache? Jonathan was right that reloading a cache is probably pretty useless.

          Ok, lets just drop it completely then.

          Show
          Pavel Yaskevich added a comment - Since that for saving each cache, we do n+1 iterations through the whole cache where n is the number of column families. It seems rather inefficient, we could probably write all caches (for all CFs) simultaneously for a more efficient process. I was thinking about that - it would require to keep all of the files open, do we want that? Do we care about reloading the row cache? Jonathan was right that reloading a cache is probably pretty useless. Ok, lets just drop it completely then.
          Hide
          Sylvain Lebresne added a comment -

          A few comments after quickly reading through the last patches:

          • CacheKey could have a serializeSize() method for use rather needlessly creating ByteBuffers just to get there size in estimateSizeToSave.
          • CacheKey.serialize() is unused.
          • Since that for saving each cache, we do n+1 iterations through the whole cache where n is the number of column families. It seems rather inefficient, we could probably write all caches (for all CFs) simultaneously for a more efficient process.
          • When reading the cache, it seems we decorate each key just to "validate saved data" (we discard the DK object afterwards). But I don't think decorating a key entails any kind of validation of the key so this feel useless.
          • Do we care about reloading the row cache? Jonathan was right that reloading a cache is probably pretty useless.
          Show
          Sylvain Lebresne added a comment - A few comments after quickly reading through the last patches: CacheKey could have a serializeSize() method for use rather needlessly creating ByteBuffers just to get there size in estimateSizeToSave. CacheKey.serialize() is unused. Since that for saving each cache, we do n+1 iterations through the whole cache where n is the number of column families. It seems rather inefficient, we could probably write all caches (for all CFs) simultaneously for a more efficient process. When reading the cache, it seems we decorate each key just to "validate saved data" (we discard the DK object afterwards). But I don't think decorating a key entails any kind of validation of the key so this feel useless. Do we care about reloading the row cache? Jonathan was right that reloading a cache is probably pretty useless.
          Hide
          Pavel Yaskevich added a comment -

          It's difficult to review this patchset because a lot of refactoring was introduced orthogonal to the new functionality. I'm specifically looking at the CacheKey heirarchy; the old "abstract translateKey" seemed like a reasonable compromise of flexibility without complexity to me. Granted that CacheKey may be an improvement, I'd like to do that kind of refactor in another ticket to simplify review of the new functionality.

          Personally I like CacheKey because it allows to remove AutoSaving

          {Key/Row}

          Cache classes and make code better generalized. But I agree that it makes it kind of hard to review...

          Do we actually benefit from caching DecoratedKey instead of byte[] ? DecoratedKey costs a lot of memory for small row keys. Since the cache is unordered I can't think of any benefit to using DK instead of byte[]. (Another reason I think CacheKey may be the wrong way to go.)

          I don't think we do. Attaching 0008 which changes

          {Row/Key}

          Cache to use raw (ByteBuffer) key instead of DecoratedKey.

          Show
          Pavel Yaskevich added a comment - It's difficult to review this patchset because a lot of refactoring was introduced orthogonal to the new functionality. I'm specifically looking at the CacheKey heirarchy; the old "abstract translateKey" seemed like a reasonable compromise of flexibility without complexity to me. Granted that CacheKey may be an improvement, I'd like to do that kind of refactor in another ticket to simplify review of the new functionality. Personally I like CacheKey because it allows to remove AutoSaving {Key/Row} Cache classes and make code better generalized. But I agree that it makes it kind of hard to review... Do we actually benefit from caching DecoratedKey instead of byte[] ? DecoratedKey costs a lot of memory for small row keys. Since the cache is unordered I can't think of any benefit to using DK instead of byte[]. (Another reason I think CacheKey may be the wrong way to go.) I don't think we do. Attaching 0008 which changes {Row/Key} Cache to use raw (ByteBuffer) key instead of DecoratedKey.
          Hide
          Jonathan Ellis added a comment -
          • It's difficult to review this patchset because a lot of refactoring was introduced orthogonal to the new functionality. I'm specifically looking at the CacheKey heirarchy; the old "abstract translateKey" seemed like a reasonable compromise of flexibility without complexity to me. Granted that CacheKey may be an improvement, I'd like to do that kind of refactor in another ticket to simplify review of the new functionality.
          • Do we actually benefit from caching DecoratedKey instead of byte[] ? DecoratedKey costs a lot of memory for small row keys. Since the cache is unordered I can't think of any benefit to using DK instead of byte[]. (Another reason I think CacheKey may be the wrong way to go.)
          Show
          Jonathan Ellis added a comment - It's difficult to review this patchset because a lot of refactoring was introduced orthogonal to the new functionality. I'm specifically looking at the CacheKey heirarchy; the old "abstract translateKey" seemed like a reasonable compromise of flexibility without complexity to me. Granted that CacheKey may be an improvement, I'd like to do that kind of refactor in another ticket to simplify review of the new functionality. Do we actually benefit from caching DecoratedKey instead of byte[] ? DecoratedKey costs a lot of memory for small row keys. Since the cache is unordered I can't think of any benefit to using DK instead of byte[]. (Another reason I think CacheKey may be the wrong way to go.)
          Hide
          Pavel Yaskevich added a comment -

          row/key caches are made backwards compatible and save only DecoratedKey plus tweaks/fixes from Sylvain's second comment.

          Show
          Pavel Yaskevich added a comment - row/key caches are made backwards compatible and save only DecoratedKey plus tweaks/fixes from Sylvain's second comment.
          Hide
          Pavel Yaskevich added a comment -

          Yeah, I guess this is the best way to go, I will remove #7 patch and re-attach with those changes to avoid pre-loading keys as well as keeping a global state.

          Show
          Pavel Yaskevich added a comment - Yeah, I guess this is the best way to go, I will remove #7 patch and re-attach with those changes to avoid pre-loading keys as well as keeping a global state.
          Hide
          Jonathan Ellis added a comment -

          Also note that while we have "one global cache" internally, there's nothing stopping us from splitting out the different CFs to different save files. In fact that would be great from a backwards compatibility point of view; there's users out there who would really hate to blow away their cache on upgrade, and preserving the save format would avoid the need for a backwards compatibility mode.

          Show
          Jonathan Ellis added a comment - Also note that while we have "one global cache" internally, there's nothing stopping us from splitting out the different CFs to different save files. In fact that would be great from a backwards compatibility point of view; there's users out there who would really hate to blow away their cache on upgrade, and preserving the save format would avoid the need for a backwards compatibility mode.
          Hide
          Jonathan Ellis added a comment -

          It's unhygenic. For the same reason that leaving fields to be initialized by a setFoo method later, is less hygenic than an immutable field that gets set in the constructor.

          Show
          Jonathan Ellis added a comment - It's unhygenic. For the same reason that leaving fields to be initialized by a setFoo method later, is less hygenic than an immutable field that gets set in the constructor.
          Hide
          Pavel Yaskevich added a comment -

          We are doing that now because we are able to read caches independently for each of the CFS, but with a global cache we would need to load that set on cache init and keep it through Schema.load as I global state, why wouldn't changing SSTableReader.getCachedPosition to return null (and delete that key) if value was -1, be a path of least resistance in this case?

          Show
          Pavel Yaskevich added a comment - We are doing that now because we are able to read caches independently for each of the CFS, but with a global cache we would need to load that set on cache init and keep it through Schema.load as I global state, why wouldn't changing SSTableReader.getCachedPosition to return null (and delete that key) if value was -1, be a path of least resistance in this case?
          Hide
          Jonathan Ellis added a comment -

          Again, that's what we're doing now, so I don't see it as that big a deal. But I'm good with either that approach, or save-the-values-also approach. I agree with Sylvain that keeping invalid values in the cache and replacing them later is a bad idea.

          Show
          Jonathan Ellis added a comment - Again, that's what we're doing now, so I don't see it as that big a deal. But I'm good with either that approach, or save-the-values-also approach. I agree with Sylvain that keeping invalid values in the cache and replacing them later is a bad idea.
          Hide
          Pavel Yaskevich added a comment -

          I'm not a fan of that because we would need to drag read keys through all of the CFS and SSTableReaders

          Show
          Pavel Yaskevich added a comment - I'm not a fan of that because we would need to drag read keys through all of the CFS and SSTableReaders
          Hide
          Jonathan Ellis added a comment -

          I'd rather go with the current approach of leaving the cache empty until we have real values for it, and pass SSTR a Set of keys-to-load.

          Show
          Jonathan Ellis added a comment - I'd rather go with the current approach of leaving the cache empty until we have real values for it, and pass SSTR a Set of keys-to-load.
          Hide
          Pavel Yaskevich added a comment -

          How about we just change SSTableReader.getCachedPosition to return null if value of the key cache was -1?

          Show
          Pavel Yaskevich added a comment - How about we just change SSTableReader.getCachedPosition to return null if value of the key cache was -1?
          Hide
          Sylvain Lebresne added a comment -

          Alright. I don't really care about cache reloading either actually. The only thing I don't like with the two phase approach is that it populate the cache with -1 positions. If for any reason, this doesn't get updated correctly, we'll end up having the cache wrongly saying that the key doesn't exists in the sstable. Of course there is no reason for the two phase approach to not work, but there is part of me that don't like that a simple mess up in the cache loading can make some keys unaccessible. Anyway, let's just not have bugs in there

          Show
          Sylvain Lebresne added a comment - Alright. I don't really care about cache reloading either actually. The only thing I don't like with the two phase approach is that it populate the cache with -1 positions. If for any reason, this doesn't get updated correctly, we'll end up having the cache wrongly saying that the key doesn't exists in the sstable. Of course there is no reason for the two phase approach to not work, but there is part of me that don't like that a simple mess up in the cache loading can make some keys unaccessible. Anyway, let's just not have bugs in there
          Hide
          Jonathan Ellis added a comment -

          I fail to see what is so crazy about having the function that saves the cache having access to both key and value. It may require a bit of refactoring, but I don't see that as a good argument. Anyway, it's not a very big deal but I still think that the two phase loading is more fragile than it needs, and saving values would allow a proper reload.

          Why would you want to do a cache reload? That's just going to be stale... Clearing the cache I can understand, but reloading a semi-arbitrary older cache state? I don't see the value there.

          ISTM we're talking about trading one kind of ugly code (passing around the Set of keys to load to SSTR) for another (a lot of code duplication between key cache, which wants to save values, and row cache, which doesn't). It's also worth pointing out that if we're concerned about cache size, the two-phase approach gives smaller saved caches. So I think I'd lean towards the existing, two-phase approach.

          Show
          Jonathan Ellis added a comment - I fail to see what is so crazy about having the function that saves the cache having access to both key and value. It may require a bit of refactoring, but I don't see that as a good argument. Anyway, it's not a very big deal but I still think that the two phase loading is more fragile than it needs, and saving values would allow a proper reload. Why would you want to do a cache reload? That's just going to be stale... Clearing the cache I can understand, but reloading a semi-arbitrary older cache state? I don't see the value there. ISTM we're talking about trading one kind of ugly code (passing around the Set of keys to load to SSTR) for another (a lot of code duplication between key cache, which wants to save values, and row cache, which doesn't). It's also worth pointing out that if we're concerned about cache size, the two-phase approach gives smaller saved caches. So I think I'd lean towards the existing, two-phase approach.
          Hide
          Jonathan Ellis added a comment -

          I'm talking of making the key cache save being 2 files (keySave and keySaveMetadata)

          So basically "compressing" the sstable filenames into an id?

          I'm actually okay with just brute-forcing it Pavel's way for now. We can always optimize later if it's a problem. (Could we just use a compressed writer for 80% of the benefit?)

          Show
          Jonathan Ellis added a comment - I'm talking of making the key cache save being 2 files (keySave and keySaveMetadata) So basically "compressing" the sstable filenames into an id? I'm actually okay with just brute-forcing it Pavel's way for now. We can always optimize later if it's a problem. (Could we just use a compressed writer for 80% of the benefit?)
          Hide
          Pavel Yaskevich added a comment -

          rebased set of patches, where all the changes from the second Sylvain's comment are in patch #7.

          Show
          Pavel Yaskevich added a comment - rebased set of patches, where all the changes from the second Sylvain's comment are in patch #7.
          Hide
          Sylvain Lebresne added a comment -

          On the other hand, if we will be using descriptor -> id relationship, wouldn't it create the same amount of additional I/O (+ expenses on such cache maintain) as just having Descriptor as cache key?

          I'm not talking of changing KeyCacheKey in any way. I'm talking of making the key cache save being 2 files (keySave and keySaveMetadata) having the key cache save method looking something like that (in pseudo-code):

          Map<Descriptor, Integer> assoc = new HashMap();
          int nextId = 0;
          for (KeyCacheKey key : keys)
          {
             Integer id = assoc.get(key.desc);
             if (id == null)
             {
                // descriptor we haven't seen yet during this save
                id = nextId++;
                assoc.put(key.desc, id);
                keySaveMetadata.write(key.desc => id)
             }
             keySave.write(id => key.key)
          }
          

          And the reading would first load the map from keySaveMetadata and read keySave.

          Sure, I will just change that to 40 bytes and update doc for key_cache_size_in_mb

          I would be in favor of not documenting this in the yaml as it'll confuse more user than anything else and we don't want anyone to rely on that formula.

          Show
          Sylvain Lebresne added a comment - On the other hand, if we will be using descriptor -> id relationship, wouldn't it create the same amount of additional I/O (+ expenses on such cache maintain) as just having Descriptor as cache key? I'm not talking of changing KeyCacheKey in any way. I'm talking of making the key cache save being 2 files (keySave and keySaveMetadata) having the key cache save method looking something like that (in pseudo-code): Map<Descriptor, Integer> assoc = new HashMap(); int nextId = 0; for (KeyCacheKey key : keys) { Integer id = assoc.get(key.desc); if (id == null) { // descriptor we haven't seen yet during this save id = nextId++; assoc.put(key.desc, id); keySaveMetadata.write(key.desc => id) } keySave.write(id => key.key) } And the reading would first load the map from keySaveMetadata and read keySave. Sure, I will just change that to 40 bytes and update doc for key_cache_size_in_mb I would be in favor of not documenting this in the yaml as it'll confuse more user than anything else and we don't want anyone to rely on that formula.
          Hide
          Pavel Yaskevich added a comment -

          At the very least, one easy win would be to save only the keyspace, columnFamily, version and generation part of the filename, rather than the whole path to the sstable. But otherwise, when I talked about a descriptor -> id relationship, I was thinking of something simple. Like saving two files instead of on, one would be the keys with the descriptor replaced by compact ids, the other would be the metadata, i.e, the descriptor -> id map. That would really just be some internal detail of the save function. But that's really just an idea.

          The problem with using only keyspace/cf/generate is that information is not sufficient to build descriptor back on readSaved. On the other hand, if we will be using descriptor -> id relationship, wouldn't it create the same amount of additional I/O (+ expenses on such cache maintain) as just having Descriptor as cache key?

          Yeah, I know . But for the key cache, we use a constant weighter, counting 8 bytes for each "entry". Figured we could use some higher constant to get closer to the actual size taken by each entry in-memory, even if we don't account for the exact size of the key. Typically, the KeyCacheKey structure will take "at least" 32 bytes in memory (it's more than that but given there is at least the DK token and a bunch of pointers...), so typically if we were to consider each entry to be like 40 or 48 bytes, I think we would be closer to the actual in-memory size. I just want to avoid people configuring 100MB for the key cache (ok, that would be a huge one) and actually having it being more like 1GB.

          Sure, I will just change that to 40 bytes and update doc for key_cache_size_in_mb with something like "please note that actual number of entries for given amount of space is calculated using following formula: key_cache_size_in_mb * 1024 * 1024 / 48 where 48 = 8 bytes (size of value) + 40 bytes (average size of the key)".

          Show
          Pavel Yaskevich added a comment - At the very least, one easy win would be to save only the keyspace, columnFamily, version and generation part of the filename, rather than the whole path to the sstable. But otherwise, when I talked about a descriptor -> id relationship, I was thinking of something simple. Like saving two files instead of on, one would be the keys with the descriptor replaced by compact ids, the other would be the metadata, i.e, the descriptor -> id map. That would really just be some internal detail of the save function. But that's really just an idea. The problem with using only keyspace/cf/generate is that information is not sufficient to build descriptor back on readSaved. On the other hand, if we will be using descriptor -> id relationship, wouldn't it create the same amount of additional I/O (+ expenses on such cache maintain) as just having Descriptor as cache key? Yeah, I know . But for the key cache, we use a constant weighter, counting 8 bytes for each "entry". Figured we could use some higher constant to get closer to the actual size taken by each entry in-memory, even if we don't account for the exact size of the key. Typically, the KeyCacheKey structure will take "at least" 32 bytes in memory (it's more than that but given there is at least the DK token and a bunch of pointers...), so typically if we were to consider each entry to be like 40 or 48 bytes, I think we would be closer to the actual in-memory size. I just want to avoid people configuring 100MB for the key cache (ok, that would be a huge one) and actually having it being more like 1GB. Sure, I will just change that to 40 bytes and update doc for key_cache_size_in_mb with something like "please note that actual number of entries for given amount of space is calculated using following formula: key_cache_size_in_mb * 1024 * 1024 / 48 where 48 = 8 bytes (size of value) + 40 bytes (average size of the key)".
          Hide
          Sylvain Lebresne added a comment -

          You're right, my bad, I looked at the wrong constructor in CFMetada and though they got a null cfId. Nevermind.

          Show
          Sylvain Lebresne added a comment - You're right, my bad, I looked at the wrong constructor in CFMetada and though they got a null cfId. Nevermind.
          Hide
          Pavel Yaskevich added a comment -

          Sorry, I was probably not clear. I really just meant replacing the two calls to Schema.instance.getId() by metadata.cfId. We would still have the next line be {{if (cfId == null) return; }} to avoid caching 2ndary indexes.

          What I'm trying to say is that SecondaryIndex CF has a metadata.cfId just like a normal CF so check for cfId == null would be pointless in that case but such CFs don't register with the Schema so Schema.instance.getId() will return null for SecondaryIndex CF.

          Show
          Pavel Yaskevich added a comment - Sorry, I was probably not clear. I really just meant replacing the two calls to Schema.instance.getId() by metadata.cfId. We would still have the next line be {{if (cfId == null) return; }} to avoid caching 2ndary indexes. What I'm trying to say is that SecondaryIndex CF has a metadata.cfId just like a normal CF so check for cfId == null would be pointless in that case but such CFs don't register with the Schema so Schema.instance.getId() will return null for SecondaryIndex CF.
          Hide
          Sylvain Lebresne added a comment -

          hat is why I'm saying that if we will use metadata.cfId we would cache 2L CFs without even noticing

          Sorry, I was probably not clear. I really just meant replacing the two calls to Schema.instance.getId() by metadata.cfId. We would still have the next line be {{if (cfId == null) return; }} to avoid caching 2ndary indexes.

          Show
          Sylvain Lebresne added a comment - hat is why I'm saying that if we will use metadata.cfId we would cache 2L CFs without even noticing Sorry, I was probably not clear. I really just meant replacing the two calls to Schema.instance.getId() by metadata.cfId . We would still have the next line be {{if (cfId == null) return; }} to avoid caching 2ndary indexes.
          Hide
          Pavel Yaskevich added a comment -

          I don't understand. I'm just saying that a cfs object already has a reference to its metadata, so it's slightly cleaner to use that rather that do query to Schema.instance using the table and column family name.

          In my previous comment I wrote "Secondary index cfs do not register with Schema.load so they don't have a cfId." and Jonathan replied "I don't think we should be trying to cache 2I CFs." that is why I'm saying that if we will use metadata.cfId we would cache 2L CFs without even noticing.

          But - save values to avoid 'two phrase' key cache loading - would require to use a common interface for values in key/row caches with serialize/deserialize functionality which is not suitable e.g. for ColumnFamily that we store in row cache... That is why we still rely on SSTableReader.load I think, saving values would limit flexibility of the cache interface...

          I fail to see what is so crazy about having the function that saves the cache having access to both key and value. It may require a bit of refactoring, but I don't see that as a good argument. Anyway, it's not a very big deal but I still think that the two phase loading is more fragile than it needs, and saving values would allow a proper reload.

          Jonathan can you please settle this argument?

          Show
          Pavel Yaskevich added a comment - I don't understand. I'm just saying that a cfs object already has a reference to its metadata, so it's slightly cleaner to use that rather that do query to Schema.instance using the table and column family name. In my previous comment I wrote "Secondary index cfs do not register with Schema.load so they don't have a cfId." and Jonathan replied "I don't think we should be trying to cache 2I CFs." that is why I'm saying that if we will use metadata.cfId we would cache 2L CFs without even noticing. But - save values to avoid 'two phrase' key cache loading - would require to use a common interface for values in key/row caches with serialize/deserialize functionality which is not suitable e.g. for ColumnFamily that we store in row cache... That is why we still rely on SSTableReader.load I think, saving values would limit flexibility of the cache interface... I fail to see what is so crazy about having the function that saves the cache having access to both key and value. It may require a bit of refactoring, but I don't see that as a good argument. Anyway, it's not a very big deal but I still think that the two phase loading is more fragile than it needs, and saving values would allow a proper reload. Jonathan can you please settle this argument?
          Hide
          Sylvain Lebresne added a comment -

          Couldn't we do that the same way we did with compression options? I'm happy to make it a sub-task, I just want the main code to be settled before starting with that.

          ok

          Is that going to have the same use case as it did per-CF? Meaning we would be saving a top of the cache and it doesn't guarantee that system doesn't start almost cold...

          Yes, it should really do exactly the same thing that the old option, except being global.

          Do you think that it worse the effort of maintaining (also persisting) such descriptor -> id relationship exclusively for key cache? Meaning it's already very compact cache e.g. even with descriptor > 50 bytes we would need ~20 mb to store 200000 keys...

          The thing is that 200000 keys is not necessary huge (especially given you can have more key cache entry than the total number of your keys since there is an entry per-sstable). And 50 bytes for each filename is also not even a worst case at all, especially when we have CASSANDRA-2749. And with say 1M keys, if each filename is 100 bytes, add the actual keys to that, we're talking > 100MB. Without being huge, it's a noticeable wast of I/O when the cache could easily be 10 times smaller. And if we add the values it will be worth.

          There is also the fact that I would be ashamed to have to explain to user that we save those full path to sstable with each entry when they complain that the key cache on disk is more than 10 times bigger that max size they configured in the yaml file.

          At the very least, one easy win would be to save only the keyspace, columnFamily, version and generation part of the filename, rather than the whole path to the sstable. But otherwise, when I talked about a descriptor -> id relationship, I was thinking of something simple. Like saving two files instead of on, one would be the keys with the descriptor replaced by compact ids, the other would be the metadata, i.e, the descriptor -> id map. That would really just be some internal detail of the save function. But that's really just an idea.

          We do that because CLHM only allows to measure values, to do something about it we would need to re-write Weighter interface and change core semantics of CLHM...

          Yeah, I know . But for the key cache, we use a constant weighter, counting 8 bytes for each "entry". Figured we could use some higher constant to get closer to the actual size taken by each entry in-memory, even if we don't account for the exact size of the key. Typically, the KeyCacheKey structure will take "at least" 32 bytes in memory (it's more than that but given there is at least the DK token and a bunch of pointers...), so typically if we were to consider each entry to be like 40 or 48 bytes, I think we would be closer to the actual in-memory size. I just want to avoid people configuring 100MB for the key cache (ok, that would be a huge one) and actually having it being more like 1GB.

          Another option would be to reuse the technique used to measure memtables, but I'm fine leaving that to another ticket.

          But - save values to avoid 'two phrase' key cache loading - would require to use a common interface for values in key/row caches with serialize/deserialize functionality which is not suitable e.g. for ColumnFamily that we store in row cache... That is why we still rely on SSTableReader.load I think, saving values would limit flexibility of the cache interface...

          I fail to see what is so crazy about having the function that saves the cache having access to both key and value. It may require a bit of refactoring, but I don't see that as a good argument. Anyway, it's not a very big deal but I still think that the two phase loading is more fragile than it needs, and saving values would allow a proper reload.

          This would mean that we will be caching even secondary index CFs which is, as was said, is not desired.

          I don't understand. I'm just saying that a cfs object already has a reference to its metadata, so it's slightly cleaner to use that rather that do query to Schema.instance using the table and column family name.

          Otherwise, forget to say that the patch adds the following useless line in
          CFS.java:

          /** Lock to allow migrations to block all flushing, so we can be sure not to write orphaned data files */
          public final Lock flushLock = new ReentrantLock();
          
          Show
          Sylvain Lebresne added a comment - Couldn't we do that the same way we did with compression options? I'm happy to make it a sub-task, I just want the main code to be settled before starting with that. ok Is that going to have the same use case as it did per-CF? Meaning we would be saving a top of the cache and it doesn't guarantee that system doesn't start almost cold... Yes, it should really do exactly the same thing that the old option, except being global. Do you think that it worse the effort of maintaining (also persisting) such descriptor -> id relationship exclusively for key cache? Meaning it's already very compact cache e.g. even with descriptor > 50 bytes we would need ~20 mb to store 200000 keys... The thing is that 200000 keys is not necessary huge (especially given you can have more key cache entry than the total number of your keys since there is an entry per-sstable). And 50 bytes for each filename is also not even a worst case at all, especially when we have CASSANDRA-2749 . And with say 1M keys, if each filename is 100 bytes, add the actual keys to that, we're talking > 100MB. Without being huge, it's a noticeable wast of I/O when the cache could easily be 10 times smaller. And if we add the values it will be worth. There is also the fact that I would be ashamed to have to explain to user that we save those full path to sstable with each entry when they complain that the key cache on disk is more than 10 times bigger that max size they configured in the yaml file. At the very least, one easy win would be to save only the keyspace, columnFamily, version and generation part of the filename, rather than the whole path to the sstable. But otherwise, when I talked about a descriptor -> id relationship, I was thinking of something simple. Like saving two files instead of on, one would be the keys with the descriptor replaced by compact ids, the other would be the metadata, i.e, the descriptor -> id map. That would really just be some internal detail of the save function. But that's really just an idea. We do that because CLHM only allows to measure values, to do something about it we would need to re-write Weighter interface and change core semantics of CLHM... Yeah, I know . But for the key cache, we use a constant weighter, counting 8 bytes for each "entry". Figured we could use some higher constant to get closer to the actual size taken by each entry in-memory, even if we don't account for the exact size of the key. Typically, the KeyCacheKey structure will take "at least" 32 bytes in memory (it's more than that but given there is at least the DK token and a bunch of pointers...), so typically if we were to consider each entry to be like 40 or 48 bytes, I think we would be closer to the actual in-memory size. I just want to avoid people configuring 100MB for the key cache (ok, that would be a huge one) and actually having it being more like 1GB. Another option would be to reuse the technique used to measure memtables, but I'm fine leaving that to another ticket. But - save values to avoid 'two phrase' key cache loading - would require to use a common interface for values in key/row caches with serialize/deserialize functionality which is not suitable e.g. for ColumnFamily that we store in row cache... That is why we still rely on SSTableReader.load I think, saving values would limit flexibility of the cache interface... I fail to see what is so crazy about having the function that saves the cache having access to both key and value. It may require a bit of refactoring, but I don't see that as a good argument. Anyway, it's not a very big deal but I still think that the two phase loading is more fragile than it needs, and saving values would allow a proper reload. This would mean that we will be caching even secondary index CFs which is, as was said, is not desired. I don't understand. I'm just saying that a cfs object already has a reference to its metadata, so it's slightly cleaner to use that rather that do query to Schema.instance using the table and column family name. Otherwise, forget to say that the patch adds the following useless line in CFS.java: /** Lock to allow migrations to block all flushing, so we can be sure not to write orphaned data files */ public final Lock flushLock = new ReentrantLock();
          Hide
          Pavel Yaskevich added a comment - - edited

          I don't think CacheService.reloadKeyCache works correctly. It only populate the cache with fake values that won't get updated unless you reload the sstables, which has no reason to happen. I'm fine removing the key cache reloading altogether, but as an alternative, why not save the value of the key cache too? The thing is, I'm not very comfortable with the current 'two phase' key cache loading: if we ever have a bug in the SSTReader.load method, the actual pre-loading with -1 values will be harmful, which seems unnecessarily fragile. Saving the values on disk would avoid that.

          Agreed, I just thought that it's valuable to have that key cache reload around, I'm going to remove it. But - save values to avoid 'two phrase' key cache loading - would require to use a common interface for values in key/row caches with serialize/deserialize functionality which is not suitable e.g. for ColumnFamily that we store in row cache... That is why we still rely on SSTableReader.load I think, saving values would limit flexibility of the cache interface...

          In CFS, it's probably faster/simpler to use metadata.cfId rather than Schema.instance.getId(table.name, this.columnFamily)

          This would mean that we will be caching even secondary index CFs which is, as was said, is not desired.

          In CacheService, calling scheduleSaving with -1 as second argument would be slightly faster than using Integer.MAX_VALUE.

          This could have even worse performance because it will change semantics and call hotKeySet method on the ICache, for CLHM this is not O(1) operation as doc for "descendingKeySetWithLimit(int limit);" says.

          I wonder if the reduce cache capacity thing still makes sense after this patch?

          I think it does because it also helps to reclaim some memory when system is starving.

          Show
          Pavel Yaskevich added a comment - - edited I don't think CacheService.reloadKeyCache works correctly. It only populate the cache with fake values that won't get updated unless you reload the sstables, which has no reason to happen. I'm fine removing the key cache reloading altogether, but as an alternative, why not save the value of the key cache too? The thing is, I'm not very comfortable with the current 'two phase' key cache loading: if we ever have a bug in the SSTReader.load method, the actual pre-loading with -1 values will be harmful, which seems unnecessarily fragile. Saving the values on disk would avoid that. Agreed, I just thought that it's valuable to have that key cache reload around, I'm going to remove it. But - save values to avoid 'two phrase' key cache loading - would require to use a common interface for values in key/row caches with serialize/deserialize functionality which is not suitable e.g. for ColumnFamily that we store in row cache... That is why we still rely on SSTableReader.load I think, saving values would limit flexibility of the cache interface... In CFS, it's probably faster/simpler to use metadata.cfId rather than Schema.instance.getId(table.name, this.columnFamily) This would mean that we will be caching even secondary index CFs which is, as was said, is not desired. In CacheService, calling scheduleSaving with -1 as second argument would be slightly faster than using Integer.MAX_VALUE. This could have even worse performance because it will change semantics and call hotKeySet method on the ICache, for CLHM this is not O(1) operation as doc for "descendingKeySetWithLimit(int limit);" says. I wonder if the reduce cache capacity thing still makes sense after this patch? I think it does because it also helps to reclaim some memory when system is starving.
          Hide
          Pavel Yaskevich added a comment -

          I'm not a fan of that idea. We just cannot release this without a way to deactivate the row cache as this would make the row cache unusable for most users. IMHO, that's a good definition of something that should not be moved to a separate task.

          Couldn't we do that the same way we did with compression options? I'm happy to make it a sub-task, I just want the main code to be settled before starting with that.

          Why does the getRowCacheKeysToSave() option disappeared?

          Is that going to have the same use case as it did per-CF? Meaning we would be saving a top of the cache and it doesn't guarantee that system doesn't start almost cold...

          Talking of the key cache save, the format used by the patch is really really not compact. For each key we save the full path to the sstable, which can easily be > 50 bytes. Maybe we could associate an int to each descriptor during the save and save the association of descriptor -> id separately. * Still worth allowing to chose how may keys to save

          Do you think that it worse the effort of maintaining (also persisting) such descriptor -> id relationship exclusively for key cache? Meaning it's already very compact cache e.g. even with descriptor > 50 bytes we would need ~20 mb to store 200000 keys...

          The cache sizings don't take the keys into account. For the row cache, one could make the argument that the overhead of the keys is negligible compared to the values. For the key cache however, the key are bigger than the values.

          We do that because CLHM only allows to measure values, to do something about it we would need to re-write Weighter interface and change core semantics of CLHM...

          Show
          Pavel Yaskevich added a comment - I'm not a fan of that idea. We just cannot release this without a way to deactivate the row cache as this would make the row cache unusable for most users. IMHO, that's a good definition of something that should not be moved to a separate task. Couldn't we do that the same way we did with compression options? I'm happy to make it a sub-task, I just want the main code to be settled before starting with that. Why does the getRowCacheKeysToSave() option disappeared? Is that going to have the same use case as it did per-CF? Meaning we would be saving a top of the cache and it doesn't guarantee that system doesn't start almost cold... Talking of the key cache save, the format used by the patch is really really not compact. For each key we save the full path to the sstable, which can easily be > 50 bytes. Maybe we could associate an int to each descriptor during the save and save the association of descriptor -> id separately. * Still worth allowing to chose how may keys to save Do you think that it worse the effort of maintaining (also persisting) such descriptor -> id relationship exclusively for key cache? Meaning it's already very compact cache e.g. even with descriptor > 50 bytes we would need ~20 mb to store 200000 keys... The cache sizings don't take the keys into account. For the row cache, one could make the argument that the overhead of the keys is negligible compared to the values. For the key cache however, the key are bigger than the values. We do that because CLHM only allows to measure values, to do something about it we would need to re-write Weighter interface and change core semantics of CLHM...
          Hide
          Sylvain Lebresne added a comment -

          Preceding point apart, we would at least need a way to deactivate row caching on a per-cf basis. We may also want that for key cache, though this seems less critical. My initial idea would be to either have a boolean flag if we only want to allow disabling row cache, or some multi-value caches option that could be "none", "key_only", "row_only" or "all".

          This is going to be moved to the separate task.

          I'm not a fan of that idea. We just cannot release this without a way to deactivate the row cache as this would make the row cache unusable for most users. IMHO, that's a good definition of something that should not be moved to a separate task.

          Why does the getRowCacheKeysToSave() option disappeared?

          Because we don't control that anymore, rely on cache LRU policy instead.

          I don't understand how "relying on cache LRU policy" has anything to do with that. The initial motivation for that option is that people don't want to reload the full extend of the row cache on restart because it takes forever, but they don't want to start with cold caches either. I don't see how making the cache global changes anything on that. I agree that the number of row cache key to save should now be a global option, but I disagree that it should be removed.

          Otherwise:

          • The code around CFS.prepareRowForCaching is weird. First the comment seems to suggest that prepareRowForCaching is used exclusively from CacheService while it's use below in cacheRow. It also adds a copy of the columns which I don't think is necessary since we already copy in MappedFileDataInput. Overall I'm not sure prepareRowForCaching is useful and CacheService.readSavedRowCache could use cacheRow directly
          • I don't think CacheService.reloadKeyCache works correctly. It only populate the cache with fake values that won't get updated unless you reload the sstables, which has no reason to happen. I'm fine removing the key cache reloading altogether, but as an alternative, why not save the value of the key cache too? The thing is, I'm not very comfortable with the current 'two phase' key cache loading: if we ever have a bug in the SSTReader.load method, the actual pre-loading with -1 values will be harmful, which seems unnecessarily fragile. Saving the values on disk would avoid that.
          • Talking of the key cache save, the format used by the patch is really really not compact. For each key we save the full path to the sstable, which can easily be > 50 bytes. Maybe we could associate an int to each descriptor during the save and save the association of descriptor -> id separately. * Still worth allowing to chose how may keys to save
          • The cache sizings don't take the keys into account. For the row cache, one could make the argument that the overhead of the keys is negligible compared to the values. For the key cache however, the key are bigger than the values.
          • The patch mistakenly remove the help for 'nodetool upgradesstables' (in NodeCmd.java)
          • Would be worth adding a global cache log line in StatusLogger.
          • Patch wrongly reintroduces memtable_operations and memtable_throughput to CliHelp.
          • The default row cache provider since 1.0 is the serializing one, this patch sets the ConcurrentLinkedHashCacheProvider instead.

          And a number of nits:

          • In CFS, it's probably faster/simpler to use metadata.cfId rather than Schema.instance.getId(table.name, this.columnFamily)
          • In CacheService, calling scheduleSaving with -1 as second argument would be slightly faster than using Integer.MAX_VALUE.
          • In SSTableReader.cacheKey, the assert key.key == null is useless in trunk (DK with key == null can't be constructed).
          • In AbstractCassandraDaemon, there's a unecessary import of javax.management.RuntimeErrorException
          • There is some comments duplication in the yaml file
          • I wonder if the reduce cache capacity thing still makes sense after this patch?
          • In AutosavingCache, I think we could declare AutoSavingCache<K extends CacheKey, V> and get rid of the translateKey() method
          Show
          Sylvain Lebresne added a comment - Preceding point apart, we would at least need a way to deactivate row caching on a per-cf basis. We may also want that for key cache, though this seems less critical. My initial idea would be to either have a boolean flag if we only want to allow disabling row cache, or some multi-value caches option that could be "none", "key_only", "row_only" or "all". This is going to be moved to the separate task. I'm not a fan of that idea. We just cannot release this without a way to deactivate the row cache as this would make the row cache unusable for most users. IMHO, that's a good definition of something that should not be moved to a separate task. Why does the getRowCacheKeysToSave() option disappeared? Because we don't control that anymore, rely on cache LRU policy instead. I don't understand how "relying on cache LRU policy" has anything to do with that. The initial motivation for that option is that people don't want to reload the full extend of the row cache on restart because it takes forever, but they don't want to start with cold caches either. I don't see how making the cache global changes anything on that. I agree that the number of row cache key to save should now be a global option, but I disagree that it should be removed. Otherwise: The code around CFS.prepareRowForCaching is weird. First the comment seems to suggest that prepareRowForCaching is used exclusively from CacheService while it's use below in cacheRow. It also adds a copy of the columns which I don't think is necessary since we already copy in MappedFileDataInput. Overall I'm not sure prepareRowForCaching is useful and CacheService.readSavedRowCache could use cacheRow directly I don't think CacheService.reloadKeyCache works correctly. It only populate the cache with fake values that won't get updated unless you reload the sstables, which has no reason to happen. I'm fine removing the key cache reloading altogether, but as an alternative, why not save the value of the key cache too? The thing is, I'm not very comfortable with the current 'two phase' key cache loading: if we ever have a bug in the SSTReader.load method, the actual pre-loading with -1 values will be harmful, which seems unnecessarily fragile. Saving the values on disk would avoid that. Talking of the key cache save, the format used by the patch is really really not compact. For each key we save the full path to the sstable, which can easily be > 50 bytes. Maybe we could associate an int to each descriptor during the save and save the association of descriptor -> id separately. * Still worth allowing to chose how may keys to save The cache sizings don't take the keys into account. For the row cache, one could make the argument that the overhead of the keys is negligible compared to the values. For the key cache however, the key are bigger than the values. The patch mistakenly remove the help for 'nodetool upgradesstables' (in NodeCmd.java) Would be worth adding a global cache log line in StatusLogger. Patch wrongly reintroduces memtable_operations and memtable_throughput to CliHelp. The default row cache provider since 1.0 is the serializing one, this patch sets the ConcurrentLinkedHashCacheProvider instead. And a number of nits: In CFS, it's probably faster/simpler to use metadata.cfId rather than Schema.instance.getId(table.name, this.columnFamily) In CacheService, calling scheduleSaving with -1 as second argument would be slightly faster than using Integer.MAX_VALUE. In SSTableReader.cacheKey, the assert key.key == null is useless in trunk (DK with key == null can't be constructed). In AbstractCassandraDaemon, there's a unecessary import of javax.management.RuntimeErrorException There is some comments duplication in the yaml file I wonder if the reduce cache capacity thing still makes sense after this patch? In AutosavingCache, I think we could declare AutoSavingCache<K extends CacheKey, V> and get rid of the translateKey() method
          Hide
          Pavel Yaskevich added a comment -

          rebased with the lastest trunk (last commit 58518301472fc99b01cfd4bcf90bf81b5f0694ee)

          Show
          Pavel Yaskevich added a comment - rebased with the lastest trunk (last commit 58518301472fc99b01cfd4bcf90bf81b5f0694ee)
          Hide
          Sylvain Lebresne added a comment -

          Would you mind rebasing ?

          Show
          Sylvain Lebresne added a comment - Would you mind rebasing ?
          Hide
          Pavel Yaskevich added a comment -

          RowCacheKey to use cfId instead of keyspace/cf pair (added to updated 0006 patch).

          Show
          Pavel Yaskevich added a comment - RowCacheKey to use cfId instead of keyspace/cf pair (added to updated 0006 patch).
          Hide
          Jonathan Ellis added a comment -

          Secondary index cfs do not register with Schema.load so they don't have a cfId.

          I don't think we should be trying to cache 2I CFs.

          Show
          Jonathan Ellis added a comment - Secondary index cfs do not register with Schema.load so they don't have a cfId. I don't think we should be trying to cache 2I CFs.
          Hide
          Pavel Yaskevich added a comment -

          And I forgot to mention that:

          • We probably need to have a story as far as upgrading to this patch is concerned (cleaning old saved caches, migrating to the new global options, ...). It's probably fine just leaving instructions in the NEWS file as a start, but I'd rather do it with the patch to avoid forgetting.

          Information was added to NEWS.txt

          • I think it'd be better to size the key cache by it's size in bytes rather than by number of entries, like for the row cache, since 'size alloted to the cache' is really the only measure that make sense for a user.

          cassandra.yaml option changed to key_cache_size_in_mb

          Show
          Pavel Yaskevich added a comment - And I forgot to mention that: We probably need to have a story as far as upgrading to this patch is concerned (cleaning old saved caches, migrating to the new global options, ...). It's probably fine just leaving instructions in the NEWS file as a start, but I'd rather do it with the patch to avoid forgetting. Information was added to NEWS.txt I think it'd be better to size the key cache by it's size in bytes rather than by number of entries, like for the row cache, since 'size alloted to the cache' is really the only measure that make sense for a user. cassandra.yaml option changed to key_cache_size_in_mb
          Hide
          Pavel Yaskevich added a comment -

          Preceding point apart, we would at least need a way to deactivate row caching on a per-cf basis. We may also want that for key cache, though this seems less critical. My initial idea would be to either have a boolean flag if we only want to allow disabling row cache, or some multi-value caches option that could be "none", "key_only", "row_only" or "all".

          This is going to be moved to the separate task.

          For the Row cache key, we should really use the cfId instead of table and cfname.

          Secondary index cfs do not register with Schema.load so they don't have a cfId.

          I think it would be worth factoring what can be of readSaved. In particular, we could create a KeyCacheKey (like RowCacheKey and that would really just be the usual pair of (Descriptor, DecoratedKey), have it share an interface with RowCacheKey (serialize and deserialize basically) and use that.

          Done.

          Putting the clone of the key into the constructor of RowCacheKey is inefficient, we don't care about cloning when we invalidate or query the cache (nor when we deserialize RowCacheKeys).

          Fixed.

          nit: not fond of declaration like public abstract Set<?> readSaved(), making it harder to know what the method returns just to save a few characters.

          Changed to Set<? extends CacheKey> as both key and row cache key now share the same interface - CacheKey.

          I believe the CacheService.instance will load the CacheService that will trigger an exception if there is a problem, not set the instance to null. So in particular, this message will never be written. That code is moved by the 4th patch but the problem remains I think (note that we do want to make sure CacheService get loaded quickly to throw an exception if we have an initialization problem, it's just not the right way I believe).

          This is the same as done in the StorageService constructor to make sure that StreamingService.instance is available when needed, because static fields are initialized on the first demand.

          I'm not super fond of that key cache preloading. If the key cache save is outdated, we'll have a bunch of uselessly preloaded stuff (not a huge deal but...). Maybe we could keep doing as before and just save the set of keys for each column family instead. That would needs buffering of the keys during the cache save though, but not sure it's such a huge deal and it would reduce the size of the saved cache quite a bit.

          The idea was to rely on cache LRU policy and save an actual global state of the cache to minimize cache's specific configuration per cf...

          In CacheService, setRowCacheSavePeriodInSecond directly schedule the saving to the new value, but the get method grabs the value from DatabaseDescriptor, so it will always return the initially set value, which is not what we want. I think we should keep the Integer that were previously in CFS (but I'm fine moving them to CacheService).

          Fixed.

          Why does the getRowCacheKeysToSave() option disappeared?

          Because we don't control that anymore, rely on cache LRU policy instead.

          The patch does the following change: " + int newCapacity = (int) (DatabaseDescriptor.getReduceCacheCapacityTo() * getCapacity());" ....

          This was unclear at first but fixed to use "weightedSize()" instead of "getCapacity()" now.

          I really think that making DecoratedKey equals method deal with RowCacheKey is asking for trouble. Not sure why it would be useful either?

          Fixed.

          Rebased with the latest trunk (last commit f76e9aeaed3e73bcfaf6bccd62f9f02f31b09960)

          Show
          Pavel Yaskevich added a comment - Preceding point apart, we would at least need a way to deactivate row caching on a per-cf basis. We may also want that for key cache, though this seems less critical. My initial idea would be to either have a boolean flag if we only want to allow disabling row cache, or some multi-value caches option that could be "none", "key_only", "row_only" or "all". This is going to be moved to the separate task. For the Row cache key, we should really use the cfId instead of table and cfname. Secondary index cfs do not register with Schema.load so they don't have a cfId. I think it would be worth factoring what can be of readSaved. In particular, we could create a KeyCacheKey (like RowCacheKey and that would really just be the usual pair of (Descriptor, DecoratedKey), have it share an interface with RowCacheKey (serialize and deserialize basically) and use that. Done. Putting the clone of the key into the constructor of RowCacheKey is inefficient, we don't care about cloning when we invalidate or query the cache (nor when we deserialize RowCacheKeys). Fixed. nit: not fond of declaration like public abstract Set<?> readSaved(), making it harder to know what the method returns just to save a few characters. Changed to Set<? extends CacheKey> as both key and row cache key now share the same interface - CacheKey. I believe the CacheService.instance will load the CacheService that will trigger an exception if there is a problem, not set the instance to null. So in particular, this message will never be written. That code is moved by the 4th patch but the problem remains I think (note that we do want to make sure CacheService get loaded quickly to throw an exception if we have an initialization problem, it's just not the right way I believe). This is the same as done in the StorageService constructor to make sure that StreamingService.instance is available when needed, because static fields are initialized on the first demand. I'm not super fond of that key cache preloading. If the key cache save is outdated, we'll have a bunch of uselessly preloaded stuff (not a huge deal but...). Maybe we could keep doing as before and just save the set of keys for each column family instead. That would needs buffering of the keys during the cache save though, but not sure it's such a huge deal and it would reduce the size of the saved cache quite a bit. The idea was to rely on cache LRU policy and save an actual global state of the cache to minimize cache's specific configuration per cf... In CacheService, setRowCacheSavePeriodInSecond directly schedule the saving to the new value, but the get method grabs the value from DatabaseDescriptor, so it will always return the initially set value, which is not what we want. I think we should keep the Integer that were previously in CFS (but I'm fine moving them to CacheService). Fixed. Why does the getRowCacheKeysToSave() option disappeared? Because we don't control that anymore, rely on cache LRU policy instead. The patch does the following change: " + int newCapacity = (int) (DatabaseDescriptor.getReduceCacheCapacityTo() * getCapacity());" .... This was unclear at first but fixed to use "weightedSize()" instead of "getCapacity()" now. I really think that making DecoratedKey equals method deal with RowCacheKey is asking for trouble. Not sure why it would be useful either? Fixed. Rebased with the latest trunk (last commit f76e9aeaed3e73bcfaf6bccd62f9f02f31b09960)
          Hide
          Sylvain Lebresne added a comment -

          I did a few pass over the patch. I haven't applied the patches yet (they need rebase anyway) and a few stuffs from the first patches is fixed in the later ones, which I realize only when reading those later patches, so even if I've tried to update my comments, there may be a few outdated ones, sorry about that.

          First, some "top-level" remarks/comments:

          • At least for the row cache, I fear this may sometimes be less efficient than what we have now, because some cf with less than good hit rate may evict rows of cf with very good hit rate, which wouldn't happen in the current implementation with reasonably tuned cache sizes. Aren't we screwing people that are doing fine tuning in the name of simplicity?
          • Preceding point apart, we would at least need a way to deactivate row caching on a per-cf basis. We may also want that for key cache, though this seems less critical. My initial idea would be to either have a boolean flag if we only want to allow disabling row cache, or some multi-value caches option that could be "none", "key_only", "row_only" or "all".
          • We probably need to have a story as far as upgrading to this patch is concerned (cleaning old saved caches, migrating to the new global options, ...). It's probably fine just leaving instructions in the NEWS file as a start, but I'd rather do it with the patch to avoid forgetting.
          • I think it'd be better to size the key cache by it's size in bytes rather than by number of entries, like for the row cache, since 'size alloted to the cache' is really the only measure that make sense for a user.

          Then a bit more specifically on some patches:

          patch 2:

          • For the Row cache key, we should really use the cfId instead of table and cfname.
          • I think it would be worth factoring what can be of readSaved. In particular, we could create a KeyCacheKey (like RowCacheKey and that would really just be the usual pair of (Descriptor, DecoratedKey), have it share an interface with RowCacheKey (serialize and deserialize basically) and use that.
          • Putting the clone of the key into the constructor of RowCacheKey is inefficient, we don't care about cloning when we invalidate or query the cache (nor when we deserialize RowCacheKeys).
          • nit: not fond of declaration like public abstract Set<?> readSaved(), making it harder to know what the method returns just to save a few characters.

          patch 3:

          • In DatabaseDescriptor:
                if (CacheService.instance == null)
                    logger.error("Could not initialize Cache Service.");
              

            I believe the CacheService.instance will load the CacheService that will trigger an exception if there is a problem, not set the instance to null. So in particular, this message will never be written. That code is moved by the 4th patch but the problem remains I think (note that we do want to make sure CacheService get loaded quickly to throw an exception if we have an
            initialization problem, it's just not the right way I believe).

          • I'm not super fond of that key cache preloading. If the key cache save is outdated, we'll have a bunch of uselessly preloaded stuff (not a huge deal but...). Maybe we could keep doing as before and just save the set of keys for each column family instead. That would needs buffering of the keys during the cache save though, but not sure it's such a huge deal and it would reduce the size of the saved cache quite a bit.
          • In CacheService, setRowCacheSavePeriodInSecond directly schedule the saving to the new value, but the get method grabs the value from DatabaseDescriptor, so it will always return the initially set value, which is not what we want. I think we should keep the Integer that were previously in CFS (but I'm fine moving them to CacheService).
          • Why does the getRowCacheKeysToSave() option disappeared?

          patch 4:

          • The patch does the following change:
              -            int newCapacity = (int) (DatabaseDescriptor.getReduceCacheCapacityTo() * size());
              +            int newCapacity = (int) (DatabaseDescriptor.getReduceCacheCapacityTo() * getCapacity());
              

            but that's not the semantic of the operation. The initial was the right one.

          • I really think that making DecoratedKey equals method deal with RowCacheKey is asking for trouble. Not sure why it would be useful either?
          Show
          Sylvain Lebresne added a comment - I did a few pass over the patch. I haven't applied the patches yet (they need rebase anyway) and a few stuffs from the first patches is fixed in the later ones, which I realize only when reading those later patches, so even if I've tried to update my comments, there may be a few outdated ones, sorry about that. First, some "top-level" remarks/comments: At least for the row cache, I fear this may sometimes be less efficient than what we have now, because some cf with less than good hit rate may evict rows of cf with very good hit rate, which wouldn't happen in the current implementation with reasonably tuned cache sizes. Aren't we screwing people that are doing fine tuning in the name of simplicity? Preceding point apart, we would at least need a way to deactivate row caching on a per-cf basis. We may also want that for key cache, though this seems less critical. My initial idea would be to either have a boolean flag if we only want to allow disabling row cache, or some multi-value caches option that could be "none", "key_only", "row_only" or "all". We probably need to have a story as far as upgrading to this patch is concerned (cleaning old saved caches, migrating to the new global options, ...). It's probably fine just leaving instructions in the NEWS file as a start, but I'd rather do it with the patch to avoid forgetting. I think it'd be better to size the key cache by it's size in bytes rather than by number of entries, like for the row cache, since 'size alloted to the cache' is really the only measure that make sense for a user. Then a bit more specifically on some patches: patch 2: For the Row cache key, we should really use the cfId instead of table and cfname. I think it would be worth factoring what can be of readSaved. In particular, we could create a KeyCacheKey (like RowCacheKey and that would really just be the usual pair of (Descriptor, DecoratedKey), have it share an interface with RowCacheKey (serialize and deserialize basically) and use that. Putting the clone of the key into the constructor of RowCacheKey is inefficient, we don't care about cloning when we invalidate or query the cache (nor when we deserialize RowCacheKeys). nit: not fond of declaration like public abstract Set<?> readSaved(), making it harder to know what the method returns just to save a few characters. patch 3: In DatabaseDescriptor: if (CacheService.instance == null) logger.error("Could not initialize Cache Service."); I believe the CacheService.instance will load the CacheService that will trigger an exception if there is a problem, not set the instance to null. So in particular, this message will never be written. That code is moved by the 4th patch but the problem remains I think (note that we do want to make sure CacheService get loaded quickly to throw an exception if we have an initialization problem, it's just not the right way I believe). I'm not super fond of that key cache preloading. If the key cache save is outdated, we'll have a bunch of uselessly preloaded stuff (not a huge deal but...). Maybe we could keep doing as before and just save the set of keys for each column family instead. That would needs buffering of the keys during the cache save though, but not sure it's such a huge deal and it would reduce the size of the saved cache quite a bit. In CacheService, setRowCacheSavePeriodInSecond directly schedule the saving to the new value, but the get method grabs the value from DatabaseDescriptor, so it will always return the initially set value, which is not what we want. I think we should keep the Integer that were previously in CFS (but I'm fine moving them to CacheService). Why does the getRowCacheKeysToSave() option disappeared? patch 4: The patch does the following change: - int newCapacity = (int) (DatabaseDescriptor.getReduceCacheCapacityTo() * size()); + int newCapacity = (int) (DatabaseDescriptor.getReduceCacheCapacityTo() * getCapacity()); but that's not the semantic of the operation. The initial was the right one. I really think that making DecoratedKey equals method deal with RowCacheKey is asking for trouble. Not sure why it would be useful either?
          Hide
          Pavel Yaskevich added a comment -

          rebased with latest trunk

          Show
          Pavel Yaskevich added a comment - rebased with latest trunk
          Hide
          Pavel Yaskevich added a comment -

          Please run `ant gen-thrift-java` and `ant avro-generate` right after you apply the set of patches.

          Show
          Pavel Yaskevich added a comment - Please run `ant gen-thrift-java` and `ant avro-generate` right after you apply the set of patches.
          Hide
          Pavel Yaskevich added a comment -

          rebased with latest trunk (last commit b9944399520c839f75725ec0f43e23fae5487ff4)

          Show
          Pavel Yaskevich added a comment - rebased with latest trunk (last commit b9944399520c839f75725ec0f43e23fae5487ff4)
          Hide
          Pavel Yaskevich added a comment -

          Comment from Jonathan:

          Also: we should size the global cache in bytes. There are two possibly approaches:

          Do it "manually" by estimating bytes from Key length plus CLHM overhead
          Use JAMM as we do for memtables
          Since the key cache is simpler than a memtable, the first approach might be workable here. But we know JAMM works reasonably well, so that's fine too.

          Hmm. I guess we could have the cache provider include a sizeInMemory method?

          For serialized off-heap cache we can just use the FreeableMemory size(). For on-heap cache we can use the serializedSize * liveRatio from the CF's memtable.

          Show
          Pavel Yaskevich added a comment - Comment from Jonathan: Also: we should size the global cache in bytes. There are two possibly approaches: Do it "manually" by estimating bytes from Key length plus CLHM overhead Use JAMM as we do for memtables Since the key cache is simpler than a memtable, the first approach might be workable here. But we know JAMM works reasonably well, so that's fine too. Hmm. I guess we could have the cache provider include a sizeInMemory method? For serialized off-heap cache we can just use the FreeableMemory size(). For on-heap cache we can use the serializedSize * liveRatio from the CF's memtable.
          Hide
          Pavel Yaskevich added a comment -

          updated my first comment, so CacheSegment will be designed to support get/put operations and flush/read from disk.

          Show
          Pavel Yaskevich added a comment - updated my first comment, so CacheSegment will be designed to support get/put operations and flush/read from disk.
          Hide
          Jonathan Ellis added a comment -

          But the point of making it a global cache is that per-CF cache sizes go away. Instead you rely on global LRU + weight to do the right thing (presumably, a better job than trying to manually size, anyway).

          Show
          Jonathan Ellis added a comment - But the point of making it a global cache is that per-CF cache sizes go away. Instead you rely on global LRU + weight to do the right thing (presumably, a better job than trying to manually size, anyway).
          Hide
          Pavel Yaskevich added a comment - - edited

          If you will modify capacity for segment it is going to store the value and signal global cache to change it's capacity appropriately. Useful when you want to play with row cache sizes for individual CFs.

          Show
          Pavel Yaskevich added a comment - - edited If you will modify capacity for segment it is going to store the value and signal global cache to change it's capacity appropriately. Useful when you want to play with row cache sizes for individual CFs.
          Hide
          Jonathan Ellis added a comment -

          But if it's One Big Cache how does modifying capacity per CF make sense?

          Show
          Jonathan Ellis added a comment - But if it's One Big Cache how does modifying capacity per CF make sense?
          Hide
          Pavel Yaskevich added a comment -

          The point is that we won't have any CFS-specific cache properties CacheSegment is going to hold everything - that will be useful in pair with JMX to modificate capacity and get cache statistics per CFS.

          Show
          Pavel Yaskevich added a comment - The point is that we won't have any CFS-specific cache properties CacheSegment is going to hold everything - that will be useful in pair with JMX to modificate capacity and get cache statistics per CFS.
          Hide
          Jonathan Ellis added a comment -

          I don't understand why we would have CFS-specific cache properties in a global cache.

          Show
          Jonathan Ellis added a comment - I don't understand why we would have CFS-specific cache properties in a global cache.
          Hide
          Pavel Yaskevich added a comment -

          So yes, CacheSegment is a segment of the cache which is allocated per ColumnFamilyStore. It is going to be used to put/get to global cache and to hold CFS specific properties of the cache such as current cache capacity and size (modification of those properties will be reflected on the global cache level so we don't need to worry about eviction policies). Also segment will be responsible to save/read cache to/from disk.

          Show
          Pavel Yaskevich added a comment - So yes, CacheSegment is a segment of the cache which is allocated per ColumnFamilyStore. It is going to be used to put/get to global cache and to hold CFS specific properties of the cache such as current cache capacity and size (modification of those properties will be reflected on the global cache level so we don't need to worry about eviction policies). Also segment will be responsible to save/read cache to/from disk.
          Hide
          Jonathan Ellis added a comment -

          All of this is straightforward with the exception of CacheSegment, can you elaborate on that part?

          Show
          Jonathan Ellis added a comment - All of this is straightforward with the exception of CacheSegment, can you elaborate on that part?
          Hide
          Pavel Yaskevich added a comment - - edited

          I think following steps are going to be appropriate to archive the goal:

          • row_cache_provider should be moved to global config
          • {key/row}

            _cache_save_period parameters should be moved to global config

          • {keys/rows}

            _cached should be moved to the global config

          • new o.a.c.service.CacheService instance to hold key and row caches as static members initialized at startup
            that service will be also responsible for cache save to disk operations.
          • new CacheSegment class should be issued from global cache for each of the ColumnFamilyStores registered in the system to be responsible for put/get operations, flush/read from disk.
          • cache sizes should be converted to memory units to support precise weighting of the cache using Weigher backed by Jamm
          Show
          Pavel Yaskevich added a comment - - edited I think following steps are going to be appropriate to archive the goal: row_cache_provider should be moved to global config {key/row} _cache_save_period parameters should be moved to global config {keys/rows} _cached should be moved to the global config new o.a.c.service.CacheService instance to hold key and row caches as static members initialized at startup that service will be also responsible for cache save to disk operations. new CacheSegment class should be issued from global cache for each of the ColumnFamilyStores registered in the system to be responsible for put/get operations, flush/read from disk. cache sizes should be converted to memory units to support precise weighting of the cache using Weigher backed by Jamm

            People

            • Assignee:
              Pavel Yaskevich
              Reporter:
              Pavel Yaskevich
              Reviewer:
              Sylvain Lebresne
            • Votes:
              2 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development