zhuzhurk commented on code in PR #24251: URL: https://github.com/apache/flink/pull/24251#discussion_r1477168412
########## docs/content/docs/deployment/config.md: ########## @@ -27,16 +27,100 @@ under the License. # Configuration -All configuration is done in `conf/flink-conf.yaml`, which is expected to be a flat collection of [YAML key value pairs](http://www.yaml.org/spec/1.2/spec.html) with format `key: value`. +All configuration is done in Flink Configuration File (see [Flink Configuration File](#flink-configuration-file)). Review Comment: > All configuration is done in Flink Configuration File -> All configuration can be set in Flink configuration file in the `conf/` directory. ########## docs/content/docs/deployment/config.md: ########## @@ -27,16 +27,100 @@ under the License. # Configuration -All configuration is done in `conf/flink-conf.yaml`, which is expected to be a flat collection of [YAML key value pairs](http://www.yaml.org/spec/1.2/spec.html) with format `key: value`. +All configuration is done in Flink Configuration File (see [Flink Configuration File](#flink-configuration-file)). The configuration is parsed and evaluated when the Flink processes are started. Changes to the configuration file require restarting the relevant processes. -The out of the box configuration will use your default Java installation. You can manually set the environment variable `JAVA_HOME` or the configuration key `env.java.home` in `conf/flink-conf.yaml` if you want to manually override the Java runtime to use. +The out of the box configuration will use your default Java installation. You can manually set the environment variable `JAVA_HOME` or the configuration key `env.java.home` in Flink configuration file if you want to manually override the Java runtime to use. You can specify a different configuration directory location by defining the `FLINK_CONF_DIR` environment variable. For resource providers which provide non-session deployments, you can specify per-job configurations this way. Make a copy of the `conf` directory from the Flink distribution and modify the settings on a per-job basis. Note that this is not supported in Docker or standalone Kubernetes deployments. On Docker-based deployments, you can use the `FLINK_PROPERTIES` environment variable for passing configuration values. On session clusters, the provided configuration will only be used for configuring [execution](#execution) parameters, e.g. configuration parameters affecting the job, not the underlying cluster. +# Flink Configuration File + +Starting with version 1.19, Flink has officially introduced full support for the standard YAML 1.2 syntax. Compared to the previous versions which only supported simple key-value pairs, this update provides users with more flexible and powerful configuration capabilities. To take advantage of this new feature, users need to use the newly introduced `config.yaml` configuration file. The existing `flink-conf.yaml` configuration file is deprecated and will no longer work in the upcoming version 2.0. To ensure a smooth transition, users are advised to migrate their existing Flink configuration to the new configuration file as soon as possible. + +This section will help users understand how to configure the Flink cluster and jobs through the `config.yaml` configuration file, as well as how to migrate old configuration to the new configuration file. + +### Usage + +Starting from Flink-1.19, the default configuration file has been changed to `config.yaml` and placed in the `conf/` directory. Users should directly modify this file to configure Flink. + +To continue using the legacy configuration file `flink-conf.yaml`, users just need to copy this file into the `conf/` directory. Once the legacy configuration file `flink-conf.yaml` is detected, Flink will prioritize using it as the configuration file. + +The usage for `config.yaml` is as follows: + +#### Config Key + +- Users can organize Config Keys in a nested format, such as: + +```config.yaml +restart-strategy: + type: failure-rate + failure-rate: + delay: 1 s + failure-rate-interval: 1 min + max-failures-per-interval: 1 +``` + +- Users can also organize Config Keys in a flatten format, such as: + +```flink-conf.yaml +restart-strategy.type: failure-rate +restart-strategy.failure-rate.delay: 1 s +restart-strategy.failure-rate.failure-rate-interval: 1 min +restart-strategy.failure-rate.max-failures-per-interval: 1 +``` + +#### Config Value + +The `config.yaml` configuration file allows users to configure values following the [YAML 1.2 core schema](https://yaml.org/spec/1.2.2/#103-core-schema). +Users can configure the values corresponding to the Config Type in the following format: + +{{< config_file >}} + +Additionally, users can configure the value for all Config Types as strings by simply enclosing the original value in single quotes or double quotes. + + +### Migrate from flink-conf.yaml to config.yaml +#### Behavior Changes +`config.yaml` strictly follows the YAML 1.2 syntax and is compatible with `flink-conf.yaml` in most cases, except for the following behavior changes: + +- Null value: + - `flink-conf.yaml`: Only supports leaving the value blank. + - `config.yaml`: Supports leaving it blank, or explicitly set it to null, Null, NULL, or `~`. + +- Comment: + - `flink-conf.yaml`: Everything after the first occurrence of `#` in each line is considered a comment. + - `config.yaml`: When there is at least one space between the `#` and the preceding content, or when the `#` is at the beginning of a line, the subsequent content is considered a comment. + +- Special character escaping in strings: + - `flink-conf.yaml`: Only needs to escape elements in Lists and Maps. + - List elements containing a semicolon ";" require escaping. + - Map elements containing a comma "," or colon ":" require escaping. + - `config.yaml`: Requires escaping special characters as defined in the YAML 1.2 specification; see the definition of [special characters](https://yaml.org/spec/1.2.2/#53-indicator-characters). + +- Duplicated keys: + - `flink-conf.yaml`: Allows duplicated keys and takes the last key-value pair for the corresponding key that appears in the file. + - `config.yaml`: Does not allow duplicated keys, and an error will be reported when loading the configuration. + +- Handling of invalid configuration: + - `flink-conf.yaml`: Invalid key-value pairs will be ignored. + - `config.yaml`: An error will be reported when loading the configuration. + +#### Migration Tool +To facilitate user migration, Flink provides a configuration file migration script that can automate the migration process. The usage is as follows: + +- Place the old configuration file `flink-conf.yaml` in the `conf/` directory. +- Execute the following command in the `$FLINK_HOME/` directory: +````migrate-tool.sh +bin/migrate-config-file.sh +```` +After running the command above, the migration script will automatically read the old configuration file `flink-conf.yaml` from the `conf/` directory and output the migrated results to the new configuration file `config.yaml` in the `conf/` directory. Note that due to historical reasons, all values in flink-conf.yaml will be recognized as String type, so the values in the generated config.yaml file will also be of String type, which means some values will be enclosed in quotes. However, Flink will convert them to the actual types defined in the configuration during subsequent configuration parsing. Review Comment: > due to historical reasons due to the limitation of the legacy configuration parser > defined in the configuration defined using `ConfigOption` ########## docs/content.zh/docs/deployment/config.md: ########## @@ -31,7 +31,7 @@ All configuration is done in Flink Configuration File (see [Flink Configuration The configuration is parsed and evaluated when the Flink processes are started. Changes to the configuration file require restarting the relevant processes. -The out of the box configuration will use your default Java installation. You can manually set the environment variable `JAVA_HOME` or the configuration key `env.java.home` in Flink Configuration file if you want to manually override the Java runtime to use. +The out of the box configuration will use your default Java installation. You can manually set the environment variable `JAVA_HOME` or the configuration key `env.java.home` in Flink Configuration file using a flattened format if you want to manually override the Java runtime to use. Review Comment: It's better to add an extra statement/note to tell users that a flattened format (i.e. one-line key-value format) is required for this configuration. ########## docs/content.zh/docs/deployment/cli.md: ########## @@ -30,7 +30,7 @@ under the License. Flink provides a Command-Line Interface (CLI) `bin/flink` to run programs that are packaged as JAR files and to control their execution. The CLI is part of any Flink setup, available in local single node setups and in distributed setups. -It connects to the running JobManager specified in `conf/flink-conf.yaml`. +It connects to the running JobManager specified in [`conf/config.yaml`]({{< ref "docs/deployment/config#flink-configuration-file" >}}). Review Comment: Maybe it's better to use `Flink configuration file` or `Flink 配置文件`, instead of `conf/config.yaml` or `conf/flink-conf.yaml`? So that users will not get confused if they are still using `flink-conf.yaml`. However, it's good to add a reference to `docs/deployment/config#flink-configuration-file` for further explanation. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
