NicoK commented on a change in pull request #16964: URL: https://github.com/apache/flink/pull/16964#discussion_r697286723
########## File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java ########## @@ -221,6 +229,9 @@ public ColumnFamilyOptions createColumnOptions(Collection<AutoCloseable> handles // ------------------------------------------------------------------------ + public static final long DEFAULT_LOG_FILE_SIZE = 10 * 1024 * 1024L; + public static final int DEFAULT_LOG_FILE_NUM = 10; Review comment: When I implemented FLINK-23812, I actually also created code with some defaults but didn't want to change default behaviour that shortly before the feature freeze and wanted to start a discussion on the ml first on how to set these parameters before creating a PR. I was thinking of using these: ``` diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBConfigurableOptions.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBConfigurableOptions.java index 7c46f2f9b08..311360de839 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBConfigurableOptions.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBConfigurableOptions.java @@ -28,6 +28,7 @@ import org.rocksdb.InfoLogLevel; import java.io.Serializable; import static org.apache.flink.configuration.ConfigOptions.key; +import static org.apache.flink.configuration.CoreOptions.FLINK_LOG_MAX; import static org.apache.flink.configuration.description.LinkElement.link; import static org.rocksdb.CompactionStyle.FIFO; import static org.rocksdb.CompactionStyle.LEVEL; @@ -96,7 +97,7 @@ public class RocksDBConfigurableOptions implements Serializable { public static final ConfigOption<MemorySize> LOG_MAX_FILE_SIZE = key("state.backend.rocksdb.log.max-file-size") .memoryType() - .noDefaultValue() + .defaultValue(MemorySize.ofMebiBytes(100)) // same as log4j-console.properties .withDescription( "The maximum size of RocksDB's file used for logging. " + "If the log files becomes larger than this, a new file will be created." @@ -105,14 +106,14 @@ public class RocksDBConfigurableOptions implements Serializable { public static final ConfigOption<Integer> LOG_FILE_NUM = key("state.backend.rocksdb.log.file-num") .intType() - .noDefaultValue() + .defaultValue(FLINK_LOG_MAX.defaultValue()) .withDescription( "The maximum number of files RocksDB should keep for logging (default RocksDB setting: 1000)."); public static final ConfigOption<String> LOG_DIR = key("state.backend.rocksdb.log.dir") .stringType() - .noDefaultValue() + .noDefaultValue() // could also go with FLINK_LOG_DIR (env.log.dir)! .withDescription( "The directory for RocksDB's logging files. " + "If empty (default RocksDB setting), log files will be in the same directory as data files. " ``` What do you think about these instead? -- 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