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



##########
File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBResource.java
##########
@@ -101,7 +101,7 @@ public ColumnFamilyOptions 
createColumnOptions(ColumnFamilyOptions currentOption
                                        LOG.error("Close previous 
ColumnOptions's instance failed.", e);
                                }
 
-                               return 
PredefinedOptions.FLASH_SSD_OPTIMIZED.createColumnOptions(handlesToClose);
+                               return 
PredefinedOptions.FLASH_SSD_OPTIMIZED.createColumnOptions(handlesToClose).optimizeForPointLookup(40960);

Review comment:
       We should cover both the `optimizeForPointLookup` and normal case here, 
instead of `optimizeForPointLookup` only

##########
File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBResource.java
##########
@@ -155,7 +155,7 @@ protected void before() throws Throwable {
                        
PredefinedOptions.DEFAULT.createColumnOptions(handlesToClose), handlesToClose);
                this.writeOptions = new WriteOptions();
                this.writeOptions.disableWAL();
-               this.readOptions = new ReadOptions();
+               this.readOptions = 
RocksDBOperationUtils.createTotalOrderSeekReadOptions();

Review comment:
       Ditto.

##########
File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
##########
@@ -235,7 +262,9 @@ public Object answer(InvocationOnMock invocationOnMock) 
throws Throwable {
        @Test
        public void testCorrectMergeOperatorSet() throws Exception {
                prepareRocksDB();
-               final ColumnFamilyOptions columnFamilyOptions = spy(new 
ColumnFamilyOptions());
+               final ColumnFamilyOptions columnFamilyOptions = 
userMisuseOptimizeForPointLookup ?
+                       spy(optimizePointLookupOption(new 
ColumnFamilyOptions())) :
+                       spy(new ColumnFamilyOptions());

Review comment:
       These `spy` are required by the later `Mockito.verify`. Since this issue 
is a blocker and the mockito tests has been existing for a while, let's keep it 
as is here, but we should find some time to try best to remove these mockito 
tests later.




----------------------------------------------------------------
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