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/02/27 16:02:49 UTC

svn commit: r916957 - in /incubator/cassandra/branches/cassandra-0.6: conf/ src/java/org/apache/cassandra/config/ src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/io/ src/java/org/apache/cassandra/tools/ src/java/org/apache/cassandra/uti...

Author: jbellis
Date: Sat Feb 27 15:02:48 2010
New Revision: 916957

URL: http://svn.apache.org/viewvc?rev=916957&view=rev
Log:
Convert KeysCachedFraction to KeysCached, temporarily breaking things since it gives KeysCached per SSTable not per CF. Also refactors how key cache is instantiated, and removes SSTableAccessor from test suite.
patch by Stu Hood and jbellis for CASSANDRA-801

Modified:
    incubator/cassandra/branches/cassandra-0.6/conf/storage-conf.xml
    incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/CFMetaData.java
    incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
    incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/BinaryMemtable.java
    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/db/CompactionManager.java
    incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/Memtable.java
    incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableReader.java
    incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableWriter.java
    incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/tools/SSTableImport.java
    incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/utils/FBUtilities.java
    incubator/cassandra/branches/cassandra-0.6/test/conf/storage-conf.xml
    incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/io/SSTableUtils.java
    incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
    incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/tools/SSTableImportTest.java

Modified: incubator/cassandra/branches/cassandra-0.6/conf/storage-conf.xml
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/conf/storage-conf.xml?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/conf/storage-conf.xml (original)
+++ incubator/cassandra/branches/cassandra-0.6/conf/storage-conf.xml Sat Feb 27 15:02:48 2010
@@ -83,25 +83,22 @@
        ~ An optional `Comment` attribute may be used to attach additional
        ~ human-readable information about the column family to its definition.
        ~ 
-       ~ The optional KeysCachedFraction attribute specifies
-       ~ The fraction of keys per sstable whose locations we keep in
+       ~ The optional KeysCached attribute specifies
+       ~ the number of keys per sstable whose locations we keep in
        ~ memory in "mostly LRU" order.  (JUST the key locations, NOT any
-       ~ column values.) The amount of memory used by the default setting of 
-       ~ 0.01 is comparable to the amount used by the internal per-sstable key
-       ~ index. Consider increasing this if you have fewer, wider rows.
-       ~ Set to 0 to disable entirely.
+       ~ column values.) Specify a fraction (value less than 1), a percentage
+       ~ (ending in a % sign) or an absolute number of keys to cache. 
        ~
        ~ The optional RowsCached attribute specifies the number of rows
-       ~ whose entire contents we cache in memory, either as a fixed number
-       ~ of rows or as a percent of rows in the ColumnFamily.  
-       ~ Do not use this on ColumnFamilies with large rows, or
-       ~ ColumnFamilies with high write:read ratios.  As with key caching,
-       ~ valid values are from 0 to 1.  The default 0 disables it entirely.
+       ~ whose entire contents we cache in memory. Do not use this on
+       ~ ColumnFamilies with large rows, or ColumnFamilies with high write:read
+       ~ ratios. Specify a fraction (value less than 1), a percentage (ending in
+       ~ a % sign) or an absolute number of rows to cache. 
       -->
       <ColumnFamily CompareWith="BytesType" 
                     Name="Standard1" 
                     RowsCached="10%"
-                    KeysCachedFraction="0"/>
+                    KeysCached="0"/>
       <ColumnFamily CompareWith="UTF8Type" Name="Standard2"/>
       <ColumnFamily CompareWith="TimeUUIDType" Name="StandardByUUID1"/>
       <ColumnFamily ColumnType="Super"
@@ -109,7 +106,7 @@
                     CompareSubcolumnsWith="UTF8Type"
                     Name="Super1"
                     RowsCached="1000"
-                    KeysCachedFraction="0"
+                    KeysCached="50%"
                     Comment="A column family with supercolumns, whose column and subcolumn names are UTF8 strings"/>
 
       <!--

