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/06/19 05:48:08 UTC
svn commit: r956181 - in /cassandra/branches/cassandra-0.6: ./
src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/io/
test/unit/org/apache/cassandra/io/
Author: jbellis
Date: Sat Jun 19 03:48:08 2010
New Revision: 956181
URL: http://svn.apache.org/viewvc?rev=956181&view=rev
Log:
add JMX metrics for row-level bloom filter false positives. patch by Jeff Hodges; reviewed by jbellis for CASSANDRA-1212
Added:
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/BloomFilterTracker.java
cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java
Modified:
cassandra/branches/cassandra-0.6/CHANGES.txt
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableReader.java
Modified: cassandra/branches/cassandra-0.6/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/CHANGES.txt?rev=956181&r1=956180&r2=956181&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.6/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.6/CHANGES.txt Sat Jun 19 03:48:08 2010
@@ -24,6 +24,7 @@
sending streams out of order (CASSANDRA-1169)
* remove inactive stream managers after AES streams files (CASSANDRA-1169)
* allow removing entire row through batch_mutate Deletion (CASSANDRA-1027)
+ * add JMX metrics for row-level bloom filter false positives (CASSANDRA-1212)
* added a redhat init script to contrib (CASSANDRA-1201)
Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=956181&r1=956180&r2=956181&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Sat Jun 19 03:48:08 2010
@@ -32,12 +32,12 @@ import java.util.regex.Pattern;
import javax.management.MBeanServer;
import javax.management.ObjectName;
-import org.apache.log4j.Logger;
-import org.apache.commons.collections.IteratorUtils;
-
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
+import org.apache.log4j.Logger;
+import org.apache.commons.collections.IteratorUtils;
+
import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
import org.apache.cassandra.concurrent.NamedThreadFactory;
import org.apache.cassandra.config.DatabaseDescriptor;
@@ -1187,4 +1187,52 @@ public class ColumnFamilyStore implement
{
return memtablesPendingFlush;
}
+
+ public long getBloomFilterFalsePositives()
+ {
+ long count = 0L;
+ for (SSTableReader sstable: getSSTables())
+ {
+ count += sstable.getBloomFilterFalsePositiveCount();
+ }
+ return count;
+ }
+
+ public long getRecentBloomFilterFalsePositives()
+ {
+ long count = 0L;
+ for (SSTableReader sstable: getSSTables())
+ {
+ count += sstable.getRecentBloomFilterFalsePositiveCount();
+ }
+ return count;
+ }
+
+ public double getBloomFilterFalseRatio()
+ {
+ Long falseCount = 0L;
+ Long trueCount = 0L;
+ for (SSTableReader sstable: getSSTables())
+ {
+ falseCount += sstable.getBloomFilterFalsePositiveCount();
+ trueCount += sstable.getBloomFilterTruePositiveCount();
+ }
+ if (falseCount.equals(0L) && trueCount.equals(0L))
+ return 0d;
+ return falseCount.doubleValue() / (trueCount.doubleValue() + falseCount.doubleValue());
+ }
+
+ public double getRecentBloomFilterFalseRatio()
+ {
+ Long falseCount = 0L;
+ Long trueCount = 0L;
+ for (SSTableReader sstable: getSSTables())
+ {
+ falseCount += sstable.getRecentBloomFilterFalsePositiveCount();
+ trueCount += sstable.getRecentBloomFilterTruePositiveCount();
+ }
+ if (falseCount.equals(0L) && trueCount.equals(0L))
+ return 0d;
+ return falseCount.doubleValue() / (trueCount.doubleValue() + falseCount.doubleValue());
+ }
}
Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java?rev=956181&r1=956180&r2=956181&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java (original)
+++ cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java Sat Jun 19 03:48:08 2010
@@ -157,4 +157,12 @@ public interface ColumnFamilyStoreMBean
* @return
*/
public long getMeanRowCompactedSize();
+
+ public long getBloomFilterFalsePositives();
+
+ public long getRecentBloomFilterFalsePositives();
+
+ public double getBloomFilterFalseRatio();
+
+ public double getRecentBloomFilterFalseRatio();
}
Added: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/BloomFilterTracker.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/BloomFilterTracker.java?rev=956181&view=auto
==============================================================================
--- cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/BloomFilterTracker.java (added)
+++ cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/BloomFilterTracker.java Sat Jun 19 03:48:08 2010
@@ -0,0 +1,57 @@
+package org.apache.cassandra.io;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+public class BloomFilterTracker
+{
+ private AtomicLong falsePositiveCount = new AtomicLong(0);
+ private AtomicLong truePositiveCount = new AtomicLong(0);
+ private long lastFalsePositiveCount = 0L;
+ private long lastTruePositiveCount = 0L;
+
+ public void addFalsePositive()
+ {
+ falsePositiveCount.incrementAndGet();
+ }
+
+ public void addTruePositive()
+ {
+ truePositiveCount.incrementAndGet();
+ }
+
+ public long getFalsePositiveCount()
+ {
+ return falsePositiveCount.get();
+ }
+
+ public long getRecentFalsePositiveCount()
+ {
+ long fpc = getFalsePositiveCount();
+ try
+ {
+ return (fpc - lastFalsePositiveCount);
+ }
+ finally
+ {
+ lastFalsePositiveCount = fpc;
+ }
+ }
+
+ public long getTruePositiveCount()
+ {
+ return truePositiveCount.get();
+ }
+
+ public long getRecentTruePositiveCount()
+ {
+ long tpc = getTruePositiveCount();
+ try
+ {
+ return (tpc - lastTruePositiveCount);
+ }
+ finally
+ {
+ lastTruePositiveCount = tpc;
+ }
+ }
+}
Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableReader.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableReader.java?rev=956181&r1=956180&r2=956181&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableReader.java (original)
+++ cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/io/SSTableReader.java Sat Jun 19 03:48:08 2010
@@ -27,8 +27,6 @@ import java.nio.MappedByteBuffer;
import org.apache.log4j.Logger;
-import org.apache.commons.lang.StringUtils;
-
import org.apache.cassandra.cache.InstrumentedCache;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.utils.BloomFilter;
@@ -42,8 +40,6 @@ import org.apache.cassandra.io.util.Buff
import org.apache.cassandra.io.util.FileDataInput;
import org.apache.cassandra.io.util.MappedFileDataInput;
-import org.cliffc.high_scale_lib.NonBlockingHashMap;
-
/**
* SSTableReaders are open()ed by Table.onStart; after that they are created by SSTableWriter.renameAndOpen.
* Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
@@ -138,6 +134,8 @@ public class SSTableReader extends SSTab
private InstrumentedCache<Pair<String, DecoratedKey>, PositionSize> keyCache;
+ private BloomFilterTracker bloomFilterTracker = new BloomFilterTracker();
+
SSTableReader(String filename, IPartitioner partitioner, IndexSummary indexSummary, BloomFilter bloomFilter)
throws IOException
{
@@ -326,7 +324,10 @@ public class SSTableReader extends SSTab
// next, see if the sampled index says it's impossible for the key to be present
IndexSummary.KeyPosition sampledPosition = getIndexScanPosition(decoratedKey);
if (sampledPosition == null)
+ {
+ bloomFilterTracker.addFalsePositive();
return null;
+ }
// get either a buffered or a mmap'd input for the on-disk index
long p = sampledPosition.indexPosition;
@@ -350,8 +351,10 @@ public class SSTableReader extends SSTab
// handle exact sampled index hit
IndexSummary.KeyPosition kp = indexSummary.getSpannedIndexPosition(input.getAbsolutePosition());
if (kp != null && kp.key.equals(decoratedKey))
+ {
+ bloomFilterTracker.addTruePositive();
return indexSummary.getSpannedDataPosition(kp);
-
+ }
// if using mmapped i/o, skip to the next mmap buffer if necessary
if (input.isEOF() || kp != null)
{
@@ -387,10 +390,14 @@ public class SSTableReader extends SSTab
PositionSize info = getDataPositionSize(input, dataPosition);
if (keyCache != null && keyCache.getCapacity() > 0)
keyCache.put(unifiedKey, info);
+ bloomFilterTracker.addTruePositive();
return info;
}
if (v > 0)
+ {
+ bloomFilterTracker.addFalsePositive();
return null;
+ }
} while (++i < IndexSummary.INDEX_INTERVAL);
}
finally
@@ -398,6 +405,7 @@ public class SSTableReader extends SSTab
if (input != null)
input.close();
}
+ bloomFilterTracker.addFalsePositive();
return null;
}
@@ -539,4 +547,24 @@ public class SSTableReader extends SSTab
? Column.serializer()
: SuperColumn.serializer(getColumnComparator());
}
+
+ public long getBloomFilterFalsePositiveCount()
+ {
+ return bloomFilterTracker.getFalsePositiveCount();
+ }
+
+ public long getRecentBloomFilterFalsePositiveCount()
+ {
+ return bloomFilterTracker.getRecentFalsePositiveCount();
+ }
+
+ public long getBloomFilterTruePositiveCount()
+ {
+ return bloomFilterTracker.getTruePositiveCount();
+ }
+
+ public long getRecentBloomFilterTruePositiveCount()
+ {
+ return bloomFilterTracker.getRecentTruePositiveCount();
+ }
}
Added: cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java?rev=956181&view=auto
==============================================================================
--- cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java (added)
+++ cassandra/branches/cassandra-0.6/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java Sat Jun 19 03:48:08 2010
@@ -0,0 +1,50 @@
+package org.apache.cassandra.io;
+
+import org.junit.Test;
+
+import org.apache.cassandra.CleanupHelper;
+
+import static org.junit.Assert.assertEquals;
+
+public class BloomFilterTrackerTest extends CleanupHelper
+{
+ @Test
+ public void testAddingFalsePositives()
+ {
+ BloomFilterTracker bft = new BloomFilterTracker();
+ assertEquals(0L, bft.getFalsePositiveCount());
+ assertEquals(0L, bft.getRecentFalsePositiveCount());
+ bft.addFalsePositive();
+ bft.addFalsePositive();
+ assertEquals(2L, bft.getFalsePositiveCount());
+ assertEquals(2L, bft.getRecentFalsePositiveCount());
+ assertEquals(0L, bft.getRecentFalsePositiveCount());
+ assertEquals(2L, bft.getFalsePositiveCount()); // sanity check
+ }
+
+ @Test
+ public void testAddingTruePositives()
+ {
+ BloomFilterTracker bft = new BloomFilterTracker();
+ assertEquals(0L, bft.getTruePositiveCount());
+ assertEquals(0L, bft.getRecentTruePositiveCount());
+ bft.addTruePositive();
+ bft.addTruePositive();
+ assertEquals(2L, bft.getTruePositiveCount());
+ assertEquals(2L, bft.getRecentTruePositiveCount());
+ assertEquals(0L, bft.getRecentTruePositiveCount());
+ assertEquals(2L, bft.getTruePositiveCount()); // sanity check
+ }
+
+ @Test
+ public void testAddingToOneLeavesTheOtherAlone()
+ {
+ BloomFilterTracker bft = new BloomFilterTracker();
+ bft.addFalsePositive();
+ assertEquals(0L, bft.getTruePositiveCount());
+ assertEquals(0L, bft.getRecentTruePositiveCount());
+ bft.addTruePositive();
+ assertEquals(1L, bft.getFalsePositiveCount());
+ assertEquals(1L, bft.getRecentFalsePositiveCount());
+ }
+}