Author: jbellis
Date: Sat Feb 27 15:13:55 2010
New Revision: 916960
URL: http://svn.apache.org/viewvc?rev=916960&view=rev
Log:
Migrate rowCache into SSTableTracker and recalculate cache size when sstable
collection changes
patch by Stu Hood and jbellis for CASSANDRA-801
Modified:
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableTracker.java
Modified:
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=916960&r1=916959&r2=916960&view=diff
==============================================================================
---
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
(original)
+++
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
Sat Feb 27 15:13:55 2010
@@ -38,8 +38,6 @@
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
-import org.apache.cassandra.cache.InstrumentedCache;
-import org.apache.cassandra.cache.JMXInstrumentedCache;
import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
import org.apache.cassandra.concurrent.NamedThreadFactory;
import org.apache.cassandra.config.DatabaseDescriptor;
@@ -111,8 +109,6 @@
/* active memtable associated with this ColumnFamilyStore. */
private Memtable memtable_;
- private final JMXInstrumentedCache<String, ColumnFamily> rowCache;
-
// TODO binarymemtable ops are not threadsafe (do they need to be?)
private AtomicReference<BinaryMemtable> binaryMemtable_;
@@ -188,11 +184,6 @@
}
ssTables_ = new SSTableTracker(table, columnFamilyName);
ssTables_.add(sstables);
-
- int rowCacheSize = DatabaseDescriptor.getRowsCachedFor(table,
columnFamilyName, ssTables_.estimatedKeys());
- if (logger_.isDebugEnabled())
- logger_.debug("row cache capacity for " + columnFamilyName + " is
" + rowCacheSize);
- rowCache = new JMXInstrumentedCache<String, ColumnFamily>(table,
columnFamilyName + "RowCache", rowCacheSize);
}
public static ColumnFamilyStore createColumnFamilyStore(String table,
String columnFamily) throws IOException
@@ -701,12 +692,12 @@
private ColumnFamily cacheRow(String key) throws IOException
{
ColumnFamily cached;
- if ((cached = rowCache.get(key)) == null)
+ if ((cached = ssTables_.getRowCache().get(key)) == null)
{
cached = getTopLevelColumns(new IdentityQueryFilter(key, new
QueryPath(columnFamily_)), Integer.MIN_VALUE);
if (cached == null)
return null;
- rowCache.put(key, cached);
+ ssTables_.getRowCache().put(key, cached);
}
return cached;
}
@@ -725,7 +716,7 @@
{
if (filter.path.superColumnName == null)
{
- if (rowCache.getCapacity() == 0)
+ if (ssTables_.getRowCache().getCapacity() == 0)
return removeDeleted(getTopLevelColumns(filter, gcBefore),
gcBefore);
ColumnFamily cached = cacheRow(filter.key);
@@ -738,7 +729,7 @@
// we are querying subcolumns of a supercolumn: fetch the
supercolumn with NQF, then filter in-memory.
ColumnFamily cf;
SuperColumn sc;
- if (rowCache.getCapacity() == 0)
+ if (ssTables_.getRowCache().getCapacity() == 0)
{
QueryFilter nameFilter = new NamesQueryFilter(filter.key, new
QueryPath(columnFamily_), filter.path.superColumnName);
cf = getTopLevelColumns(nameFilter, gcBefore);
@@ -1075,13 +1066,13 @@
/** raw cached row -- does not fetch the row if it is not present. not
counted in cache statistics. */
public ColumnFamily getRawCachedRow(String key)
{
- return rowCache.getCapacity() == 0 ? null : rowCache.getInternal(key);
+ return ssTables_.getRowCache().getCapacity() == 0 ? null :
ssTables_.getRowCache().getInternal(key);
}
void invalidateCachedRow(String key)
{
- if (rowCache != null)
- rowCache.remove(key);
+ if (ssTables_.getRowCache() != null)
+ ssTables_.getRowCache().remove(key);
}
public void forceMajorCompaction()
Modified:
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableTracker.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableTracker.java?rev=916960&r1=916959&r2=916960&view=diff
==============================================================================
---
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableTracker.java
(original)
+++
incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableTracker.java
Sat Feb 27 15:13:55 2010
@@ -27,6 +27,7 @@
import org.apache.cassandra.cache.JMXInstrumentedCache;
import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamily;
import org.apache.cassandra.db.DecoratedKey;
import org.apache.cassandra.utils.Pair;
@@ -44,6 +45,7 @@
private final String cfname;
private final JMXInstrumentedCache<Pair<String, DecoratedKey>,
SSTable.PositionSize> keyCache;
+ private final JMXInstrumentedCache<String, ColumnFamily> rowCache;
public SSTableTracker(String ksname, String cfname)
{
@@ -51,6 +53,7 @@
this.cfname = cfname;
sstables = Collections.emptySet();
keyCache = new JMXInstrumentedCache<Pair<String, DecoratedKey>,
SSTable.PositionSize>(ksname, cfname + "KeyCache", 0);
+ rowCache = new JMXInstrumentedCache<String, ColumnFamily>(ksname,
cfname + "RowCache", 0);
}
public synchronized void replace(Collection<SSTableReader> oldSSTables,
Iterable<SSTableReader> replacements) throws IOException
@@ -76,15 +79,7 @@
}
sstables = Collections.unmodifiableSet(sstablesNew);
-
- int keyCacheSize = DatabaseDescriptor.getKeysCachedFor(ksname, cfname,
estimatedKeys());
- 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);
- }
+ updateCacheSizes();
}
public synchronized void add(Iterable<SSTableReader> sstables)
@@ -105,6 +100,31 @@
replace(compacted, Collections.<SSTableReader>emptyList());
}
+ /**
+ * Resizes the key and row caches based on the current key estimate.
+ */
+ public synchronized void updateCacheSizes()
+ {
+ long keys = estimatedKeys();
+
+ 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);
+ }
+ }
+
// the modifiers create new, unmodifiable objects each time; the volatile
fences the assignment
// so we don't need any further synchronization for the common case here
public Set<SSTableReader> getSSTables()
@@ -127,6 +147,11 @@
sstables = Collections.emptySet();
}
+ public JMXInstrumentedCache<String, ColumnFamily> getRowCache()
+ {
+ return rowCache;
+ }
+
public long estimatedKeys()
{
long n = 0;