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 2011/02/10 18:08:43 UTC

svn commit: r1069468 - in /cassandra/branches/cassandra-0.7: CHANGES.txt src/java/org/apache/cassandra/db/ColumnFamilyStore.java src/java/org/apache/cassandra/io/sstable/SSTableReader.java

Author: jbellis
Date: Thu Feb 10 17:08:42 2011
New Revision: 1069468

URL: http://svn.apache.org/viewvc?rev=1069468&view=rev
Log:
copy DecoratedKey.key when inserting into caches
patch by mdennis; reviewed by jbellis for CASSANDRA-2102

Modified:
    cassandra/branches/cassandra-0.7/CHANGES.txt
    cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java

Modified: cassandra/branches/cassandra-0.7/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/CHANGES.txt?rev=1069468&r1=1069467&r2=1069468&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.7/CHANGES.txt Thu Feb 10 17:08:42 2011
@@ -63,6 +63,8 @@
  * bound hints CF throughput between 32M and 256M (CASSANDRA-2148)
  * continue starting when invalid saved cache entries are encountered
    (CASSANDRA-2076)
+ * copy DecoratedKey.key when inserting into caches to avoid retaining
+   a reference to the underlying buffer (CASSANDRA-2102)
 
 
 0.7.0-final

Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=1069468&r1=1069467&r2=1069468&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Thu Feb 10 17:08:42 2011
@@ -1149,7 +1149,8 @@ public class ColumnFamilyStore implement
                 }
             }
 
-            ssTables.getRowCache().put(key, cached);
+            // TFFT resuses the underlying buffer for the key
+            ssTables.getRowCache().put(new DecoratedKey(key.token, ByteBufferUtil.clone(key.key)), cached);
         }
         return cached;
     }

Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java?rev=1069468&r1=1069467&r2=1069468&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java Thu Feb 10 17:08:42 2011
@@ -307,7 +307,7 @@ public class SSTableReader extends SSTab
                     if (shouldAddEntry)
                         indexSummary.addEntry(decoratedKey, indexPosition);
                     if (cacheLoading && keysToLoadInCache.contains(decoratedKey))
-                        keyCache.put(new Pair<Descriptor, DecoratedKey>(descriptor, decoratedKey), dataPosition);
+                        cacheKey(decoratedKey, dataPosition);
                 }
 
                 indexSummary.incrementRowid();
@@ -417,7 +417,8 @@ public class SSTableReader extends SSTab
 
     public void cacheKey(DecoratedKey key, Long info)
     {
-        keyCache.put(new Pair<Descriptor, DecoratedKey>(descriptor, key), info);
+        //TFFT reuses the underlying buffer for the key
+        keyCache.put(new Pair<Descriptor, DecoratedKey>(descriptor, new DecoratedKey(key.token, ByteBufferUtil.clone(key.key))), info);
     }
 
     public Long getCachedPosition(DecoratedKey key)
@@ -481,7 +482,7 @@ public class SSTableReader extends SSTab
                             if (op == Operator.EQ)
                                 bloomFilterTracker.addTruePositive();
                             // store exact match for the key
-                            keyCache.put(unifiedKey, dataPosition);
+                            cacheKey(unifiedKey.right, dataPosition);
                         }
                         return dataPosition;
                     }