This is an automated email from the ASF dual-hosted git repository. bbejeck pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push: new a1b1e08 KAFKA-8323: Close RocksDBStore's BloomFilter (#6672) a1b1e08 is described below commit a1b1e088b98763818e933dce335b580d02916640 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 3cf8e94..e4c416c 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[]>, BulkLoadingSt 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[]>, BulkLoadingSt 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); @@ -392,6 +395,7 @@ public class RocksDBStore implements KeyValueStore<Bytes, byte[]>, BulkLoadingSt wOptions.close(); fOptions.close(); db.close(); + filter.close(); dbAccessor = null; userSpecifiedOptions = null;