You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by gy...@apache.org on 2022/07/08 12:40:49 UTC

[flink-kubernetes-operator] 03/03: [docs] Break up documentation into sections

This is an automated email from the ASF dual-hosted git repository.

gyfora pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/flink-kubernetes-operator.git

commit f93da5c30dd958309ba492b6e1358afa53a287d1
Author: Gyula Fora <g_...@apple.com>
AuthorDate: Fri Jul 8 08:25:49 2022 +0200

    [docs] Break up documentation into sections
---
 docs/content/docs/operations/configuration.md      | 30 ++++++---
 .../shortcodes/generated/dynamic_section.html      | 72 ++++++++++++++++++++
 .../kubernetes_operator_config_configuration.html  |  6 --
 .../kubernetes_operator_metric_configuration.html  | 36 ++++++++++
 .../generated/system_advanced_section.html         | 60 +++++++++++++++++
 .../shortcodes/generated/system_section.html       | 78 ++++++++++++++++++++++
 .../configuration/ConfigOptionsDocGenerator.java   |  4 +-
 .../flink/kubernetes/operator/FlinkOperator.java   |  2 +-
 .../operator/config/FlinkConfigManager.java        |  2 +-
 .../config/FlinkOperatorConfiguration.java         |  7 +-
 .../config/KubernetesOperatorConfigOptions.java    | 41 ++++++++++--
 .../metrics/KubernetesOperatorMetricOptions.java   |  8 +++
 .../operator/admission/FlinkOperatorWebhook.java   |  2 +-
 13 files changed, 319 insertions(+), 29 deletions(-)

diff --git a/docs/content/docs/operations/configuration.md b/docs/content/docs/operations/configuration.md
index 89e0edb..460120c 100644
--- a/docs/content/docs/operations/configuration.md
+++ b/docs/content/docs/operations/configuration.md
@@ -52,7 +52,7 @@ To learn more about metrics and logging configuration please refer to the dedica
 
 ## Dynamic Operator Configuration
 
-The Kubernetes operator supports dynamic config changes through the operator ConfigMaps. Dynamic operator configuration is enabled by default, and can be disabled by setting `kubernetes.operator.dynamic.config.enabled`  to false. Time interval for checking dynamic config changes is specified by `kubernetes.operator.dynamic.config.check.interval` of which default value is 5 minutes. 
+The Kubernetes operator supports dynamic config changes through the operator ConfigMaps. Dynamic operator configuration is enabled by default, and can be disabled by setting `kubernetes.operator.dynamic.config.enabled`  to false. Time interval for checking dynamic config changes is specified by `kubernetes.operator.dynamic.config.check.interval` of which default value is 5 minutes.
 
 Verify whether dynamic operator configuration updates is enabled via the `deploy/flink-kubernetes-operator` log has:
 
@@ -70,14 +70,26 @@ Verify whether the config value of `kubernetes.operator.reconcile.interval` is u
 
 ## Operator Configuration Reference
 
-{{< generated/kubernetes_operator_config_configuration >}}
+### System Configuration
 
-## Job Specific Configuration Reference
+General operator system configuration. Cannot be overridden on a per-resource basis.
 
-Job specific configuration can be configured under `spec.flinkConfiguration` and it will override flink configurations defined in `flink-conf.yaml`.
+{{< generated/system_section >}}
 
