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:01 UTC

svn commit: r902914 - in /incubator/cassandra/trunk: src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/io/ test/unit/org/apache/cassandra/io/

Author: jbellis
Date: Mon Jan 25 18:36:00 2010
New Revision: 902914

URL: http://svn.apache.org/viewvc?rev=902914&view=rev
Log:
use 0-capacity cache instead of null to indicate no caching; this means we don't need to worry about creating & destroying cache objects from JMX
patch by jbellis; reviewed by goffinet for CASSANDRA-708

Modified:
    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
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableAccessor.java

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=902914&r1=902913&r2=902914&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:00 2010
@@ -109,7 +109,7 @@
     /* active memtable associated with this ColumnFamilyStore. */
     private Memtable memtable_;
 
-    private JMXInstrumentedCache<String, ColumnFamily> rowCache;
+    private final JMXInstrumentedCache<String, ColumnFamily> rowCache;
 
     // TODO binarymemtable ops are not threadsafe (do they need to be?)
     private AtomicReference<BinaryMemtable> binaryMemtable_;
@@ -187,37 +187,31 @@
         ssTables_ = new SSTableTracker(sstables);
 
         double v = DatabaseDescriptor.getRowsCachedFraction(table, columnFamilyName);
-        if (v > 0)
-        {
-            int cacheSize;
-            if (v < 1)
-                cacheSize = Math.max(1, (int)(v * SSTableReader.estimatedKeys(columnFamilyName)));
-            else
-                cacheSize = (int)v;
-            if (logger_.isDebugEnabled())
-                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)
+        int cacheSize;
+        if (0 < v && v < 1)
+            cacheSize = Math.max(1, (int)(v * SSTableReader.estimatedKeys(columnFamilyName)));
+        else
+            cacheSize = (int)v;
+        if (logger_.isDebugEnabled())
+            logger_.debug("row cache capacity for " + columnFamilyName + " is " + cacheSize);
+        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>()
         {
-            // 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()
             {
-                public Iterator<InstrumentedCache> iterator()
+                final Iterator<SSTableReader> iter = ssTables_.iterator();
+                return new AbstractIterator<InstrumentedCache>()
                 {
-                    final Iterator<SSTableReader> iter = ssTables_.iterator();
-                    return new AbstractIterator<InstrumentedCache>()
+                    @Override
+                    protected InstrumentedCache computeNext()
                     {
-                        @Override
-                        protected InstrumentedCache computeNext()
-                        {
-                            return iter.hasNext() ? iter.next().getKeyCache() : endOfData();
-                        }
-                    };
-                }
-            }, table, columnFamilyName + "KeyCache");
-        }
+                        return iter.hasNext() ? iter.next().getKeyCache() : endOfData();
+                    }
+                };
+            }
+        }, table, columnFamilyName + "KeyCache");
     }
 
     public static ColumnFamilyStore createColumnFamilyStore(String table, String columnFamily) throws IOException
@@ -799,7 +793,7 @@
         {
             if (filter.path.superColumnName == null)
             {
-                if (rowCache == null)
+                if (rowCache.getCapacity() == 0)
                     return removeDeleted(getTopLevelColumns(filter, gcBefore), gcBefore);
 
                 ColumnFamily cached = cacheRow(filter.key);
@@ -812,7 +806,7 @@
             // we are querying subcolumns of a supercolumn: fetch the supercolumn with NQF, then filter in-memory.
             ColumnFamily cf;
             SuperColumn sc;
-            if (rowCache == null)
+            if (rowCache.getCapacity() == 0)
             {
                 QueryFilter nameFilter = new NamesQueryFilter(filter.key, new QueryPath(columnFamily_), filter.path.superColumnName);
                 cf = getTopLevelColumns(nameFilter, gcBefore);
@@ -1135,7 +1129,7 @@
     /** raw cached row -- does not fetch the row if it is not present */
     public ColumnFamily getRawCachedRow(String key)
     {
-        return rowCache == null ? null : rowCache.get(key);
+        return rowCache.getCapacity() == 0 ? null : rowCache.get(key);
     }
 
     void invalidateCachedRow(String key)

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=902914&r1=902913&r2=902914&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 25 18:36:00 2010
@@ -219,6 +219,7 @@
             throws IOException
     {
         super(filename, partitioner);
+        assert keyCache != null;
 
         if (DatabaseDescriptor.getIndexAccessMode() == DatabaseDescriptor.DiskAccessMode.mmap)
         {
@@ -288,7 +289,7 @@
 
     private SSTableReader(String filename, IPartitioner partitioner) throws IOException
     {
-        this(filename, partitioner, null, null, null, null);
+        this(filename, partitioner, null, null, null, SSTableReader.createKeyCache(0));
     }
 
     public List<KeyPosition> getIndexPositions()
@@ -385,7 +386,7 @@
     {
         if (!bf.isPresent(partitioner.convertToDiskFormat(decoratedKey)))
             return null;
-        if (keyCache != null)
+        if (keyCache.getCapacity() > 0)
         {
             PositionSize cachedPosition = keyCache.get(decoratedKey);
             if (cachedPosition != null)
@@ -446,7 +447,7 @@
                     {
                         info = new PositionSize(position, length() - position);
                     }
-                    if (keyCache != null)
+                    if (keyCache.getCapacity() > 0)
                         keyCache.put(decoratedKey, info);
                     return info;
                 }

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=902914&r1=902913&r2=902914&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 25 18:36:00 2010
@@ -152,9 +152,7 @@
         rename(filterFilename());
         path = rename(path); // important to do this last since index & filter file names are derived from it
 
-        InstrumentedCache<DecoratedKey, PositionSize> keyCache = cacheFraction > 0
-                                                                 ? SSTableReader.createKeyCache((int)(cacheFraction * keysWritten))
-                                                                 : null;
+        InstrumentedCache<DecoratedKey, PositionSize> keyCache = SSTableReader.createKeyCache((int)(cacheFraction * keysWritten));
         return new SSTableReader(path, partitioner, indexPositions, spannedIndexDataPositions, bf, keyCache);
     }
 

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableAccessor.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableAccessor.java?rev=902914&r1=902913&r2=902914&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableAccessor.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableAccessor.java Mon Jan 25 18:36:00 2010
@@ -27,7 +27,7 @@
     public static SSTableReader getSSTableReader(String filename, IPartitioner<?> partitioner)
     throws IOException
     {
-        SSTableReader sstable =  new SSTableReader(filename, partitioner, null, null, null, null);
+        SSTableReader sstable =  new SSTableReader(filename, partitioner, null, null, null, SSTableReader.createKeyCache(0));
         sstable.loadBloomFilter();
         sstable.loadIndexFile();
         return sstable;