Author: jbellis
Date: Mon May 17 19:07:39 2010
New Revision: 945315

URL: http://svn.apache.org/viewvc?rev=945315&view=rev
Log:
do not recalculate cache capacity post-compaction if it's been manually 
modified.  patch by jbellis; reviewed by Ryan King for CASSANDRA-1079

Modified:
    cassandra/branches/cassandra-0.6/CHANGES.txt
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/cache/InstrumentedCache.java
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableTracker.java

Modified: cassandra/branches/cassandra-0.6/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/CHANGES.txt?rev=945315&r1=945314&r2=945315&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.6/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.6/CHANGES.txt Mon May 17 19:07:39 2010
@@ -18,6 +18,8 @@
    finished (CASSANDRA-1076)
  * added Collections.shuffle(splits) before returning the splits in 
    ColumnFamilyInputFormat (CASSANDRA-1096)
+ * do not recalculate cache capacity post-compaction if it's been manually 
+   modified (CASSANDRA-1079)
 
 
 0.6.1

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/cache/InstrumentedCache.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/cache/InstrumentedCache.java?rev=945315&r1=945314&r2=945315&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/cache/InstrumentedCache.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/cache/InstrumentedCache.java
 Mon May 17 19:07:39 2010
@@ -33,6 +33,7 @@ public class InstrumentedCache<K, V>
     private final AtomicLong hits = new AtomicLong(0);
     private final AtomicLong lastRequests = new AtomicLong(0);
     private final AtomicLong lastHits = new AtomicLong(0);
+    private volatile boolean capacityModified;
 
     public InstrumentedCache(int capacity)
     {
@@ -69,10 +70,16 @@ public class InstrumentedCache<K, V>
         return capacity;
     }
 
+    public boolean isCapacityModified()
+    {
+        return capacityModified;
+    }
+
     public void setCapacity(int capacity)
     {
         map.setCapacity(capacity);
         this.capacity = capacity;
+        capacityModified = true;
      }
 
     public int getSize()

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableTracker.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableTracker.java?rev=945315&r1=945314&r2=945315&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableTracker.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableTracker.java
 Mon May 17 19:07:39 2010
@@ -106,22 +106,28 @@ public class SSTableTracker implements I
     public synchronized void updateCacheSizes()
     {
         long keys = estimatedKeys();
-        
-        int keyCacheSize = DatabaseDescriptor.getKeysCachedFor(ksname, cfname, 
keys);
-        if (keyCacheSize != keyCache.getCapacity())
+
+        if (!keyCache.isCapacityModified())
         {
-            // update cache size for the new key volume
-            if (logger.isDebugEnabled())
-                logger.debug("key cache capacity for " + cfname + " is " + 
keyCacheSize);
-            keyCache.setCapacity(keyCacheSize);
+            int keyCacheSize = DatabaseDescriptor.getKeysCachedFor(ksname, 
cfname, keys);
+            if (keyCacheSize != keyCache.getCapacity())
+            {
+                // update cache size for the new key volume
+                if (logger.isDebugEnabled())
+                    logger.debug("key cache capacity for " + cfname + " is " + 
keyCacheSize);
+                keyCache.setCapacity(keyCacheSize);
+            }
         }
 
-        int rowCacheSize = DatabaseDescriptor.getRowsCachedFor(ksname, cfname, 
keys);
-        if (rowCacheSize != rowCache.getCapacity())
-        {   
-            if (logger.isDebugEnabled())
-                logger.debug("row cache capacity for " + cfname + " is " + 
rowCacheSize);
-            rowCache.setCapacity(rowCacheSize);
+        if (!rowCache.isCapacityModified())
+        {
+            int rowCacheSize = DatabaseDescriptor.getRowsCachedFor(ksname, 
cfname, keys);
+            if (rowCacheSize != rowCache.getCapacity())
+            {
+                if (logger.isDebugEnabled())
+                    logger.debug("row cache capacity for " + cfname + " is " + 
rowCacheSize);
+                rowCache.setCapacity(rowCacheSize);
+            }
         }
     }
 


Reply via email to