-- For application clusters, `spec.flinkConfiguration` will be located in `FlinkDeployment` CustomResource.
-- For session clusters, configuring `spec.flinkConfiguration` in parent `FlinkDeployment` will be applied to all session jobs within the session cluster.
-  - You can configure some additional job specific supplemental configuration through `spec.flinkConfiguration` in `FlinkSessionJob` CustomResource. 
-  Those session job level configurations will override the parent session cluster's Flink configuration. Please note only the following configurations are considered to be valid configurations.
-    - `kubernetes.operator.user.artifacts.http.header`
+### Resource/User Configuration
+
+These options can be configured on both an operator and a per-resource level. When set under `spec.flinkConfiguration` for the Flink resources it will override the default value provided in the operator default configuration (`flink-conf.yaml`).
+
+{{< generated/dynamic_section >}}
+
+### System Metrics Configuration
+
+Operator system metrics configuration. Cannot be overridden on a per-resource basis.
+
+{{< generated/kubernetes_operator_metric_configuration >}}
+
+### Advanced System Configuration
+
+Advanced operator system configuration. Cannot be overridden on a per-resource basis.
+
+{{< generated/system_advanced_section >}}
diff --git a/docs/layouts/shortcodes/generated/dynamic_section.html b/docs/layouts/shortcodes/generated/dynamic_section.html
new file mode 100644
index 0000000..8d8fb62
--- /dev/null
+++ b/docs/layouts/shortcodes/generated/dynamic_section.html
@@ -0,0 +1,72 @@
+<table class="configuration table table-bordered">
+    <thead>
+        <tr>
+            <th class="text-left" style="width: 20%">Key</th>
+            <th class="text-left" style="width: 15%">Default</th>
+            <th class="text-left" style="width: 10%">Type</th>
+            <th class="text-left" style="width: 55%">Description</th>
+        </tr>
+    </thead>
+    <tbody>
+        <tr>
+            <td><h5>kubernetes.operator.deployment.readiness.timeout</h5></td>
+            <td style="word-wrap: break-word;">1 min</td>
+            <td>Duration</td>
+            <td>The timeout for deployments to become ready/stable before being rolled back if rollback is enabled.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.deployment.rollback.enabled</h5></td>
+            <td style="word-wrap: break-word;">false</td>
+            <td>Boolean</td>
+            <td>Whether to enable rolling back failed deployment upgrades.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.jm-deployment-recovery.enabled</h5></td>
+            <td style="word-wrap: break-word;">true</td>
+            <td>Boolean</td>
+            <td>Whether to enable recovery of missing/deleted jobmanager deployments.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.job.upgrade.ignore-pending-savepoint</h5></td>
+            <td style="word-wrap: break-word;">false</td>
+            <td>Boolean</td>
+            <td>Whether to ignore pending savepoint during job upgrade.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.job.upgrade.last-state-fallback.enabled</h5></td>
+            <td style="word-wrap: break-word;">true</td>
+            <td>Boolean</td>
+            <td>Enables last-state fallback for savepoint upgrade mode. When the job is not running thus savepoint cannot be triggered but HA metadata is available for last state restore the operator can initiate the upgrade process when the flag is enabled.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.periodic.savepoint.interval</h5></td>
+            <td style="word-wrap: break-word;">0 ms</td>
+            <td>Duration</td>
+            <td>Interval at which periodic savepoints will be triggered. The triggering schedule is not guaranteed, savepoints will be triggered as part of the regular reconcile loop.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.savepoint.history.max.age</h5></td>
+            <td style="word-wrap: break-word;">86400000 ms</td>
+            <td>Duration</td>
+            <td>Maximum age for savepoint history entries to retain. Due to lazy clean-up, the most recent savepoint may live longer than the max age.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.savepoint.history.max.count</h5></td>
+            <td style="word-wrap: break-word;">10</td>
+            <td>Integer</td>
+            <td>Maximum number of savepoint history entries to retain.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.savepoint.trigger.grace-period</h5></td>
+            <td style="word-wrap: break-word;">1 min</td>
+            <td>Duration</td>
+            <td>The interval before a savepoint trigger attempt is marked as unsuccessful.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.user.artifacts.http.header</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>Map</td>
+            <td>Custom HTTP header for HttpArtifactFetcher. The header will be applied when getting the session job artifacts. Expected format: headerKey1:headerValue1,headerKey2:headerValue2.</td>
+        </tr>
+    </tbody>
+</table>
diff --git a/docs/layouts/shortcodes/generated/kubernetes_operator_config_configuration.html b/docs/layouts/shortcodes/generated/kubernetes_operator_config_configuration.html
index 5c34d95..cfc1ba9 100644
--- a/docs/layouts/shortcodes/generated/kubernetes_operator_config_configuration.html
+++ b/docs/layouts/shortcodes/generated/kubernetes_operator_config_configuration.html
@@ -80,12 +80,6 @@
             <td>Boolean</td>
             <td>Enables last-state fallback for savepoint upgrade mode. When the job is not running thus savepoint cannot be triggered but HA metadata is available for last state restore the operator can initiate the upgrade process when the flag is enabled.</td>
         </tr>
-        <tr>
-            <td><h5>kubernetes.operator.josdk.metrics.enabled</h5></td>
-            <td style="word-wrap: break-word;">true</td>
-            <td>Boolean</td>
-            <td>Enable forwarding of Java Operator SDK metrics to the Flink metric registry.</td>
-        </tr>
         <tr>
             <td><h5>kubernetes.operator.observer.progress-check.interval</h5></td>
             <td style="word-wrap: break-word;">10 s</td>
