Author: jbellis
Date: Sun Sep 18 04:51:12 2011
New Revision: 1172198

URL: http://svn.apache.org/viewvc?rev=1172198&view=rev
Log:
remove unnecessary copy whenadding to row cache
patch by Yang; reviewed by jbellis for CASSANDRA-3223

Modified:
    cassandra/branches/cassandra-1.0.0/CHANGES.txt
    
cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java

Modified: cassandra/branches/cassandra-1.0.0/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0.0/CHANGES.txt?rev=1172198&r1=1172197&r2=1172198&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0.0/CHANGES.txt (original)
+++ cassandra/branches/cassandra-1.0.0/CHANGES.txt Sun Sep 18 04:51:12 2011
@@ -1,6 +1,7 @@
 1.0.0-rc1
  * Fix counting CFMetadata towards Memtable liveRatio (CASSANDRA-3023)
  * Kill server on wrapped OOME such as from FileChannel.map (CASSANDRA-3201)
+ * remove unnecessary copy when adding to row cache (CASSANDRA-3223)
 
 
 1.0.0-beta1

Modified: 
cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=1172198&r1=1172197&r2=1172198&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
 (original)
+++ 
cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
 Sun Sep 18 04:51:12 2011
@@ -1121,17 +1121,6 @@ public class ColumnFamilyStore implement
             if (cached == null)
                 return null;
 
-            if (!rowCache.isPutCopying())
-            {
-                // make a deep copy of column data so we don't keep references 
to direct buffers, which
-                // would prevent munmap post-compaction.
-                for (IColumn column : cached)
-                {
-                    cached.remove(column.name());
-                    cached.addColumn(column.localCopy(this));
-                }
-            }
-
             // avoid keeping a permanent reference to the original key buffer
             rowCache.put(new DecoratedKey(key.token, 
ByteBufferUtil.clone(key.key)), cached);
         }


Reply via email to