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;