diff --git a/docs/layouts/shortcodes/generated/kubernetes_operator_metric_configuration.html b/docs/layouts/shortcodes/generated/kubernetes_operator_metric_configuration.html
new file mode 100644
index 0000000..37ee840
--- /dev/null
+++ b/docs/layouts/shortcodes/generated/kubernetes_operator_metric_configuration.html
@@ -0,0 +1,36 @@
+<table class="configuration table table-bordered">
+    <thead>
+        <tr>
+            <th class="text-left" style="width: 20%">Key</th>
+            <th class="text-left" style="width: 15%">Default</th>
+            <th class="text-left" style="width: 10%">Type</th>
+            <th class="text-left" style="width: 55%">Description</th>
+        </tr>
+    </thead>
+    <tbody>
+        <tr>
+            <td><h5>kubernetes.operator.josdk.metrics.enabled</h5></td>
+            <td style="word-wrap: break-word;">true</td>
+            <td>Boolean</td>
+            <td>Enable forwarding of Java Operator SDK metrics to the Flink metric registry.</td>
+        </tr>
+        <tr>
+            <td><h5>metrics.scope.k8soperator.resource</h5></td>
+            <td style="word-wrap: break-word;">"&lt;host&gt;.k8soperator.&lt;namespace&gt;.&lt;name&gt;.resource.&lt;resourcens&gt;.&lt;resourcename&gt;"</td>
+            <td>String</td>
+            <td>Defines the scope format string that is applied to all metrics scoped to the kubernetes operator resource.</td>
+        </tr>
+        <tr>
+            <td><h5>metrics.scope.k8soperator.resourcens</h5></td>
+            <td style="word-wrap: break-word;">"&lt;host&gt;.k8soperator.&lt;namespace&gt;.&lt;name&gt;.namespace.&lt;resourcens&gt;"</td>
+            <td>String</td>
+            <td>Defines the scope format string that is applied to all metrics scoped to the kubernetes operator resource namespace.</td>
+        </tr>
+        <tr>
+            <td><h5>metrics.scope.k8soperator.system</h5></td>
+            <td style="word-wrap: break-word;">"&lt;host&gt;.k8soperator.&lt;namespace&gt;.&lt;name&gt;.system"</td>
+            <td>String</td>
+            <td>Defines the scope format string that is applied to all metrics scoped to the kubernetes operator.</td>
+        </tr>
+    </tbody>
+</table>
diff --git a/docs/layouts/shortcodes/generated/system_advanced_section.html b/docs/layouts/shortcodes/generated/system_advanced_section.html
new file mode 100644
index 0000000..72d525d
--- /dev/null
+++ b/docs/layouts/shortcodes/generated/system_advanced_section.html
@@ -0,0 +1,60 @@
+<table class="configuration table table-bordered">
+    <thead>
+        <tr>
+            <th class="text-left" style="width: 20%">Key</th>
+            <th class="text-left" style="width: 15%">Default</th>
+            <th class="text-left" style="width: 10%">Type</th>
+            <th class="text-left" style="width: 55%">Description</th>
+        </tr>
+    </thead>
+    <tbody>
+        <tr>
+            <td><h5>kubernetes.operator.config.cache.size</h5></td>
+            <td style="word-wrap: break-word;">1000</td>
+            <td>Integer</td>
+            <td>Max config cache size.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.config.cache.timeout</h5></td>
+            <td style="word-wrap: break-word;">10 min</td>
+            <td>Duration</td>
+            <td>Expiration time for cached configs.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.dynamic.config.check.interval</h5></td>
+            <td style="word-wrap: break-word;">5 min</td>
+            <td>Duration</td>
+            <td>Time interval for checking config changes.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.dynamic.config.enabled</h5></td>
+            <td style="word-wrap: break-word;">true</td>
+            <td>Boolean</td>
+            <td>Whether to enable on-the-fly config changes through the operator configmap.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.observer.progress-check.interval</h5></td>
+            <td style="word-wrap: break-word;">10 s</td>
+            <td>Duration</td>
+            <td>The interval for observing status for in-progress operations such as deployment and savepoints.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.observer.rest-ready.delay</h5></td>
+            <td style="word-wrap: break-word;">10 s</td>
+            <td>Duration</td>
+            <td>Final delay before deployment is marked ready after port becomes accessible.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.savepoint.history.max.age.threshold</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>Duration</td>
+            <td>Maximum age threshold for savepoint history entries to retain.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.savepoint.history.max.count.threshold</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>Integer</td>
+            <td>Maximum number threshold of savepoint history entries to retain.</td>
+        </tr>
+    </tbody>
+</table>
diff --git a/docs/layouts/shortcodes/generated/system_section.html b/docs/layouts/shortcodes/generated/system_section.html
new file mode 100644
index 0000000..887326b
--- /dev/null
+++ b/docs/layouts/shortcodes/generated/system_section.html
@@ -0,0 +1,78 @@
+<table class="configuration table table-bordered">
+    <thead>
+        <tr>
+            <th class="text-left" style="width: 20%">Key</th>
+            <th class="text-left" style="width: 15%">Default</th>
+            <th class="text-left" style="width: 10%">Type</th>
+            <th class="text-left" style="width: 55%">Description</th>
+        </tr>
+    </thead>
+    <tbody>
+        <tr>
+            <td><h5>kubernetes.operator.dynamic.namespaces.enabled</h5></td>
+            <td style="word-wrap: break-word;">false</td>
+            <td>Boolean</td>
+            <td>Enables dynamic change of watched/monitored namespaces.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.flink.client.cancel.timeout</h5></td>
+            <td style="word-wrap: break-word;">1 min</td>
+            <td>Duration</td>
+            <td>The timeout for the reconciler to wait for flink to cancel job.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.flink.client.timeout</h5></td>
+            <td style="word-wrap: break-word;">10 s</td>
+            <td>Duration</td>
+            <td>The timeout for the observer to wait the flink rest client to return.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.reconcile.interval</h5></td>
+            <td style="word-wrap: break-word;">1 min</td>
+            <td>Duration</td>
+            <td>The interval for the controller to reschedule the reconcile process.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.reconcile.parallelism</h5></td>
+            <td style="word-wrap: break-word;">5</td>
+            <td>Integer</td>
+            <td>The maximum number of threads running the reconciliation loop. Use -1 for infinite.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.resource.cleanup.timeout</h5></td>
+            <td style="word-wrap: break-word;">1 min</td>
+            <td>Duration</td>
+            <td>The timeout for the resource clean up to wait for flink to shutdown cluster.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.retry.initial.interval</h5></td>
+            <td style="word-wrap: break-word;">5 s</td>
+            <td>Duration</td>
+            <td>Initial interval of automatic reconcile retries on recoverable errors.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.retry.interval.multiplier</h5></td>
+            <td style="word-wrap: break-word;">2.0</td>
+            <td>Double</td>
+            <td>Interval multiplier of automatic reconcile retries on recoverable errors.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.retry.max.attempts</h5></td>
+            <td style="word-wrap: break-word;">10</td>
+            <td>Integer</td>
+            <td>Max attempts of automatic reconcile retries on recoverable errors.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.user.artifacts.base.dir</h5></td>
+            <td style="word-wrap: break-word;">"/opt/flink/artifacts"</td>
+            <td>String</td>
+            <td>The base dir to put the session job artifacts.</td>
+        </tr>
+        <tr>
+            <td><h5>kubernetes.operator.watched.namespaces</h5></td>
+            <td style="word-wrap: break-word;">"JOSDK_ALL_NAMESPACES"</td>
+            <td>String</td>
+            <td>Comma separated list of namespaces the operator monitors for custom resources.</td>
+        </tr>
+    </tbody>
+</table>
diff --git a/flink-kubernetes-docs/src/main/java/org/apache/flink/kubernetes/operator/docs/configuration/ConfigOptionsDocGenerator.java b/flink-kubernetes-docs/src/main/java/org/apache/flink/kubernetes/operator/docs/configuration/ConfigOptionsDocGenerator.java
index ba182d3..43c81b4 100644
--- a/flink-kubernetes-docs/src/main/java/org/apache/flink/kubernetes/operator/docs/configuration/ConfigOptionsDocGenerator.java
+++ b/flink-kubernetes-docs/src/main/java/org/apache/flink/kubernetes/operator/docs/configuration/ConfigOptionsDocGenerator.java
@@ -72,7 +72,9 @@ public class ConfigOptionsDocGenerator {
     static final OptionsClassLocation[] LOCATIONS =
             new OptionsClassLocation[] {
                 new OptionsClassLocation(
-                        "flink-kubernetes-operator", "org.apache.flink.kubernetes.operator.config")
+                        "flink-kubernetes-operator", "org.apache.flink.kubernetes.operator.config"),
+                new OptionsClassLocation(
+                        "flink-kubernetes-operator", "org.apache.flink.kubernetes.operator.metrics")
             };
     static final String DEFAULT_PATH_PREFIX = "src/main/java";
 
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/FlinkOperator.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/FlinkOperator.java
index 13bbe89..3324478 100644
--- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/FlinkOperator.java
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/FlinkOperator.java
@@ -110,7 +110,7 @@ public class FlinkOperator {
             LOG.info("Configuring operator with {} reconciliation threads.", parallelism);
             overrider.withConcurrentReconciliationThreads(parallelism);
         }
