This is an automated email from the ASF dual-hosted git repository. bbejeck pushed a commit to branch 2.2 in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/2.2 by this push: new 40ee415 KAFKA-8323: Close RocksDBStore's BloomFilter (#6672) 40ee415 is described below commit 40ee41518dc782401867f2934502176bf013b497 Author: A. Sophie Blee-Goldman <sop...@confluent.io> AuthorDate: Fri May 3 16:50:18 2019 -0700 KAFKA-8323: Close RocksDBStore's BloomFilter (#6672) Any RocksJava object that inherits from org.rocksdb.AbstractNativeReference must be closed explicitly in order to free up the memory of the backing C++ object. The BloomFilter extends RocksObject (which implements AbstractNativeReference) and should be also be closed in RocksDBStore#close to avoid leaking memory. Reviewers: Bill Bejeck <bbej...@gmail.com> --- .../java/org/apache/kafka/streams/state/internals/RocksDBStore.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 2ca3ad3..109b8c3 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 @@ -90,6 +90,7 @@ public class RocksDBStore implements KeyValueStore<Bytes, byte[]> { private RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter userSpecifiedOptions; WriteOptions wOptions; FlushOptions fOptions; + private BloomFilter filter; private volatile boolean prepareForBulkload = false; ProcessorContext internalProcessorContext; @@ -120,7 +121,9 @@ public class RocksDBStore implements KeyValueStore<Bytes, byte[]> { final BlockBasedTableConfig tableConfig = new BlockBasedTableConfig(); tableConfig.setBlockCacheSize(BLOCK_CACHE_SIZE); tableConfig.setBlockSize(BLOCK_SIZE); - tableConfig.setFilter(new BloomFilter()); + + filter = new BloomFilter(); + tableConfig.setFilter(filter); userSpecifiedOptions.optimizeFiltersForHits(); userSpecifiedOptions.setTableFormatConfig(tableConfig); @@ -376,6 +379,7 @@ public class RocksDBStore implements KeyValueStore<Bytes, byte[]> { wOptions.close(); fOptions.close(); db.close(); + filter.close(); dbAccessor = null; userSpecifiedOptions = null;