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 2010/01/25 19:36:40 UTC
svn commit: r902917 - in
/incubator/cassandra/trunk/src/java/org/apache/cassandra:
cache/IAggregatableCacheProvider.java cache/JMXAggregatingCache.java
db/ColumnFamilyStore.java
Author: jbellis
Date: Mon Jan 25 18:36:39 2010
New Revision: 902917
URL: http://svn.apache.org/viewvc?rev=902917&view=rev
Log:
better division of aggregated capacity among sstable caches
patch by jbellis; reviewed by goffinet for CASSANDRA-708
Added:
incubator/cassandra/trunk/src/java/org/apache/cassandra/cache/IAggregatableCacheProvider.java (with props)
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/cache/JMXAggregatingCache.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
Added: incubator/cassandra/trunk/src/java/org/apache/cassandra/cache/IAggregatableCacheProvider.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/cache/IAggregatableCacheProvider.java?rev=902917&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/cache/IAggregatableCacheProvider.java (added)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/cache/IAggregatableCacheProvider.java Mon Jan 25 18:36:39 2010
@@ -0,0 +1,7 @@
+package org.apache.cassandra.cache;
+
+public interface IAggregatableCacheProvider<K, V>
+{
+ public InstrumentedCache<K, V> getCache();
+ public long getObjectCount();
+}
Propchange: incubator/cassandra/trunk/src/java/org/apache/cassandra/cache/IAggregatableCacheProvider.java
------------------------------------------------------------------------------
svn:eol-style = native
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/cache/JMXAggregatingCache.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/cache/JMXAggregatingCache.java?rev=902917&r1=902916&r2=902917&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/cache/JMXAggregatingCache.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/cache/JMXAggregatingCache.java Mon Jan 25 18:36:39 2010
@@ -1,40 +1,49 @@
package org.apache.cassandra.cache;
+import java.util.Iterator;
+
+import com.google.common.collect.AbstractIterator;
+
public class JMXAggregatingCache implements JMXAggregatingCacheMBean
{
- private final Iterable<InstrumentedCache> caches;
+ private final Iterable<IAggregatableCacheProvider> cacheProviders;
- public JMXAggregatingCache(Iterable<InstrumentedCache> caches, String table, String name)
+ public JMXAggregatingCache(Iterable<IAggregatableCacheProvider> caches, String table, String name)
{
- this.caches = caches;
+ this.cacheProviders = caches;
AbstractCache.registerMBean(this, table, name);
}
public int getCapacity()
{
int capacity = 0;
- for (InstrumentedCache cache : caches)
+ for (IAggregatableCacheProvider cacheProvider : cacheProviders)
{
- capacity += cache.getCapacity();
+ capacity += cacheProvider.getCache().getCapacity();
}
return capacity;
}
public void setCapacity(int capacity)
{
- double ratio = capacity / getCapacity();
- for (InstrumentedCache cache : caches)
+ long totalObjects = 0;
+ for (IAggregatableCacheProvider cacheProvider : cacheProviders)
+ {
+ totalObjects += cacheProvider.getObjectCount();
+ }
+ for (IAggregatableCacheProvider cacheProvider : cacheProviders)
{
- cache.setCapacity(Math.max(1, (int)(cache.getCapacity() * ratio)));
+ double ratio = ((double)cacheProvider.getObjectCount()) / totalObjects;
+ cacheProvider.getCache().setCapacity((int)(capacity * ratio));
}
}
public int getSize()
{
int size = 0;
- for (InstrumentedCache cache : caches)
+ for (IAggregatableCacheProvider cacheProvider : cacheProviders)
{
- size += cache.getSize();
+ size += cacheProvider.getCache().getSize();
}
return size;
}
@@ -43,9 +52,9 @@
{
int n = 0;
double rate = 0;
- for (InstrumentedCache cache : caches)
+ for (IAggregatableCacheProvider cacheProvider : cacheProviders)
{
- rate += cache.getHitRate();
+ rate += cacheProvider.getCache().getHitRate();
n++;
}
return rate / n;
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=902917&r1=902916&r2=902917&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Mon Jan 25 18:36:39 2010
@@ -31,6 +31,7 @@
import java.util.concurrent.atomic.AtomicReference;
import com.google.common.collect.AbstractIterator;
+import org.apache.cassandra.cache.IAggregatableCacheProvider;
import org.apache.cassandra.cache.InstrumentedCache;
import org.apache.cassandra.cache.JMXAggregatingCache;
import org.apache.cassandra.cache.JMXInstrumentedCache;
@@ -197,17 +198,33 @@
rowCache = new JMXInstrumentedCache<String, ColumnFamily>(table, columnFamilyName + "RowCache", cacheSize);
// we don't need to keep a reference to the key cache aggregator, just create it so it registers itself w/ JMX
- new JMXAggregatingCache(new Iterable<InstrumentedCache>()
+ new JMXAggregatingCache(new Iterable<IAggregatableCacheProvider>()
{
- public Iterator<InstrumentedCache> iterator()
+ public Iterator<IAggregatableCacheProvider> iterator()
{
final Iterator<SSTableReader> iter = ssTables_.iterator();
- return new AbstractIterator<InstrumentedCache>()
+ return new AbstractIterator<IAggregatableCacheProvider>()
{
@Override
- protected InstrumentedCache computeNext()
+ protected IAggregatableCacheProvider computeNext()
{
- return iter.hasNext() ? iter.next().getKeyCache() : endOfData();
+ if (!iter.hasNext())
+ return endOfData();
+
+ return new IAggregatableCacheProvider()
+ {
+ SSTableReader sstable = iter.next();
+
+ public InstrumentedCache getCache()
+ {
+ return sstable.getKeyCache();
+ }
+
+ public long getObjectCount()
+ {
+ return sstable.getIndexPositions().size() * SSTableReader.indexInterval();
+ }
+ };
}
};
}