-        if (configManager.getOperatorConfiguration().getJosdkMetricsEnabled()) {
+        if (configManager.getOperatorConfiguration().isJosdkMetricsEnabled()) {
             overrider.withMetrics(
                     new OperatorJosdkMetrics(metricGroup, configManager.getDefaultConfig()));
         }
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java
index caccf10..c70dbae 100644
--- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java
@@ -132,7 +132,7 @@ public class FlinkConfigManager {
                         .orElse(Set.of());
         this.operatorConfiguration = FlinkOperatorConfiguration.fromConfiguration(newConf);
         var newNs = this.operatorConfiguration.getWatchedNamespaces();
-        if (this.operatorConfiguration.getDynamicNamespacesEnabled() && !oldNs.equals(newNs)) {
+        if (this.operatorConfiguration.isDynamicNamespacesEnabled() && !oldNs.equals(newNs)) {
             this.namespaceListener.accept(operatorConfiguration.getWatchedNamespaces());
         }
         this.defaultConfig = newConf.clone();
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkOperatorConfiguration.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkOperatorConfiguration.java
index d295501..8983b15 100644
--- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkOperatorConfiguration.java
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkOperatorConfiguration.java
@@ -19,6 +19,7 @@
 package org.apache.flink.kubernetes.operator.config;
 
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.operator.metrics.KubernetesOperatorMetricOptions;
 import org.apache.flink.kubernetes.operator.utils.EnvUtils;
 
 import io.javaoperatorsdk.operator.api.config.RetryConfiguration;
@@ -42,8 +43,8 @@ public class FlinkOperatorConfiguration {
     Duration flinkClientTimeout;
     String flinkServiceHostOverride;
     Set<String> watchedNamespaces;
-    Boolean dynamicNamespacesEnabled;
-    Boolean josdkMetricsEnabled;
+    boolean dynamicNamespacesEnabled;
+    boolean josdkMetricsEnabled;
     Duration flinkCancelJobTimeout;
     Duration flinkShutdownClusterTimeout;
     String artifactsBaseDir;
@@ -110,7 +111,7 @@ public class FlinkOperatorConfiguration {
                         KubernetesOperatorConfigOptions.OPERATOR_DYNAMIC_NAMESPACES_ENABLED);
 
         boolean josdkMetricsEnabled =
-                operatorConfig.get(KubernetesOperatorConfigOptions.OPERATOR_JOSDK_METRICS_ENABLED);
+                operatorConfig.get(KubernetesOperatorMetricOptions.OPERATOR_JOSDK_METRICS_ENABLED);
 
         RetryConfiguration retryConfiguration = new FlinkOperatorRetryConfiguration(operatorConfig);
 
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/KubernetesOperatorConfigOptions.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/KubernetesOperatorConfigOptions.java
index b01e845..d76aaef 100644
--- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/KubernetesOperatorConfigOptions.java
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/KubernetesOperatorConfigOptions.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.kubernetes.operator.config;
 
+import org.apache.flink.annotation.docs.Documentation;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
 
@@ -30,6 +31,11 @@ import java.util.Map;
 /** This class holds configuration constants used by flink operator. */
 public class KubernetesOperatorConfigOptions {
 
+    public static final String SECTION_SYSTEM = "system";
+    public static final String SECTION_ADVANCED = "system_advanced";
+    public static final String SECTION_DYNAMIC = "dynamic";
+
+    @Documentation.Section(SECTION_SYSTEM)
     public static final ConfigOption<Duration> OPERATOR_RECONCILE_INTERVAL =
             ConfigOptions.key("kubernetes.operator.reconcile.interval")
                     .durationType()
@@ -38,6 +44,7 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "The interval for the controller to reschedule the reconcile process.");
 
+    @Documentation.Section(SECTION_ADVANCED)
     public static final ConfigOption<Duration> OPERATOR_OBSERVER_REST_READY_DELAY =
             ConfigOptions.key("kubernetes.operator.observer.rest-ready.delay")
                     .durationType()
@@ -45,6 +52,7 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "Final delay before deployment is marked ready after port becomes accessible.");
 
+    @Documentation.Section(SECTION_SYSTEM)
     public static final ConfigOption<Integer> OPERATOR_RECONCILE_PARALLELISM =
             ConfigOptions.key("kubernetes.operator.reconcile.parallelism")
                     .intType()
@@ -53,6 +61,7 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "The maximum number of threads running the reconciliation loop. Use -1 for infinite.");
 
+    @Documentation.Section(SECTION_ADVANCED)
     public static final ConfigOption<Duration> OPERATOR_OBSERVER_PROGRESS_CHECK_INTERVAL =
             ConfigOptions.key("kubernetes.operator.observer.progress-check.interval")
                     .durationType()
@@ -60,6 +69,7 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "The interval for observing status for in-progress operations such as deployment and savepoints.");
 
+    @Documentation.Section(SECTION_DYNAMIC)
     public static final ConfigOption<Duration> OPERATOR_SAVEPOINT_TRIGGER_GRACE_PERIOD =
             ConfigOptions.key("kubernetes.operator.savepoint.trigger.grace-period")
                     .durationType()
@@ -69,6 +79,7 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "The interval before a savepoint trigger attempt is marked as unsuccessful.");
 
