diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 6457a2c8fab..ede618237cf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -195,7 +195,7 @@ public class RocksDBStore implements KeyValueStore, BatchWritingS filter = new BloomFilter(); tableConfig.setFilterPolicy(filter); - userSpecifiedOptions.optimizeFiltersForHits(); + userSpecifiedOptions.setOptimizeFiltersForHits(true); userSpecifiedOptions.setTableFormatConfig(tableConfig); userSpecifiedOptions.setWriteBufferSize(WRITE_BUFFER_SIZE); userSpecifiedOptions.setCompressionType(COMPRESSION_TYPE); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java index 8a02289890e..70224c8013c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java @@ -1256,7 +1256,7 @@ public class RocksDBStoreTest extends AbstractKeyValueStoreTest { if (enableBloomFilters) { filter = new BloomFilter(); tableConfig.setFilterPolicy(filter); - options.optimizeFiltersForHits(); + options.setOptimizeFiltersForHits(true); bloomFiltersSet = true; } else { options.setOptimizeFiltersForHits(false);