This is an automated email from the ASF dual-hosted git repository.
brandonwilliams pushed a commit to branch cassandra-3.0
in repository https://gitbox.apache.org/repos/asf/cassandra.git
The following commit(s) were added to refs/heads/cassandra-3.0 by this push:
new 4e5bd57 Bloom false positive rate includes true negatives
4e5bd57 is described below
commit 4e5bd572f4cca2831b87d6aa1b322e8ad846d015
Author: Jaroslaw Grabowski <[email protected]>
AuthorDate: Tue May 26 15:15:11 2020 +0200
Bloom false positive rate includes true negatives
Patch by Jaroslaw Grabowski; reviewed by brandonwilliams and paulo for
CASSANDRA-15384
Before this change the bloom filter false positive rate was calculated
without true negatives which resulted in high rates. In an extreme case,
where all queries return no data, the false positive rate could go up to
1.0.
This change includes true negatives in [recent] bloom filter false ratio.
---
CHANGES.txt | 1 +
.../cassandra/io/sstable/BloomFilterTracker.java | 25 ++++++++++
.../cassandra/io/sstable/format/SSTableReader.java | 10 ++++
.../io/sstable/format/big/BigTableReader.java | 1 +
.../org/apache/cassandra/metrics/TableMetrics.java | 56 ++++++++++++----------
5 files changed, 69 insertions(+), 24 deletions(-)
diff --git a/CHANGES.txt b/CHANGES.txt
index aaf8fe1..0ce3375 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
3.0.25:
+ * Fix bloom filter false ratio calculation by including true negatives
(CASSANDRA-15834)
* Prevent loss of commit log data when moving sstables between nodes
(CASSANDRA-16619)
* Fix materialized view builders inserting truncated data (CASSANDRA-16567)
* Don't wait for schema migrations from removed nodes (CASSANDRA-16577)
diff --git a/src/java/org/apache/cassandra/io/sstable/BloomFilterTracker.java
b/src/java/org/apache/cassandra/io/sstable/BloomFilterTracker.java
index 221c8a2..07523a0 100644
--- a/src/java/org/apache/cassandra/io/sstable/BloomFilterTracker.java
+++ b/src/java/org/apache/cassandra/io/sstable/BloomFilterTracker.java
@@ -23,8 +23,10 @@ public class BloomFilterTracker
{
private final AtomicLong falsePositiveCount = new AtomicLong(0);
private final AtomicLong truePositiveCount = new AtomicLong(0);
+ private final AtomicLong trueNegativeCount = new AtomicLong(0);
private long lastFalsePositiveCount = 0L;
private long lastTruePositiveCount = 0L;
+ private long lastTrueNegativeCount = 0L;
public void addFalsePositive()
{
@@ -36,6 +38,11 @@ public class BloomFilterTracker
truePositiveCount.incrementAndGet();
}
+ public void addTrueNegative()
+ {
+ trueNegativeCount.incrementAndGet();
+ }
+
public long getFalsePositiveCount()
{
return falsePositiveCount.get();
@@ -71,4 +78,22 @@ public class BloomFilterTracker
lastTruePositiveCount = tpc;
}
}
+
+ public long getTrueNegativeCount()
+ {
+ return trueNegativeCount.get();
+ }
+
+ public long getRecentTrueNegativeCount()
+ {
+ long tnc = getTrueNegativeCount();
+ try
+ {
+ return (tnc - lastTrueNegativeCount);
+ }
+ finally
+ {
+ lastTrueNegativeCount = tnc;
+ }
+ }
}
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index e7f9613..6bb17ba 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -1853,6 +1853,16 @@ public abstract class SSTableReader extends SSTable
implements SelfRefCounted<SS
return bloomFilterTracker.getRecentTruePositiveCount();
}
+ public long getBloomFilterTrueNegativeCount()
+ {
+ return bloomFilterTracker.getTrueNegativeCount();
+ }
+
+ public long getRecentBloomFilterTrueNegativeCount()
+ {
+ return bloomFilterTracker.getRecentTrueNegativeCount();
+ }
+
public InstrumentingCache<KeyCacheKey, RowIndexEntry> getKeyCache()
{
return keyCache;
diff --git
a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
index eeea18f..598f757 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
@@ -145,6 +145,7 @@ public class BigTableReader extends SSTableReader
{
listener.onSSTableSkipped(this, SkippingReason.BLOOM_FILTER);
Tracing.trace("Bloom filter allows skipping sstable {}",
descriptor.generation);
+ bloomFilterTracker.addTrueNegative();
return null;
}
}
diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java
b/src/java/org/apache/cassandra/metrics/TableMetrics.java
index 14e6423..aee0590 100644
--- a/src/java/org/apache/cassandra/metrics/TableMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java
@@ -496,68 +496,76 @@ public class TableMetrics
{
public Double getValue()
{
- long falseCount = 0L;
- long trueCount = 0L;
+ long falsePositiveCount = 0L;
+ long truePositiveCount = 0L;
+ long trueNegativeCount = 0L;
for (SSTableReader sstable : cfs.getSSTables(SSTableSet.LIVE))
{
- falseCount += sstable.getBloomFilterFalsePositiveCount();
- trueCount += sstable.getBloomFilterTruePositiveCount();
+ falsePositiveCount +=
sstable.getBloomFilterFalsePositiveCount();
+ truePositiveCount +=
sstable.getBloomFilterTruePositiveCount();
+ trueNegativeCount +=
sstable.getBloomFilterTrueNegativeCount();
}
- if (falseCount == 0L && trueCount == 0L)
+ if (falsePositiveCount == 0L && truePositiveCount == 0L)
return 0d;
- return (double) falseCount / (trueCount + falseCount);
+ return (double) falsePositiveCount / (truePositiveCount +
falsePositiveCount + trueNegativeCount);
}
}, new Gauge<Double>() // global gauge
{
public Double getValue()
{
- long falseCount = 0L;
- long trueCount = 0L;
+ long falsePositiveCount = 0L;
+ long truePositiveCount = 0L;
+ long trueNegativeCount = 0L;
for (Keyspace keyspace : Keyspace.all())
{
for (SSTableReader sstable :
keyspace.getAllSSTables(SSTableSet.LIVE))
{
- falseCount +=
sstable.getBloomFilterFalsePositiveCount();
- trueCount += sstable.getBloomFilterTruePositiveCount();
+ falsePositiveCount +=
sstable.getBloomFilterFalsePositiveCount();
+ truePositiveCount +=
sstable.getBloomFilterTruePositiveCount();
+ trueNegativeCount +=
sstable.getBloomFilterTrueNegativeCount();
}
}
- if (falseCount == 0L && trueCount == 0L)
+ if (falsePositiveCount == 0L && truePositiveCount == 0L)
return 0d;
- return (double) falseCount / (trueCount + falseCount);
+ return (double) falsePositiveCount / (truePositiveCount +
falsePositiveCount + trueNegativeCount);
}
});
recentBloomFilterFalseRatio =
createTableGauge("RecentBloomFilterFalseRatio", new Gauge<Double>()
{
public Double getValue()
{
- long falseCount = 0L;
- long trueCount = 0L;
+ long falsePositiveCount = 0L;
+ long truePositiveCount = 0L;
+ long trueNegativeCount = 0L;
for (SSTableReader sstable: cfs.getSSTables(SSTableSet.LIVE))
{
- falseCount +=
sstable.getRecentBloomFilterFalsePositiveCount();
- trueCount +=
sstable.getRecentBloomFilterTruePositiveCount();
+ falsePositiveCount +=
sstable.getRecentBloomFilterFalsePositiveCount();
+ truePositiveCount +=
sstable.getRecentBloomFilterTruePositiveCount();
+ trueNegativeCount +=
sstable.getRecentBloomFilterTrueNegativeCount();
}
- if (falseCount == 0L && trueCount == 0L)
+ if (falsePositiveCount == 0L && truePositiveCount == 0L)
return 0d;
- return (double) falseCount / (trueCount + falseCount);
+ return (double) falsePositiveCount / (truePositiveCount +
falsePositiveCount + trueNegativeCount);
}
}, new Gauge<Double>() // global gauge
{
public Double getValue()
{
- long falseCount = 0L;
- long trueCount = 0L;
+ long falsePositiveCount = 0L;
+ long truePositiveCount = 0L;
+ long trueNegativeCount = 0L;
for (Keyspace keyspace : Keyspace.all())
{
for (SSTableReader sstable :
keyspace.getAllSSTables(SSTableSet.LIVE))
{
- falseCount +=
sstable.getRecentBloomFilterFalsePositiveCount();
- trueCount +=
sstable.getRecentBloomFilterTruePositiveCount();
+ falsePositiveCount +=
sstable.getRecentBloomFilterFalsePositiveCount();
+ truePositiveCount +=
sstable.getRecentBloomFilterTruePositiveCount();
+ trueNegativeCount +=
sstable.getRecentBloomFilterTrueNegativeCount();
}
}
- if (falseCount == 0L && trueCount == 0L)
+ if (falsePositiveCount == 0L && truePositiveCount == 0L)
return 0d;
- return (double) falseCount / (trueCount + falseCount);
+ return (double) falsePositiveCount / (truePositiveCount +
falsePositiveCount + trueNegativeCount);
}
});
bloomFilterDiskSpaceUsed =
createTableGauge("BloomFilterDiskSpaceUsed", new Gauge<Long>()
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]