+    @Documentation.Section(SECTION_SYSTEM)
     public static final ConfigOption<Duration> OPERATOR_FLINK_CLIENT_TIMEOUT =
             ConfigOptions.key("kubernetes.operator.flink.client.timeout")
                     .durationType()
@@ -77,6 +88,7 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "The timeout for the observer to wait the flink rest client to return.");
 
+    @Documentation.Section(SECTION_SYSTEM)
     public static final ConfigOption<Duration> OPERATOR_FLINK_CLIENT_CANCEL_TIMEOUT =
             ConfigOptions.key("kubernetes.operator.flink.client.cancel.timeout")
                     .durationType()
@@ -85,6 +97,7 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "The timeout for the reconciler to wait for flink to cancel job.");
 
+    @Documentation.Section(SECTION_SYSTEM)
     public static final ConfigOption<Duration> OPERATOR_RESOURCE_CLEANUP_TIMEOUT =
             ConfigOptions.key("kubernetes.operator.resource.cleanup.timeout")
                     .durationType()
@@ -94,12 +107,14 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "The timeout for the resource clean up to wait for flink to shutdown cluster.");
 
+    @Documentation.Section(SECTION_DYNAMIC)
     public static final ConfigOption<Boolean> DEPLOYMENT_ROLLBACK_ENABLED =
             ConfigOptions.key("kubernetes.operator.deployment.rollback.enabled")
                     .booleanType()
                     .defaultValue(false)
                     .withDescription("Whether to enable rolling back failed deployment upgrades.");
 
