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/18 16:51:01 UTC
svn commit: r900434 - in
/incubator/cassandra/trunk/src/java/org/apache/cassandra:
cache/JMXAggregatingCache.java db/ColumnFamilyStore.java
io/SSTableReader.java io/SSTableWriter.java
Author: jbellis
Date: Mon Jan 18 15:51:00 2010
New Revision: 900434
URL: http://svn.apache.org/viewvc?rev=900434&view=rev
Log:
add JMXAggregatingCache to present all the SSTR keycaches in one mbean
patch by jbellis; reviewed by goffinet for CASSANDRA-688
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/cache/JMXAggregatingCache.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java
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=900434&r1=900433&r2=900434&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 18 15:51:00 2010
@@ -1,5 +1,44 @@
package org.apache.cassandra.cache;
-public class JMXAggregatingCache
+public class JMXAggregatingCache implements JMXAggregatingCacheMBean
{
+ private final Iterable<InstrumentedCache> caches;
+
+ public JMXAggregatingCache(Iterable<InstrumentedCache> caches, String table, String name)
+ {
+ this.caches = caches;
+ AbstractCache.registerMBean(this, table, name);
+ }
+
+ public int getCapacity()
+ {
+ int capacity = 0;
+ for (InstrumentedCache cache : caches)
+ {
+ capacity += cache.getCapacity();
+ }
+ return capacity;
+ }
+
+ public int getSize()
+ {
+ int size = 0;
+ for (InstrumentedCache cache : caches)
+ {
+ size += cache.getSize();
+ }
+ return size;
+ }
+
+ public double getHitRate()
+ {
+ int n = 0;
+ double rate = 0;
+ for (InstrumentedCache cache : caches)
+ {
+ rate += cache.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=900434&r1=900433&r2=900434&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 18 15:51:00 2010
@@ -30,7 +30,9 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
+import com.google.common.collect.AbstractIterator;
import org.apache.cassandra.cache.InstrumentedCache;
+import org.apache.cassandra.cache.JMXAggregatingCache;
import org.apache.cassandra.cache.JMXInstrumentedCache;
import org.apache.cassandra.service.SliceRange;
import org.apache.log4j.Logger;
@@ -194,6 +196,26 @@
logger_.debug("enabling row cache for " + columnFamilyName + " with size " + cacheSize);
rowCache = new JMXInstrumentedCache<String, ColumnFamily>(table, columnFamilyName + "RowCache", cacheSize);
}
+
+ if (DatabaseDescriptor.getKeysCachedFraction(table, columnFamilyName) > 0)
+ {
+ // we don't need to keep a reference to the aggregator, just create it so it registers itself w/ JMX
+ new JMXAggregatingCache(new Iterable<InstrumentedCache>()
+ {
+ public Iterator<InstrumentedCache> iterator()
+ {
+ final Iterator<SSTableReader> iter = ssTables_.iterator();
+ return new AbstractIterator<InstrumentedCache>()
+ {
+ @Override
+ protected InstrumentedCache computeNext()
+ {
+ return iter.hasNext() ? iter.next().getKeyCache() : endOfData();
+ }
+ };
+ }
+ }, table, columnFamilyName + "KeyCache");
+ }
}
public static ColumnFamilyStore createColumnFamilyStore(String table, String columnFamily) throws IOException
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java?rev=900434&r1=900433&r2=900434&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java Mon Jan 18 15:51:00 2010
@@ -30,6 +30,7 @@
import org.apache.commons.lang.StringUtils;
+import org.apache.cassandra.cache.InstrumentedCache;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.utils.BloomFilter;
@@ -203,18 +204,18 @@
private final MappedByteBuffer[] buffers;
- public static ConcurrentLinkedHashMap<DecoratedKey, PositionSize> createKeyCache(int size)
+ public static InstrumentedCache<DecoratedKey, PositionSize> createKeyCache(int size)
{
- return ConcurrentLinkedHashMap.create(ConcurrentLinkedHashMap.EvictionPolicy.SECOND_CHANCE, size);
+ return new InstrumentedCache<DecoratedKey, PositionSize>(size);
}
- private ConcurrentLinkedHashMap<DecoratedKey, PositionSize> keyCache;
+ private InstrumentedCache<DecoratedKey, PositionSize> keyCache;
SSTableReader(String filename,
IPartitioner partitioner,
List<KeyPosition> indexPositions, Map<KeyPosition, PositionSize> spannedIndexDataPositions,
BloomFilter bloomFilter,
- ConcurrentLinkedHashMap<DecoratedKey, PositionSize> keyCache)
+ InstrumentedCache<DecoratedKey, PositionSize> keyCache)
throws IOException
{
super(filename, partitioner);
@@ -575,6 +576,11 @@
return ColumnFamily.create(getTableName(), getColumnFamilyName());
}
+ public InstrumentedCache<DecoratedKey, PositionSize> getKeyCache()
+ {
+ return keyCache;
+ }
+
public ICompactSerializer2<IColumn> getColumnSerializer()
{
return DatabaseDescriptor.getColumnFamilyType(getTableName(), getColumnFamilyName()).equals("Standard")
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java?rev=900434&r1=900433&r2=900434&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java Mon Jan 18 15:51:00 2010
@@ -27,6 +27,7 @@
import org.apache.log4j.Logger;
+import org.apache.cassandra.cache.InstrumentedCache;
import org.apache.cassandra.db.DecoratedKey;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.service.StorageService;
@@ -151,9 +152,9 @@
rename(filterFilename());
path = rename(path); // important to do this last since index & filter file names are derived from it
- ConcurrentLinkedHashMap<DecoratedKey, SSTableReader.PositionSize> keyCache = cacheFraction > 0
- ? SSTableReader.createKeyCache((int) (cacheFraction * keysWritten))
- : null;
+ InstrumentedCache<DecoratedKey, PositionSize> keyCache = cacheFraction > 0
+ ? SSTableReader.createKeyCache((int)(cacheFraction * keysWritten))
+ : null;
return new SSTableReader(path, partitioner, indexPositions, spannedIndexDataPositions, bf, keyCache);
}