Updated Branches: refs/heads/cassandra-2.0 ab7fed683 -> d2c67a1cb refs/heads/trunk 02eacf93a -> 2c65393d4
r/m code to load deprecated, pre-1.2 cache format patch by jbellis; reviewed by Vijay for CASSANDRA-6031 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d2c67a1c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d2c67a1c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d2c67a1c Branch: refs/heads/cassandra-2.0 Commit: d2c67a1cb95264e8a839bdcfc0a727c892f1fc1d Parents: ab7fed6 Author: Jonathan Ellis <[email protected]> Authored: Fri Sep 13 19:02:21 2013 -0400 Committer: Jonathan Ellis <[email protected]> Committed: Fri Sep 13 19:02:34 2013 -0400 ---------------------------------------------------------------------- .../apache/cassandra/cache/AutoSavingCache.java | 33 +------------------- .../apache/cassandra/service/CacheService.java | 26 --------------- 2 files changed, 1 insertion(+), 58 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2c67a1c/src/java/org/apache/cassandra/cache/AutoSavingCache.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/cache/AutoSavingCache.java b/src/java/org/apache/cassandra/cache/AutoSavingCache.java index 61ac3a5..dea105d 100644 --- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java +++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java @@ -104,36 +104,8 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K int count = 0; long start = System.nanoTime(); - // old cache format that only saves keys - File path = getCachePath(cfs.keyspace.getName(), cfs.name, null); - if (path.exists()) - { - DataInputStream in = null; - try - { - logger.info(String.format("reading saved cache %s", path)); - in = new DataInputStream(new LengthAvailableInputStream(new BufferedInputStream(new FileInputStream(path)), path.length())); - Set<ByteBuffer> keys = new HashSet<ByteBuffer>(); - while (in.available() > 0) - { - keys.add(ByteBufferUtil.readWithLength(in)); - count++; - } - cacheLoader.load(keys, cfs); - } - catch (Exception e) - { - logger.debug(String.format("harmless error reading saved cache %s fully, keys loaded so far: %d", path.getAbsolutePath(), count), e); - return count; - } - finally - { - FileUtils.closeQuietly(in); - } - } - // modern format, allows both key and value (so key cache load can be purely sequential) - path = getCachePath(cfs.keyspace.getName(), cfs.name, CURRENT_VERSION); + File path = getCachePath(cfs.keyspace.getName(), cfs.name, CURRENT_VERSION); if (path.exists()) { DataInputStream in = null; @@ -314,8 +286,5 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K void serialize(K key, DataOutput out) throws IOException; Future<Pair<K, V>> deserialize(DataInputStream in, ColumnFamilyStore cfs) throws IOException; - - @Deprecated - void load(Set<ByteBuffer> buffer, ColumnFamilyStore cfs); } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2c67a1c/src/java/org/apache/cassandra/service/CacheService.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/CacheService.java b/src/java/org/apache/cassandra/service/CacheService.java index b787b75..db95717 100644 --- a/src/java/org/apache/cassandra/service/CacheService.java +++ b/src/java/org/apache/cassandra/service/CacheService.java @@ -305,17 +305,6 @@ public class CacheService implements CacheServiceMBean } }); } - - public void load(Set<ByteBuffer> buffers, ColumnFamilyStore cfs) - { - for (ByteBuffer key : buffers) - { - DecoratedKey dk = cfs.partitioner.decorateKey(key); - ColumnFamily data = cfs.getTopLevelColumns(QueryFilter.getIdentityFilter(dk, cfs.name, Long.MIN_VALUE), Integer.MIN_VALUE); - if (data != null) - rowCache.put(new RowCacheKey(cfs.metadata.cfId, dk), data); - } - } } public class KeyCacheSerializer implements CacheSerializer<KeyCacheKey, RowIndexEntry> @@ -356,20 +345,5 @@ public class CacheService implements CacheServiceMBean } return null; } - - public void load(Set<ByteBuffer> buffers, ColumnFamilyStore cfs) - { - for (ByteBuffer key : buffers) - { - DecoratedKey dk = cfs.partitioner.decorateKey(key); - - for (SSTableReader sstable : cfs.getSSTables()) - { - RowIndexEntry entry = sstable.getPosition(dk, Operator.EQ, false); - if (entry != null) - keyCache.put(new KeyCacheKey(sstable.descriptor, key), entry); - } - } - } } }