+    @Documentation.Section(SECTION_DYNAMIC)
     public static final ConfigOption<Duration> DEPLOYMENT_READINESS_TIMEOUT =
             ConfigOptions.key("kubernetes.operator.deployment.readiness.timeout")
                     .durationType()
@@ -108,18 +123,21 @@ public class KubernetesOperatorConfigOptions {
                             "The timeout for deployments to become ready/stable "
                                     + "before being rolled back if rollback is enabled.");
 
+    @Documentation.Section(SECTION_SYSTEM)
     public static final ConfigOption<String> OPERATOR_USER_ARTIFACTS_BASE_DIR =
             ConfigOptions.key("kubernetes.operator.user.artifacts.base.dir")
                     .stringType()
                     .defaultValue("/opt/flink/artifacts")
                     .withDescription("The base dir to put the session job artifacts.");
 
+    @Documentation.Section(SECTION_DYNAMIC)
     public static final ConfigOption<Boolean> JOB_UPGRADE_IGNORE_PENDING_SAVEPOINT =
             ConfigOptions.key("kubernetes.operator.job.upgrade.ignore-pending-savepoint")
                     .booleanType()
                     .defaultValue(false)
                     .withDescription("Whether to ignore pending savepoint during job upgrade.");
 
+    @Documentation.Section(SECTION_ADVANCED)
     public static final ConfigOption<Boolean> OPERATOR_DYNAMIC_CONFIG_ENABLED =
             ConfigOptions.key("kubernetes.operator.dynamic.config.enabled")
                     .booleanType()
@@ -127,24 +145,28 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "Whether to enable on-the-fly config changes through the operator configmap.");
 
+    @Documentation.Section(SECTION_ADVANCED)
     public static final ConfigOption<Duration> OPERATOR_DYNAMIC_CONFIG_CHECK_INTERVAL =
             ConfigOptions.key("kubernetes.operator.dynamic.config.check.interval")
                     .durationType()
                     .defaultValue(Duration.ofMinutes(5))
                     .withDescription("Time interval for checking config changes.");
 
