carp84 commented on a change in pull request #14341:
URL: https://github.com/apache/flink/pull/14341#discussion_r578993427



##########
File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java
##########
@@ -162,7 +162,9 @@ public ColumnFamilyOptions 
createColumnOptions(Collection<AutoCloseable> handles
             final long targetFileSize = 256 * 1024 * 1024;
             final long writeBufferSize = 64 * 1024 * 1024;
 
-            BloomFilter bloomFilter = new BloomFilter();
+            /* Default use Full Filters (new format) instead of Block-based 
Bloom Filter, and bits_per_key=10.
+             * https://github.com/facebook/rocksdb/wiki/RocksDB-Bloom-Filter */
+            BloomFilter bloomFilter = new BloomFilter(10, false);

Review comment:
       From the document of [partitioned index 
filters](https://github.com/facebook/rocksdb/wiki/Partitioned-Index-Filters) we 
could see using full filters instead of block-based filter is a necessity to 
enable it. Please write this down explicitly in the PR description.
   
   However, OTOH, I don't think we need to modify the setting in pre-defined 
option here since later in `RocksDBResourceContainer#getColumnOptions` we will 
rewrite the setting if user choose to enable partitioned index filter. Changing 
settings in pre-defined options is a user-detective behavior with possible 
performance impact, so we should be cautious (although I could read from 
[RocksDB 
document](https://github.com/facebook/rocksdb/wiki/RocksDB-Bloom-Filter#full-filters-new-format)
 that the new format full filters is recommended)

##########
File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBResourceContainer.java
##########
@@ -207,4 +221,37 @@ public void close() throws Exception {
             sharedResources.close();
         }
     }
+
+    /**
+     * Overwrite configured {@link Filter} if enable partitioned filter. 
Partitioned filter only
+     * worked in full bloom filter, not blocked based.
+     */
+    private void overwriteFilterIfExist(BlockBasedTableConfig 
blockBasedTableConfig) {
+        Filter filter = 
getFilterFromBlockBasedTableConfig(blockBasedTableConfig);
+        if (filter != null) {
+            // TODO Can get filter's config in the future RocksDB version, and 
build new filter use
+            // existing config.
+            BloomFilter newFilter = new BloomFilter(10, false);
+            LOG.warn(
+                    "Overwrite existing filter if '{}' is enabled.",
+                    RocksDBOptions.USE_PARTITIONED_INDEX_FILTERS);

Review comment:
       ```suggestion
               LOG.info("Existing filter has been overwritten to full filters 
since partitioned index filters is enabled.");
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to