Myasuka commented on a change in pull request #16964: URL: https://github.com/apache/flink/pull/16964#discussion_r697278207
########## File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java ########## @@ -61,7 +61,9 @@ public DBOptions createDBOptions(Collection<AutoCloseable> handlesToClose) { return new DBOptions() .setUseFsync(false) - .setInfoLogLevel(InfoLogLevel.HEADER_LEVEL) + .setInfoLogLevel(InfoLogLevel.INFO_LEVEL) + .setMaxLogFileSize(DEFAULT_LOG_FILE_SIZE) + .setKeepLogFileNum(DEFAULT_LOG_FILE_NUM) Review comment: `RocksDBConfigurableOptions` mainly focus on the settings of `DBOptions` and `ColumnFamilyOptions`. The original design of this class is used to let user could configure RocksDB via configurations instead of programmatically implemented `RocksDBOptionsFactory`. To make the minimal change, original options in `RocksDBConfigurableOptions` have no default value so that we would not make anything happen in `DefaultConfigurableOptionsFactory` just as before. Another reason why `RocksDBConfigurableOptions` have no default value is because some option behaves different on lower level predefinied options. I agree that we can set some default values in `RocksDBConfigurableOptions` nowdays, and I this could be done in another ticket. If we hope to avoid code duplication, I think we could introcude similiar function below: ~~~ java private static final Function<DBOptions, DBOptions> COMMON_DB_OPTIONS = dbOptions -> dbOptions.setUseFsync(false) .setStatsDumpPeriodSec(0) .setInfoLogLevel(InfoLogLevel.INFO_LEVEL) .setMaxLogFileSize(DEFAULT_LOG_FILE_SIZE) .setKeepLogFileNum(DEFAULT_LOG_FILE_NUM); ~~~ and call like: ~~~ public DBOptions createDBOptions(Collection<AutoCloseable> handlesToClose) { return COMMON_DB_OPTIONS.apply(new DBOptions()); } ~~~ also cc @Zakelly -- 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. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org