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);
     }