@ -35,6 +35,8 @@ import org.apache.kafka.test.TestUtils;
@@ -35,6 +35,8 @@ import org.apache.kafka.test.TestUtils;
import org.junit.After ;
import org.junit.Before ;
import org.junit.Test ;
import org.rocksdb.BlockBasedTableConfig ;
import org.rocksdb.BloomFilter ;
import org.rocksdb.Options ;
import java.io.File ;
@ -48,6 +50,7 @@ import java.util.Set;
@@ -48,6 +50,7 @@ import java.util.Set;
import static java.nio.charset.StandardCharsets.UTF_8 ;
import static org.hamcrest.CoreMatchers.equalTo ;
import static org.hamcrest.CoreMatchers.is ;
import static org.junit.Assert.assertEquals ;
import static org.junit.Assert.assertFalse ;
import static org.junit.Assert.assertThat ;
@ -60,6 +63,7 @@ public class RocksDBStoreTest {
@@ -60,6 +63,7 @@ public class RocksDBStoreTest {
private RocksDBStore rocksDBStore ;
private InternalMockProcessorContext context ;
private File dir ;
private static boolean enableBloomFilters = false ;
@Before
public void setUp ( ) {
@ -415,6 +419,53 @@ public class RocksDBStoreTest {
@@ -415,6 +419,53 @@ public class RocksDBStoreTest {
rocksDBStore . flush ( ) ;
}
@Test
public void shouldHandleToggleOfEnablingBloomFilters ( ) {
final Properties props = StreamsTestUtils . getStreamsConfig ( ) ;
props . put ( StreamsConfig . ROCKSDB_CONFIG_SETTER_CLASS_CONFIG , TestingBloomFilterRocksDBConfigSetter . class ) ;
rocksDBStore = new RocksDBStore ( "test" ) ;
dir = TestUtils . tempDirectory ( ) ;
context = new InternalMockProcessorContext ( dir ,
Serdes . String ( ) ,
Serdes . String ( ) ,
new StreamsConfig ( props ) ) ;
rocksDBStore . init ( context , rocksDBStore ) ;
final List < String > expectedValues = new ArrayList < > ( ) ;
expectedValues . add ( "a" ) ;
expectedValues . add ( "b" ) ;
expectedValues . add ( "c" ) ;
final List < KeyValue < byte [ ] , byte [ ] > > keyValues = getKeyValueEntries ( ) ;
for ( final KeyValue < byte [ ] , byte [ ] > keyValue : keyValues ) {
rocksDBStore . put ( new Bytes ( keyValue . key ) , keyValue . value ) ;
}
int expectedIndex = 0 ;
for ( final KeyValue < byte [ ] , byte [ ] > keyValue : keyValues ) {
final byte [ ] valBytes = rocksDBStore . get ( new Bytes ( keyValue . key ) ) ;
assertThat ( new String ( valBytes , UTF_8 ) , is ( expectedValues . get ( expectedIndex + + ) ) ) ;
}
assertFalse ( TestingBloomFilterRocksDBConfigSetter . bloomFiltersSet ) ;
rocksDBStore . close ( ) ;
enableBloomFilters = true ;
expectedIndex = 0 ;
// reopen with Bloom Filters enabled
// should open fine without errors
rocksDBStore . init ( context , rocksDBStore ) ;
for ( final KeyValue < byte [ ] , byte [ ] > keyValue : keyValues ) {
final byte [ ] valBytes = rocksDBStore . get ( new Bytes ( keyValue . key ) ) ;
assertThat ( new String ( valBytes , UTF_8 ) , is ( expectedValues . get ( expectedIndex + + ) ) ) ;
}
assertTrue ( TestingBloomFilterRocksDBConfigSetter . bloomFiltersSet ) ;
}
public static class MockRocksDbConfigSetter implements RocksDBConfigSetter {
static boolean called ;
@ -426,6 +477,29 @@ public class RocksDBStoreTest {
@@ -426,6 +477,29 @@ public class RocksDBStoreTest {
}
}
public static class TestingBloomFilterRocksDBConfigSetter implements RocksDBConfigSetter {
static boolean bloomFiltersSet = false ;
@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 ) {
tableConfig . setFilter ( new BloomFilter ( ) ) ;
options . optimizeFiltersForHits ( ) ;
bloomFiltersSet = true ;
} else {
options . setOptimizeFiltersForHits ( false ) ;
}
options . setTableFormatConfig ( tableConfig ) ;
}
}
private List < KeyValue < byte [ ] , byte [ ] > > getKeyValueEntries ( ) {
final List < KeyValue < byte [ ] , byte [ ] > > entries = new ArrayList < > ( ) ;
entries . add ( new KeyValue < > ( "1" . getBytes ( UTF_8 ) , "a" . getBytes ( UTF_8 ) ) ) ;
@ -433,4 +507,5 @@ public class RocksDBStoreTest {
@@ -433,4 +507,5 @@ public class RocksDBStoreTest {
entries . add ( new KeyValue < > ( "3" . getBytes ( UTF_8 ) , "c" . getBytes ( UTF_8 ) ) ) ;
return entries ;
}
}