Modified: incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/CFMetaData.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/CFMetaData.java?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/CFMetaData.java (original)
+++ incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/CFMetaData.java Sat Feb 27 15:02:48 2010
@@ -29,6 +29,9 @@
 
 public final class CFMetaData
 {
+    public final static double DEFAULT_KEY_CACHE_SIZE = 0.1;
+    public final static double DEFAULT_ROW_CACHE_SIZE = 0.0;
+
     public final String tableName;            // name of table which has this column family
     public final String cfName;               // name of the column family
     public final String columnType;           // type: super, standard, etc.
@@ -36,9 +39,9 @@
     public final AbstractType subcolumnComparator; // like comparator, for supercolumns
     public final String comment; // for humans only
     public final double rowCacheSize; // default 0
-    public final double keysCachedFraction; // default 0.01
+    public final double keyCacheSize; // default 0.01
 
-    CFMetaData(String tableName, String cfName, String columnType, AbstractType comparator, AbstractType subcolumnComparator, String comment, double rowCacheSize, double keysCachedFraction)
+    CFMetaData(String tableName, String cfName, String columnType, AbstractType comparator, AbstractType subcolumnComparator, String comment, double rowCacheSize, double keyCacheSize)
     {
         this.tableName = tableName;
         this.cfName = cfName;
@@ -47,7 +50,7 @@
         this.subcolumnComparator = subcolumnComparator;
         this.comment = comment;
         this.rowCacheSize = rowCacheSize;
-        this.keysCachedFraction = keysCachedFraction;
+        this.keyCacheSize = keyCacheSize;
     }
 
     // a quick and dirty pretty printer for describing the column family...
@@ -73,7 +76,7 @@
         if (cfm.comment != null)
             dout.writeUTF(cfm.comment);
         dout.writeDouble(cfm.rowCacheSize);
-        dout.writeDouble(cfm.keysCachedFraction);
+        dout.writeDouble(cfm.keyCacheSize);
         dout.close();
         return bout.toByteArray();
 }
@@ -105,8 +108,8 @@
         }
         String comment = din.readBoolean() ? din.readUTF() : null;
         double rowCacheSize = din.readDouble();
-        double keysCachedFraction = din.readDouble();
-        return new CFMetaData(tableName, cfName, columnType, comparator, subcolumnComparator, comment, rowCacheSize, keysCachedFraction);
+        double keyCacheSize = din.readDouble();
+        return new CFMetaData(tableName, cfName, columnType, comparator, subcolumnComparator, comment, rowCacheSize, keyCacheSize);
     }
 
     public boolean equals(Object obj)
@@ -121,6 +124,6 @@
                 && FBUtilities.equals(other.subcolumnComparator, subcolumnComparator)
                 && FBUtilities.equals(other.comment, comment)
                 && other.rowCacheSize == rowCacheSize
-                && other.keysCachedFraction == keysCachedFraction;
+                && other.keyCacheSize == keyCacheSize;
     }
 }

Modified: incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/DatabaseDescriptor.java?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/DatabaseDescriptor.java (original)
+++ incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/config/DatabaseDescriptor.java Sat Feb 27 15:02:48 2010
@@ -26,10 +26,10 @@
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.locator.EndPointSnitch;
 import org.apache.cassandra.locator.IEndPointSnitch;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.XMLUtils;
 import org.apache.log4j.Logger;
 import org.w3c.dom.Node;
@@ -458,8 +458,8 @@
                                                                             new UTF8Type(),
                                                                             null,
                                                                             "persistent metadata for the local node",
-                                                                            0d,
-                                                                            0.01d));
+                                                                            0.0,
+                                                                            0.01));
 
             systemMeta.cfMetaData.put(HintedHandOffManager.HINTS_CF, new CFMetaData(Table.SYSTEM_TABLE,
                                                                                     HintedHandOffManager.HINTS_CF,
@@ -467,8 +467,8 @@
                                                                                     new UTF8Type(),
                                                                                     new BytesType(),
                                                                                     "hinted handoff data",
-                                                                                    0d,
-                                                                                    0.01d));
+                                                                                    0.0,
+                                                                                    0.01));
 
             /* Load the seeds for node contact points */
             String[] seeds = xmlUtils.getNodeValues("/Storage/Seeds/Seed");
@@ -643,23 +643,22 @@
                         throw new ConfigurationException("CompareSubcolumnsWith is only a valid attribute on super columnfamilies (not regular columnfamily " + cfName + ")");
                     }
 
