Details
-
Bug
-
Status: Resolved
-
Low
-
Resolution: Fixed
-
None
-
Low
Description
In 1.2, we introduced the ability to turn bloom filters off completely by setting fp chance to 1.0. It looks like there is a bug with this though. When it's set to 1.0 the following errors occur because AlwaysPresentFilter is not present in the switch statement here at https://github.com/apache/cassandra/blob/cassandra-1.2/src/java/org/apache/cassandra/utils/FilterFactory.java#L91, and we default to Murmur3BloomFilter for an unknown type.
Exception in thread "main" java.lang.ClassCastException: org.apache.cassandra.utils.AlwaysPresentFilter cannot be cast to org.apache.cassandra.utils.Murmur3BloomFilter
at org.apache.cassandra.utils.FilterFactory.serializedSize(FilterFactory.java:91)
at org.apache.cassandra.io.sstable.SSTableReader.getBloomFilterSerializedSize(SSTableReader.java:531)
at org.apache.cassandra.metrics.ColumnFamilyMetrics$15.value(ColumnFamilyMetrics.java:273)
at org.apache.cassandra.metrics.ColumnFamilyMetrics$15.value(ColumnFamilyMetrics.java:268)
at org.apache.cassandra.db.ColumnFamilyStore.getBloomFilterDiskSpaceUsed(ColumnFamilyStore.java:1825)