You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2012/10/05 18:48:35 UTC

git commit: clean up unnecessary copy on KeyCache lookup patch by jbellis; reviewed by yukim for CASSANDRA-4732

Updated Branches:
  refs/heads/trunk e232407ad -> e3f642b8c


clean up unnecessary copy on KeyCache lookup
patch by jbellis; reviewed by yukim for CASSANDRA-4732


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e3f642b8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e3f642b8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e3f642b8

Branch: refs/heads/trunk
Commit: e3f642b8c50db3af3f398d40dbad26f4c417eb16
Parents: e232407
Author: Jonathan Ellis <jb...@apache.org>
Authored: Fri Oct 5 11:47:18 2012 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Fri Oct 5 11:47:54 2012 -0500

----------------------------------------------------------------------
 .../org/apache/cassandra/cache/KeyCacheKey.java    |    3 ++
 .../apache/cassandra/io/sstable/SSTableReader.java |   19 +++++++--------
 2 files changed, 12 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3f642b8/src/java/org/apache/cassandra/cache/KeyCacheKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/KeyCacheKey.java b/src/java/org/apache/cassandra/cache/KeyCacheKey.java
index ef0db40..509599b 100644
--- a/src/java/org/apache/cassandra/cache/KeyCacheKey.java
+++ b/src/java/org/apache/cassandra/cache/KeyCacheKey.java
@@ -28,6 +28,9 @@ import org.apache.cassandra.utils.Pair;
 public class KeyCacheKey implements CacheKey
 {
     public final Descriptor desc;
+
+    // keeping an array instead of a ByteBuffer lowers the overhead of the key cache working set,
+    // without extra copies on lookup since client-provided key ByteBuffers will be array-backed already
     public final byte[] key;
 
     public KeyCacheKey(Descriptor desc, ByteBuffer key)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3f642b8/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
index a67c1ab..aee576e 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -84,7 +84,7 @@ public class SSTableReader extends SSTable
     private IndexSummary indexSummary;
     private Filter bf;
 
-    private InstrumentingCache<KeyCacheKey, RowIndexEntry> keyCache;
+    private final InstrumentingCache<KeyCacheKey, RowIndexEntry> keyCache = CacheService.instance.keyCache;
 
     private final BloomFilterTracker bloomFilterTracker = new BloomFilterTracker();
 
@@ -311,7 +311,6 @@ public class SSTableReader extends SSTable
 
     public void setTrackedBy(DataTracker tracker)
     {
-        keyCache = CacheService.instance.keyCache;
         deletingTask.setTracker(tracker);
     }
 
@@ -679,14 +678,14 @@ public class SSTableReader extends SSTable
     {
         CFMetaData.Caching caching = metadata.getCaching();
 
-        if (keyCache == null
-                || caching == CFMetaData.Caching.NONE
-                || caching == CFMetaData.Caching.ROWS_ONLY
-                || keyCache.getCapacity() == 0)
+        if (caching == CFMetaData.Caching.NONE
+            || caching == CFMetaData.Caching.ROWS_ONLY
+            || keyCache.getCapacity() == 0)
+        {
             return;
+        }
 
-        // avoid keeping a permanent reference to the original key buffer
-        KeyCacheKey cacheKey = new KeyCacheKey(descriptor, ByteBufferUtil.clone(key.key));
+        KeyCacheKey cacheKey = new KeyCacheKey(descriptor, key.key);
         logger.trace("Adding cache entry for {} -> {}", cacheKey, info);
         keyCache.put(cacheKey, info);
     }
@@ -698,7 +697,7 @@ public class SSTableReader extends SSTable
 
     private RowIndexEntry getCachedPosition(KeyCacheKey unifiedKey, boolean updateStats)
     {
-        if (keyCache != null && keyCache.getCapacity() > 0)
+        if (keyCache.getCapacity() > 0)
             return updateStats ? keyCache.get(unifiedKey) : keyCache.getInternal(unifiedKey);
         return null;
     }
@@ -798,7 +797,7 @@ public class SSTableReader extends SSTable
                     {
                         // read data position from index entry
                         RowIndexEntry indexEntry = RowIndexEntry.serializer.deserialize(in, descriptor.version);
-                        if (exactMatch && keyCache != null && keyCache.getCapacity() > 0 && updateCacheAndStats)
+                        if (exactMatch && updateCacheAndStats)
                         {
                             assert key instanceof DecoratedKey; // key can be == to the index key only if it's a true row key
                             DecoratedKey decoratedKey = (DecoratedKey)key;