-                    double keysCachedFraction = 0.01d;
+                    double keyCacheSize = CFMetaData.DEFAULT_KEY_CACHE_SIZE;
                     if ((value = XMLUtils.getAttributeValue(columnFamily, "KeysCachedFraction")) != null)
                     {
-                        keysCachedFraction = Double.valueOf(value);
+                        keyCacheSize = Double.valueOf(value);
+                        // TODO: KeysCachedFraction deprecated: remove in 1.0
+                        logger_.warn("KeysCachedFraction is deprecated: use KeysCached instead.");
                     }
-                    
-                    double rowCacheSize = 0;
+                    if ((value = XMLUtils.getAttributeValue(columnFamily, "KeysCached")) != null)
+                    {
+                        keyCacheSize = FBUtilities.parseDoubleOrPercent(value);
+                    }
+
+                    double rowCacheSize = CFMetaData.DEFAULT_ROW_CACHE_SIZE;
                     if ((value = XMLUtils.getAttributeValue(columnFamily, "RowsCached")) != null)
                     {
-                        if (value.endsWith("%"))
-                        {
-                            rowCacheSize = Double.valueOf(value.substring(0, value.length() - 1)) / 100;
-                        }
-                        else
-                        {
-                            rowCacheSize = Double.valueOf(value);
-                        }
+                        rowCacheSize = FBUtilities.parseDoubleOrPercent(value);
                     }
 
                     // Parse out user-specified logical names for the various dimensions
@@ -667,12 +666,12 @@
                     String comment = xmlUtils.getNodeValue(xqlCF + "Comment");
 
                     // insert it into the table dictionary.
-                    meta.cfMetaData.put(cfName, new CFMetaData(tableName, cfName, columnType, comparator, subcolumnComparator, comment, rowCacheSize, keysCachedFraction));
+                    meta.cfMetaData.put(cfName, new CFMetaData(tableName, cfName, columnType, comparator, subcolumnComparator, comment, rowCacheSize, keyCacheSize));
                 }
 
                 tables_.put(meta.name, meta);
             }
-            }
+        }
         catch (XPathExpressionException e)
         {
             ConfigurationException ex = new ConfigurationException(e.getMessage());
@@ -1084,20 +1083,24 @@
         return getCFMetaData(tableName, cfName).subcolumnComparator;
     }
 
-    public static double getKeysCachedFraction(String tableName, String columnFamilyName)
+    /**
+     * @return The absolute number of keys that should be cached per table.
+     */
+    public static int getKeysCachedFor(String tableName, String columnFamilyName, long expectedKeys)
     {
         CFMetaData cfm = getCFMetaData(tableName, columnFamilyName);
-        if (cfm == null)
-            return 0.01d;
-        return cfm.keysCachedFraction;
+        double v = (cfm == null) ? CFMetaData.DEFAULT_KEY_CACHE_SIZE : cfm.keyCacheSize;
+        return (int)Math.min(FBUtilities.absoluteFromFraction(v, expectedKeys), Integer.MAX_VALUE);
     }
 
-    public static double getRowsCachedFraction(String tableName, String columnFamilyName)
+    /**
+     * @return The absolute number of rows that should be cached for the columnfamily.
+     */
+    public static int getRowsCachedFor(String tableName, String columnFamilyName, long expectedRows)
     {
         CFMetaData cfm = getCFMetaData(tableName, columnFamilyName);
-        if (cfm == null)
-            return 0.01d;
-        return cfm.rowCacheSize;
+        double v = (cfm == null) ? CFMetaData.DEFAULT_ROW_CACHE_SIZE : cfm.rowCacheSize;
+        return (int)Math.min(FBUtilities.absoluteFromFraction(v, expectedRows), Integer.MAX_VALUE);
     }
 
     private static class ConfigurationException extends Exception

Modified: incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/BinaryMemtable.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/BinaryMemtable.java?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/BinaryMemtable.java (original)
+++ incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/BinaryMemtable.java Sat Feb 27 15:02:48 2010
@@ -128,7 +128,7 @@
             assert bytes.length > 0;
             writer.append(key, bytes);
         }
