You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by bb...@apache.org on 2019/05/15 02:36:19 UTC

[kafka] branch trunk updated: Leverage new ConfigSetter#close method to properly open/close filter (#6730)

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 47fdf22  Leverage new ConfigSetter#close method to properly open/close filter (#6730)
47fdf22 is described below

commit 47fdf228605534302811287f77553d3c5303284e
Author: A. Sophie Blee-Goldman <so...@confluent.io>
AuthorDate: Tue May 14 19:36:05 2019 -0700

    Leverage new ConfigSetter#close method to properly open/close filter (#6730)
    
    Uses the close method added in KIP-453 to properly init & close RocksObjects in RocksDBStoreTest
    
    Reviewers: Bill Bejeck <bb...@gmail.com>
---
 .../kafka/streams/state/internals/RocksDBStoreTest.java    | 14 ++++++++++----
 1 file changed, 10 insertions(+), 4 deletions(-)

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 c9dde77..bfb1ecd 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
@@ -37,6 +37,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.rocksdb.BlockBasedTableConfig;
 import org.rocksdb.BloomFilter;
+import org.rocksdb.Filter;
 import org.rocksdb.Options;
 
 import java.io.File;
@@ -67,7 +68,6 @@ public class RocksDBStoreTest {
 
     InternalMockProcessorContext context;
     RocksDBStore rocksDBStore;
-    private static BloomFilter filter;
 
     @Before
     public void setUp() {
@@ -79,7 +79,6 @@ public class RocksDBStoreTest {
             Serdes.String(),
             Serdes.String(),
             new StreamsConfig(props));
-        filter = new BloomFilter();
     }
 
     RocksDBStore getRocksDBStore() {
@@ -88,7 +87,6 @@ public class RocksDBStoreTest {
 
     @After
     public void tearDown() {
-        filter.close();
         rocksDBStore.close();
     }
 
@@ -490,14 +488,15 @@ public class RocksDBStoreTest {
     public static class TestingBloomFilterRocksDBConfigSetter implements RocksDBConfigSetter {
 
         static boolean bloomFiltersSet;
+        static Filter filter;
 
         @Override
         public void setConfig(final String storeName, final Options options, final Map<String, Object> configs) {
-
             final BlockBasedTableConfig tableConfig = new BlockBasedTableConfig();
             tableConfig.setBlockCacheSize(50 * 1024 * 1024L);
             tableConfig.setBlockSize(4096L);
             if (enableBloomFilters) {
+                filter = new BloomFilter();
                 tableConfig.setFilter(filter);
                 options.optimizeFiltersForHits();
                 bloomFiltersSet = true;
@@ -508,6 +507,13 @@ public class RocksDBStoreTest {
 
             options.setTableFormatConfig(tableConfig);
         }
+
+        @Override
+        public void close(final String storeName, final Options options) {
+            if (filter != null) {
+                filter.close();
+            }
+        }
     }
 
     private List<KeyValue<byte[], byte[]>> getKeyValueEntries() {