+    @Documentation.Section(SECTION_ADVANCED)
     public static final ConfigOption<Duration> OPERATOR_CONFIG_CACHE_TIMEOUT =
             ConfigOptions.key("kubernetes.operator.config.cache.timeout")
                     .durationType()
                     .defaultValue(Duration.ofMinutes(10))
                     .withDescription("Expiration time for cached configs.");
 
+    @Documentation.Section(SECTION_ADVANCED)
     public static final ConfigOption<Integer> OPERATOR_CONFIG_CACHE_SIZE =
             ConfigOptions.key("kubernetes.operator.config.cache.size")
                     .intType()
                     .defaultValue(1000)
                     .withDescription("Max config cache size.");
 
+    @Documentation.Section(SECTION_DYNAMIC)
     public static final ConfigOption<Boolean> OPERATOR_JM_DEPLOYMENT_RECOVERY_ENABLED =
             ConfigOptions.key("kubernetes.operator.jm-deployment-recovery.enabled")
                     .booleanType()
@@ -154,12 +176,14 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "Whether to enable recovery of missing/deleted jobmanager deployments.");
 
+    @Documentation.Section(SECTION_DYNAMIC)
     public static final ConfigOption<Integer> OPERATOR_SAVEPOINT_HISTORY_MAX_COUNT =
             ConfigOptions.key("kubernetes.operator.savepoint.history.max.count")
                     .intType()
                     .defaultValue(10)
                     .withDescription("Maximum number of savepoint history entries to retain.");
 
+    @Documentation.Section(SECTION_ADVANCED)
     public static final ConfigOption<Integer> OPERATOR_SAVEPOINT_HISTORY_MAX_COUNT_THRESHOLD =
             ConfigOptions.key(OPERATOR_SAVEPOINT_HISTORY_MAX_COUNT.key() + ".threshold")
                     .intType()
@@ -167,6 +191,7 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "Maximum number threshold of savepoint history entries to retain.");
 
+    @Documentation.Section(SECTION_DYNAMIC)
     public static final ConfigOption<Duration> OPERATOR_SAVEPOINT_HISTORY_MAX_AGE =
             ConfigOptions.key("kubernetes.operator.savepoint.history.max.age")
                     .durationType()
@@ -174,6 +199,7 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "Maximum age for savepoint history entries to retain. Due to lazy clean-up, the most recent savepoint may live longer than the max age.");
 
+    @Documentation.Section(SECTION_ADVANCED)
     public static final ConfigOption<Duration> OPERATOR_SAVEPOINT_HISTORY_MAX_AGE_THRESHOLD =
             ConfigOptions.key(OPERATOR_SAVEPOINT_HISTORY_MAX_AGE.key() + ".threshold")
                     .durationType()
@@ -181,6 +207,7 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "Maximum age threshold for savepoint history entries to retain.");
 
+    @Documentation.Section(SECTION_DYNAMIC)
     public static final ConfigOption<Map<String, String>> JAR_ARTIFACT_HTTP_HEADER =
             ConfigOptions.key("kubernetes.operator.user.artifacts.http.header")
                     .mapType()
@@ -189,6 +216,7 @@ public class KubernetesOperatorConfigOptions {
                             "Custom HTTP header for HttpArtifactFetcher. The header will be applied when getting the session job artifacts. "
                                     + "Expected format: headerKey1:headerValue1,headerKey2:headerValue2.");
 
+    @Documentation.Section(SECTION_DYNAMIC)
     public static final ConfigOption<Duration> PERIODIC_SAVEPOINT_INTERVAL =
             ConfigOptions.key("kubernetes.operator.periodic.savepoint.interval")
                     .durationType()
@@ -197,6 +225,7 @@ public class KubernetesOperatorConfigOptions {
                             "Interval at which periodic savepoints will be triggered. "
                                     + "The triggering schedule is not guaranteed, savepoints will be triggered as part of the regular reconcile loop.");
 
+    @Documentation.Section(SECTION_SYSTEM)
     public static final ConfigOption<String> OPERATOR_WATCHED_NAMESPACES =
             ConfigOptions.key("kubernetes.operator.watched.namespaces")
                     .stringType()
@@ -204,19 +233,14 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "Comma separated list of namespaces the operator monitors for custom resources.");
 