-        SSTableReader sstable = writer.closeAndOpenReader(DatabaseDescriptor.getKeysCachedFraction(cfs.getTable().name, cfs.getColumnFamilyName()));
+        SSTableReader sstable = writer.closeAndOpenReader();
         logger.info("Completed flushing " + writer.getFilename());
         return sstable;
     }

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=916957&r1=916956&r2=916957&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:02:48 2010
@@ -193,12 +193,7 @@
         ssTables_ = new SSTableTracker();
         ssTables_.add(sstables);
 
-        double v = DatabaseDescriptor.getRowsCachedFraction(table, columnFamilyName);
-        int cacheSize;
-        if (0 < v && v < 1)
-            cacheSize = Math.max(1, (int)(v * ssTables_.estimatedKeys()));
-        else
-            cacheSize = (int)v;
+        int cacheSize = DatabaseDescriptor.getRowsCachedFor(table, columnFamilyName, ssTables_.estimatedKeys());
         if (logger_.isDebugEnabled())
             logger_.debug("row cache capacity for " + columnFamilyName + " is " + cacheSize);
         rowCache = new JMXInstrumentedCache<String, ColumnFamily>(table, columnFamilyName + "RowCache", cacheSize);

Modified: incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/CompactionManager.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/CompactionManager.java?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/CompactionManager.java (original)
+++ incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/CompactionManager.java Sat Feb 27 15:02:48 2010
@@ -310,7 +310,7 @@
             ci.close();
         }
 
-        SSTableReader ssTable = writer.closeAndOpenReader(DatabaseDescriptor.getKeysCachedFraction(table.name, cfs.getColumnFamilyName()));
+        SSTableReader ssTable = writer.closeAndOpenReader();
         cfs.replaceCompactedSSTables(sstables, Arrays.asList(ssTable));
         submitMinorIfNeeded(cfs);
 
@@ -389,7 +389,7 @@
 
         if (writer != null)
         {
-            results.add(writer.closeAndOpenReader(DatabaseDescriptor.getKeysCachedFraction(table.name, cfs.getColumnFamilyName())));
+            results.add(writer.closeAndOpenReader());
             String format = "AntiCompacted to %s.  %d/%d bytes for %d keys.  Time: %dms.";
             long dTime = System.currentTimeMillis() - startTime;
             logger.info(String.format(format, writer.getFilename(), SSTable.getTotalBytes(sstables), results.get(0).length(), totalkeysWritten, dTime));

Modified: incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/Memtable.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/Memtable.java?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/Memtable.java (original)
+++ incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/Memtable.java Sat Feb 27 15:02:48 2010
@@ -158,7 +158,7 @@
             writer.append(entry.getKey(), buffer);
         }
 
-        SSTableReader ssTable = writer.closeAndOpenReader(DatabaseDescriptor.getKeysCachedFraction(getTableName(), cfs.getColumnFamilyName()));
+        SSTableReader ssTable = writer.closeAndOpenReader();
         logger.info("Completed flushing " + ssTable.getFilename());
         return ssTable;
     }

Modified: incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableReader.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableReader.java?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableReader.java (original)
+++ incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableReader.java Sat Feb 27 15:02:48 2010
@@ -31,7 +31,6 @@
 
 import org.apache.cassandra.cache.InstrumentedCache;
 import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.utils.BloomFilter;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -42,8 +41,6 @@
 import org.apache.cassandra.io.util.MappedFileDataInput;
 
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
 
 /**
  * SSTableReaders are open()ed by Table.onStart; after that they are created by SSTableWriter.renameAndOpen.
@@ -112,11 +109,10 @@
     public static SSTableReader open(String dataFileName) throws IOException
     {
         return open(dataFileName,
-                    StorageService.getPartitioner(),
-                    DatabaseDescriptor.getKeysCachedFraction(parseTableName(dataFileName), parseColumnFamilyName(dataFileName)));
+                    StorageService.getPartitioner());
     }
 
-    public static SSTableReader open(String dataFileName, IPartitioner partitioner, double keysCacheFraction) throws IOException
+    public static SSTableReader open(String dataFileName, IPartitioner partitioner) throws IOException
     {
         assert partitioner != null;
 
@@ -125,10 +121,9 @@
         logger.info("Sampling index for " + dataFileName);
         sstable.loadIndexFile();
         sstable.loadBloomFilter();
-        if (keysCacheFraction > 0)
-        {
-            sstable.keyCache = createKeyCache((int)((sstable.getIndexPositions().size() + 1) * INDEX_INTERVAL * keysCacheFraction));
-        }
+
+        long expectedKeys = (sstable.getIndexPositions().size() + 1) * INDEX_INTERVAL;
+        sstable.keyCache = createKeyCache(parseTableName(dataFileName), parseColumnFamilyName(dataFileName), expectedKeys);
         if (logger.isDebugEnabled())
             logger.debug("INDEX LOAD TIME for "  + dataFileName + ": " + (System.currentTimeMillis() - start) + " ms.");
 
@@ -141,9 +136,10 @@
     private final MappedByteBuffer[] buffers;
 
 
-    public static InstrumentedCache<DecoratedKey, PositionSize> createKeyCache(int size)
+    public static InstrumentedCache<DecoratedKey, PositionSize> createKeyCache(String ksname, String cfname, long expectedKeys)
     {
-        return new InstrumentedCache<DecoratedKey, PositionSize>(size);
+        int keysToCache = DatabaseDescriptor.getKeysCachedFor(ksname, cfname, expectedKeys);
+        return new InstrumentedCache<DecoratedKey, PositionSize>(keysToCache);
     }
 
     private InstrumentedCache<DecoratedKey, PositionSize> keyCache;
@@ -229,7 +225,7 @@
 
     private SSTableReader(String filename, IPartitioner partitioner) throws IOException
     {
-        this(filename, partitioner, null, null, null, SSTableReader.createKeyCache(0));
+        this(filename, partitioner, null, null, null, new InstrumentedCache<DecoratedKey, PositionSize>(0));
     }
 
     public List<KeyPosition> getIndexPositions()

Modified: incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableWriter.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableWriter.java?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableWriter.java (original)
+++ incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableWriter.java Sat Feb 27 15:02:48 2010
@@ -36,7 +36,6 @@
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.util.BufferedRandomAccessFile;
 import org.apache.cassandra.io.util.DataOutputBuffer;
-import com.reardencommerce.kernel.collections.shared.evictable.ConcurrentLinkedHashMap;
 
 public class SSTableWriter extends SSTable
 {
@@ -131,7 +130,7 @@
     /**
      * Renames temporary SSTable files to valid data, index, and bloom filter files
      */
-    public SSTableReader closeAndOpenReader(double cacheFraction) throws IOException
+    public SSTableReader closeAndOpenReader() throws IOException
     {
         // bloom filter
         FileOutputStream fos = new FileOutputStream(filterFilename());
@@ -152,7 +151,7 @@
         rename(filterFilename());
         path = rename(path); // important to do this last since index & filter file names are derived from it
 
-        InstrumentedCache<DecoratedKey, PositionSize> keyCache = SSTableReader.createKeyCache((int)(cacheFraction * keysWritten));
+        InstrumentedCache<DecoratedKey, PositionSize> keyCache = SSTableReader.createKeyCache(getTableName(), getColumnFamilyName(), keysWritten);
         return new SSTableReader(path, partitioner, indexPositions, spannedIndexDataPositions, bf, keyCache);
     }
 
@@ -176,8 +175,7 @@
         SSTableWriter.rename(filterFilename(dataFileName));
         dataFileName = SSTableWriter.rename(dataFileName);
         return SSTableReader.open(dataFileName,
-                                  StorageService.getPartitioner(),
-                                  DatabaseDescriptor.getKeysCachedFraction(parseTableName(dataFileName), parseColumnFamilyName(dataFileName)));
+                                  StorageService.getPartitioner());
     }
 
 }

Modified: incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/tools/SSTableImport.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/tools/SSTableImport.java?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/tools/SSTableImport.java (original)
+++ incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/tools/SSTableImport.java Sat Feb 27 15:02:48 2010
@@ -161,7 +161,7 @@
                 cfamily.clear();
             }
             
-            writer.closeAndOpenReader(0);
+            writer.closeAndOpenReader();
         }
         catch (ClassCastException cce)
         {
@@ -209,4 +209,4 @@
         System.exit(0);
     }
 
-}
\ No newline at end of file
+}