+    @Documentation.Section(SECTION_SYSTEM)
     public static final ConfigOption<Boolean> OPERATOR_DYNAMIC_NAMESPACES_ENABLED =
             ConfigOptions.key("kubernetes.operator.dynamic.namespaces.enabled")
                     .booleanType()
                     .defaultValue(false)
                     .withDescription("Enables dynamic change of watched/monitored namespaces.");
 
-    public static final ConfigOption<Boolean> OPERATOR_JOSDK_METRICS_ENABLED =
-            ConfigOptions.key("kubernetes.operator.josdk.metrics.enabled")
-                    .booleanType()
-                    .defaultValue(true)
-                    .withDescription(
-                            "Enable forwarding of Java Operator SDK metrics to the Flink metric registry.");
-
+    @Documentation.Section(SECTION_SYSTEM)
     public static final ConfigOption<Duration> OPERATOR_RETRY_INITIAL_INTERVAL =
             ConfigOptions.key("kubernetes.operator.retry.initial.interval")
                     .durationType()
@@ -224,6 +248,7 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "Initial interval of automatic reconcile retries on recoverable errors.");
 
+    @Documentation.Section(SECTION_SYSTEM)
     public static final ConfigOption<Double> OPERATOR_RETRY_INTERVAL_MULTIPLIER =
             ConfigOptions.key("kubernetes.operator.retry.interval.multiplier")
                     .doubleType()
@@ -231,6 +256,7 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "Interval multiplier of automatic reconcile retries on recoverable errors.");
 
+    @Documentation.Section(SECTION_SYSTEM)
     public static final ConfigOption<Integer> OPERATOR_RETRY_MAX_ATTEMPTS =
             ConfigOptions.key("kubernetes.operator.retry.max.attempts")
                     .intType()
@@ -238,6 +264,7 @@ public class KubernetesOperatorConfigOptions {
                     .withDescription(
                             "Max attempts of automatic reconcile retries on recoverable errors.");
 
+    @Documentation.Section(SECTION_DYNAMIC)
     public static final ConfigOption<Boolean> OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED =
             ConfigOptions.key("kubernetes.operator.job.upgrade.last-state-fallback.enabled")
                     .booleanType()
diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/metrics/KubernetesOperatorMetricOptions.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/metrics/KubernetesOperatorMetricOptions.java
index 20e0f94..924fc4c 100644
--- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/metrics/KubernetesOperatorMetricOptions.java
+++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/metrics/KubernetesOperatorMetricOptions.java
@@ -22,6 +22,14 @@ import org.apache.flink.configuration.ConfigOptions;
 
 /** Configuration options for metrics. */
 public class KubernetesOperatorMetricOptions {
+
+    public static final ConfigOption<Boolean> OPERATOR_JOSDK_METRICS_ENABLED =
+            ConfigOptions.key("kubernetes.operator.josdk.metrics.enabled")
+                    .booleanType()
+                    .defaultValue(true)
+                    .withDescription(
+                            "Enable forwarding of Java Operator SDK metrics to the Flink metric registry.");
+
     public static final ConfigOption<String> SCOPE_NAMING_KUBERNETES_OPERATOR =
             ConfigOptions.key("metrics.scope.k8soperator.system")
                     .defaultValue("<host>.k8soperator.<namespace>.<name>.system")
diff --git a/flink-kubernetes-webhook/src/main/java/org/apache/flink/kubernetes/operator/admission/FlinkOperatorWebhook.java b/flink-kubernetes-webhook/src/main/java/org/apache/flink/kubernetes/operator/admission/FlinkOperatorWebhook.java
index 401eacb..df2f9e3 100644
--- a/flink-kubernetes-webhook/src/main/java/org/apache/flink/kubernetes/operator/admission/FlinkOperatorWebhook.java
+++ b/flink-kubernetes-webhook/src/main/java/org/apache/flink/kubernetes/operator/admission/FlinkOperatorWebhook.java
@@ -62,7 +62,7 @@ public class FlinkOperatorWebhook {
         EnvUtils.logEnvironmentInfo(LOG, "Flink Kubernetes Webhook", args);
         var informerManager = new InformerManager(new DefaultKubernetesClient());
         var configManager = new FlinkConfigManager(informerManager::setNamespaces);
-        if (!configManager.getOperatorConfiguration().getDynamicNamespacesEnabled()) {
+        if (!configManager.getOperatorConfiguration().isDynamicNamespacesEnabled()) {
             informerManager.setNamespaces(
                     configManager.getOperatorConfiguration().getWatchedNamespaces());
         }