Modified: incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/utils/FBUtilities.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/utils/FBUtilities.java?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/utils/FBUtilities.java (original)
+++ incubator/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/utils/FBUtilities.java Sat Feb 27 15:02:48 2010
@@ -63,6 +63,21 @@
         return result.toArray( new String[0] );
     }
 
+    /**
+     * Parses a string representing either a fraction, absolute value or percentage.
+     */
+    public static double parseDoubleOrPercent(String value)
+    {
+        if (value.endsWith("%"))
+        {
+            return Double.valueOf(value.substring(0, value.length() - 1)) / 100;
+        }
+        else
+        {
+            return Double.valueOf(value);
+        }
+    }
+
     public static InetAddress getLocalAddress()
     {
         if (localInetAddress_ == null)
@@ -80,6 +95,27 @@
     }
 
     /**
+     * @param fractOrAbs A double that may represent a fraction or absolute value.
+     * @param total If fractionOrAbs is a fraction, the total to take the fraction from
+     * @return An absolute value which may be larger than the total.
+     */
+    public static long absoluteFromFraction(double fractOrAbs, long total)
+    {
+        if (fractOrAbs < 0)
+            throw new UnsupportedOperationException("unexpected negative value " + fractOrAbs);
+
+        if (0 < fractOrAbs && fractOrAbs < 1)
+        {
+            // fraction
+            return Math.max(1, (long)(fractOrAbs * total));
+        }
+
+        // absolute
+        assert fractOrAbs >= 1 || fractOrAbs == 0;
+        return (long)fractOrAbs;
+    }
+
+    /**
      * Given two bit arrays represented as BigIntegers, containing the given
      * number of significant bits, calculate a midpoint.
      *

Modified: incubator/cassandra/branches/cassandra-0.6/test/conf/storage-conf.xml
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/test/conf/storage-conf.xml?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/test/conf/storage-conf.xml (original)
+++ incubator/cassandra/branches/cassandra-0.6/test/conf/storage-conf.xml Sat Feb 27 15:02:48 2010
@@ -40,11 +40,11 @@
    <MemtableOperationsInMillions>0.00002</MemtableOperationsInMillions> <!-- 20 -->
    <Keyspaces>
      <Keyspace Name = "Keyspace1">
-       <ColumnFamily Name="Standard1" RowsCached="10%" KeysCachedFraction="0"/>
+       <ColumnFamily Name="Standard1" RowsCached="10%" KeysCached="0"/>
        <ColumnFamily Name="Standard2"/>
        <ColumnFamily CompareWith="LongType" Name="StandardLong1"/>
        <ColumnFamily CompareWith="LongType" Name="StandardLong2"/>
-       <ColumnFamily ColumnType="Super" CompareSubcolumnsWith="LongType" Name="Super1" RowsCached="1000" KeysCachedFraction="0"/>
+       <ColumnFamily ColumnType="Super" CompareSubcolumnsWith="LongType" Name="Super1" RowsCached="1000" KeysCached="0"/>
        <ColumnFamily ColumnType="Super" CompareSubcolumnsWith="LongType" Name="Super2"/>
        <ColumnFamily ColumnType="Super" CompareSubcolumnsWith="LongType" Name="Super3"/>
        <ColumnFamily ColumnType="Super" CompareSubcolumnsWith="UTF8Type" Name="Super4"/>

Modified: incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/io/SSTableUtils.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/io/SSTableUtils.java?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/io/SSTableUtils.java (original)
+++ incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/io/SSTableUtils.java Sat Feb 27 15:02:48 2010
@@ -102,6 +102,6 @@
                           entry.getValue());
         new File(writer.indexFilename()).deleteOnExit();
         new File(writer.filterFilename()).deleteOnExit();
-        return writer.closeAndOpenReader(1.0);
+        return writer.closeAndOpenReader();
     }
 }

Modified: incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/tools/SSTableExportTest.java?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/tools/SSTableExportTest.java (original)
+++ incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/tools/SSTableExportTest.java Sat Feb 27 15:02:48 2010
@@ -28,7 +28,6 @@
 import org.apache.cassandra.db.filter.NamesQueryFilter;
 import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.io.SSTableAccessor;
 import org.apache.cassandra.io.SSTableReader;
 import org.apache.cassandra.io.SSTableWriter;
 import org.apache.cassandra.io.util.DataOutputBuffer;
@@ -67,7 +66,7 @@
         dob.reset();
         cfamily.clear();
      
-        writer.closeAndOpenReader(0);
+        writer.closeAndOpenReader();
         
         // Enumerate and verify
         File temp = File.createTempFile("Standard1", ".txt");
@@ -105,7 +104,7 @@
         dob.reset();
         cfamily.clear();
      
-        SSTableReader reader = writer.closeAndOpenReader(0);
+        SSTableReader reader = writer.closeAndOpenReader();
         
         // Export to JSON and verify
         File tempJson = File.createTempFile("Standard1", ".json");
@@ -145,7 +144,7 @@
         dob.reset();
         cfamily.clear();
      
-        SSTableReader reader = writer.closeAndOpenReader(0);
+        SSTableReader reader = writer.closeAndOpenReader();
         
         // Export to JSON and verify
         File tempJson = File.createTempFile("Super4", ".json");
@@ -178,7 +177,7 @@
         dob.reset();
         cfamily.clear();
         
-        SSTableReader reader = writer.closeAndOpenReader(0);
+        SSTableReader reader = writer.closeAndOpenReader();
         
         // Export to JSON and verify
         File tempJson = File.createTempFile("Standard1", ".json");
@@ -188,7 +187,7 @@
         File tempSS2 = createTemporarySSTable("Keyspace1", "Standard1");
         SSTableImport.importJson(tempJson.getPath(), "Keyspace1", "Standard1", tempSS2.getPath());        
         
-        reader = SSTableAccessor.getSSTableReader(tempSS2.getPath(), DatabaseDescriptor.getPartitioner());
+        reader = SSTableReader.open(tempSS2.getPath(), DatabaseDescriptor.getPartitioner());
         NamesQueryFilter qf = new NamesQueryFilter("rowA", new QueryPath("Standard1", null, null), "name".getBytes());
         ColumnFamily cf = qf.getSSTableColumnIterator(reader).getColumnFamily();
         assertTrue(cf != null);

Modified: incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/tools/SSTableImportTest.java?rev=916957&r1=916956&r2=916957&view=diff
==============================================================================
--- incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/tools/SSTableImportTest.java (original)
+++ incubator/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/tools/SSTableImportTest.java Sat Feb 27 15:02:48 2010
@@ -26,7 +26,6 @@
 import org.apache.cassandra.db.IColumn;
 import org.apache.cassandra.db.filter.NamesQueryFilter;
 import org.apache.cassandra.db.filter.QueryPath;
-import org.apache.cassandra.io.SSTableAccessor;
 import org.apache.cassandra.io.SSTableReader;
 import static org.apache.cassandra.utils.FBUtilities.hexToBytes;
 import static org.apache.cassandra.Util.createTemporarySSTable;
@@ -44,7 +43,7 @@
         SSTableImport.importJson(jsonUrl, "Keyspace1", "Standard1", tempSS.getPath());
 
         // Verify results
-        SSTableReader reader = SSTableAccessor.getSSTableReader(tempSS.getPath(), DatabaseDescriptor.getPartitioner());
+        SSTableReader reader = SSTableReader.open(tempSS.getPath(), DatabaseDescriptor.getPartitioner());
         NamesQueryFilter qf = new NamesQueryFilter("rowA", new QueryPath("Standard1", null, null), "colAA".getBytes());
         ColumnFamily cf = qf.getSSTableColumnIterator(reader).getColumnFamily();
         assert Arrays.equals(cf.getColumn("colAA".getBytes()).value(), hexToBytes("76616c4141"));
@@ -58,7 +57,7 @@
         SSTableImport.importJson(jsonUrl, "Keyspace1", "Super4", tempSS.getPath());
         
         // Verify results
-        SSTableReader reader = SSTableAccessor.getSSTableReader(tempSS.getPath(), DatabaseDescriptor.getPartitioner());
+        SSTableReader reader = SSTableReader.open(tempSS.getPath(), DatabaseDescriptor.getPartitioner());
         NamesQueryFilter qf = new NamesQueryFilter("rowA", new QueryPath("Super4", null, null), "superA".getBytes());
         ColumnFamily cf = qf.getSSTableColumnIterator(reader).getColumnFamily();
         IColumn superCol = cf.getColumn("superA".getBytes());