You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by GitBox <gi...@apache.org> on 2022/02/21 09:50:02 UTC

[GitHub] [flink-kubernetes-operator] bgeng777 opened a new pull request #9: [FLINK-26163] Refactor FlinkUtils#getEffectiveConfig into smaller pieces

bgeng777 opened a new pull request #9:
URL: https://github.com/apache/flink-kubernetes-operator/pull/9


   1. Refactor `FlinkUtils#getEffectiveConfig` into smaller pieces.
   2. Fix the problem that the field of `replicas` in `JobManagerSpec` will not take effect.
   3. Fix the problem that the basic `podTemplate` in `FlinkDeploymentSpec` will not take effect if JM or TM spec does not specify `podTemplate` on their own.
   4. Refactor some codes in `TestUtils` and add unit test for `FlinkUtils#getEffectiveConfig`.


-- 
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: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] asfgit closed pull request #9: [FLINK-26163] Refactor FlinkUtils#getEffectiveConfig into smaller pieces

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #9:
URL: https://github.com/apache/flink-kubernetes-operator/pull/9


   


-- 
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: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] bgeng777 commented on pull request #9: [FLINK-26163] Refactor FlinkUtils#getEffectiveConfig into smaller pieces

Posted by GitBox <gi...@apache.org>.
bgeng777 commented on pull request #9:
URL: https://github.com/apache/flink-kubernetes-operator/pull/9#issuecomment-1046699367


   cc @wangyang0918 @gyfora @tweise 


-- 
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: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] tweise commented on pull request #9: [FLINK-26163] Refactor FlinkUtils#getEffectiveConfig into smaller pieces

Posted by GitBox <gi...@apache.org>.
tweise commented on pull request #9:
URL: https://github.com/apache/flink-kubernetes-operator/pull/9#issuecomment-1047253336


   Also see #10 for related field changes


-- 
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: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] bgeng777 commented on pull request #9: [FLINK-26163] Refactor FlinkUtils#getEffectiveConfig into smaller pieces

Posted by GitBox <gi...@apache.org>.
bgeng777 commented on pull request #9:
URL: https://github.com/apache/flink-kubernetes-operator/pull/9#issuecomment-1046823084


   1. Sure. It seems that the fix of JM replicas is just some lines of codes so I did it here. I will move it out and leave the work to the separate [jira](https://issues.apache.org/jira/browse/FLINK-26216).
   2. My bad. Check the `KUBERNETES_POD_TEMPLATE` field again and I understood it wrongly before.
   3. Yes, I also think about using Builder pattern when refactoring. Considering not to change the code style too much, I did not use it initially. But I believe it could make codes more neat. Will update it 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.

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] gyfora commented on a change in pull request #9: [FLINK-26163] Refactor FlinkUtils#getEffectiveConfig into smaller pieces

Posted by GitBox <gi...@apache.org>.
gyfora commented on a change in pull request #9:
URL: https://github.com/apache/flink-kubernetes-operator/pull/9#discussion_r811273546



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/FlinkUtils.java
##########
@@ -55,109 +43,19 @@
     private static final ObjectMapper MAPPER = new ObjectMapper();
 
     public static Configuration getEffectiveConfig(FlinkDeployment flinkApp) {
-        String namespace = flinkApp.getMetadata().getNamespace();
-        String clusterId = flinkApp.getMetadata().getName();
-        FlinkDeploymentSpec spec = flinkApp.getSpec();
-
         try {
-            String flinkConfDir = System.getenv().get(ConfigConstants.ENV_FLINK_CONF_DIR);
-            Configuration effectiveConfig = loadConfiguration(flinkConfDir);
-
-            effectiveConfig.setString(KubernetesConfigOptions.NAMESPACE, namespace);
-            effectiveConfig.setString(KubernetesConfigOptions.CLUSTER_ID, clusterId);
-
-            if (spec.getIngressDomain() != null) {
-                effectiveConfig.set(
-                        KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE,
-                        KubernetesConfigOptions.ServiceExposedType.ClusterIP);
-            }
-
-            if (spec.getJob() != null) {
-                effectiveConfig.set(
-                        DeploymentOptions.TARGET, KubernetesDeploymentTarget.APPLICATION.getName());
-            } else {
-                effectiveConfig.set(
-                        DeploymentOptions.TARGET, KubernetesDeploymentTarget.SESSION.getName());
-            }
-
-            if (!StringUtils.isNullOrWhitespaceOnly(spec.getImage())) {
-                effectiveConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE, spec.getImage());
-            }
-
-            if (!StringUtils.isNullOrWhitespaceOnly(spec.getImagePullPolicy())) {
-                effectiveConfig.set(
-                        KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY,
-                        KubernetesConfigOptions.ImagePullPolicy.valueOf(spec.getImagePullPolicy()));
-            }
-
-            if (spec.getFlinkConfiguration() != null && !spec.getFlinkConfiguration().isEmpty()) {
-                spec.getFlinkConfiguration().forEach(effectiveConfig::setString);
-            }
-
-            // Pod template
-            if (spec.getPodTemplate() != null) {
-                effectiveConfig.set(
-                        KubernetesConfigOptions.KUBERNETES_POD_TEMPLATE,
-                        createTempFile(spec.getPodTemplate()));
-            }
-
-            if (spec.getJobManager() != null) {
-                if (spec.getJobManager().getResource() != null) {
-                    effectiveConfig.setString(
-                            JobManagerOptions.TOTAL_PROCESS_MEMORY.key(),
-                            spec.getJobManager().getResource().getMemory());
-                    effectiveConfig.set(
-                            KubernetesConfigOptions.JOB_MANAGER_CPU,
-                            spec.getJobManager().getResource().getCpu());
-                }
-
-                if (spec.getJobManager().getPodTemplate() != null) {
-                    effectiveConfig.set(
-                            KubernetesConfigOptions.JOB_MANAGER_POD_TEMPLATE,
-                            createTempFile(
-                                    mergePodTemplates(
-                                            spec.getPodTemplate(),
-                                            spec.getJobManager().getPodTemplate())));
-                }
-            }
-
-            if (spec.getTaskManager() != null) {
-                if (spec.getTaskManager().getTaskSlots() > 0) {
-                    effectiveConfig.set(
-                            TaskManagerOptions.NUM_TASK_SLOTS,
-                            spec.getTaskManager().getTaskSlots());
-                }
-
-                if (spec.getTaskManager().getResource() != null) {
-                    effectiveConfig.setString(
-                            TaskManagerOptions.TOTAL_PROCESS_MEMORY.key(),
-                            spec.getTaskManager().getResource().getMemory());
-                    effectiveConfig.set(
-                            KubernetesConfigOptions.TASK_MANAGER_CPU,
-                            spec.getTaskManager().getResource().getCpu());
-                }
-
-                if (spec.getTaskManager().getPodTemplate() != null) {
-                    effectiveConfig.set(
-                            KubernetesConfigOptions.TASK_MANAGER_POD_TEMPLATE,
-                            createTempFile(
-                                    mergePodTemplates(
-                                            spec.getPodTemplate(),
-                                            spec.getTaskManager().getPodTemplate())));
-                }
-            }
-
-            if (spec.getJob() != null) {
-                final URI uri = new URI(spec.getJob().getJarURI());
-                effectiveConfig.set(
-                        PipelineOptions.JARS, Collections.singletonList(uri.toString()));
-
-                if (spec.getJob().getParallelism() > 0) {
-                    effectiveConfig.set(
-                            CoreOptions.DEFAULT_PARALLELISM, spec.getJob().getParallelism());
-                }
-            }
-
+            final Configuration effectiveConfig =
+                    new FlinkConfigBuilder(flinkApp)
+                            .applyFlinkConfiguration()
+                            .applyImage()
+                            .applyImagePullPolicy()
+                            .applyCommonPodTemplate()
+                            .applyIngressDomain()
+                            .applyJobManagerSpec()
+                            .applyTaskManagerSpec()
+                            .applyJobOrSessionSpec()
+                            .build();

Review comment:
       This could be part of the FlinkConfigBuilder as a static method.
   
   ```
   public static Configuration buildFor(FlinkDeployment dep) {...}
   ```

##########
File path: flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/FlinkConfigBuilderTest.java
##########
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.operator.utils;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.DeploymentOptions;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.PipelineOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesDeploymentTarget;
+import org.apache.flink.kubernetes.operator.TestUtils;
+import org.apache.flink.kubernetes.operator.crd.FlinkDeployment;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.Pod;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import static org.apache.flink.kubernetes.operator.TestUtils.IMAGE;
+import static org.apache.flink.kubernetes.operator.TestUtils.IMAGE_POLICY;
+import static org.apache.flink.kubernetes.operator.TestUtils.SAMPLE_JAR;
+import static org.apache.flink.kubernetes.operator.TestUtils.SERVICE_ACCOUNT;
+
+/** FlinkConfigBuilderTest. */
+public class FlinkConfigBuilderTest {
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(new YAMLFactory());
+    private static FlinkDeployment flinkDeployment;
+
+    @BeforeAll
+    public static void prepareFlinkDeployment() {
+        flinkDeployment = TestUtils.buildApplicationCluster();
+        Container container0 = new Container();
+        container0.setName("container0");
+        Pod pod0 =
+                TestUtils.getTestPod(
+                        "pod0 hostname", "pod0 api version", Arrays.asList(container0));
+        Pod pod1 = TestUtils.getTestPod("pod1 hostname", "pod1 api version", new ArrayList<>());
+        Pod pod2 = TestUtils.getTestPod("pod2 hostname", "pod2 api version", new ArrayList<>());
+
+        flinkDeployment.getSpec().setPodTemplate(pod0);
+        flinkDeployment.getSpec().setIngressDomain("test.com");
+        flinkDeployment.getSpec().getJobManager().setPodTemplate(pod1);
+        flinkDeployment.getSpec().getTaskManager().setPodTemplate(pod2);
+        flinkDeployment.getSpec().getJob().setParallelism(2);
+    }
+
+    @Test
+    public void testApplyImage() {
+        Configuration configuration = new FlinkConfigBuilder(flinkDeployment).applyImage().build();
+        System.out.println(configuration);

Review comment:
       remove sysout please

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/FlinkUtils.java
##########
@@ -172,7 +70,7 @@ public static Configuration loadConfiguration(String confDir) {
         return configuration;
     }
 
-    private static String createTempFile(Pod podTemplate) throws IOException {
+    public static String createTempFile(Pod podTemplate) throws IOException {

Review comment:
       We could move this to the FlinkConfigbuilder and keep as private




-- 
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: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] bgeng777 edited a comment on pull request #9: [FLINK-26163] Refactor FlinkUtils#getEffectiveConfig into smaller pieces

Posted by GitBox <gi...@apache.org>.
bgeng777 edited a comment on pull request #9:
URL: https://github.com/apache/flink-kubernetes-operator/pull/9#issuecomment-1046823084


   1. Sure. It seems that the fix of JM replicas is just some lines of codes so I did it here. I will move it out and leave the work to the separate [jira](https://issues.apache.org/jira/browse/FLINK-26216) for better tracking.
   2. My bad. Check the `KUBERNETES_POD_TEMPLATE` field again and I understood it wrongly before.
   3. Yes, I also think about using Builder pattern when refactoring. Considering not to change the code style too much, I did not use it initially. But I believe it could make codes more neat. Will update it 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.

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] wangyang0918 commented on pull request #9: [FLINK-26163] Refactor FlinkUtils#getEffectiveConfig into smaller pieces

Posted by GitBox <gi...@apache.org>.
wangyang0918 commented on pull request #9:
URL: https://github.com/apache/flink-kubernetes-operator/pull/9#issuecomment-1046798237


   > 1. Refactor `FlinkUtils#getEffectiveConfig` into smaller pieces.
   > 2. Fix the problem that the field of `replicas` in `JobManagerSpec` will not take effect.
   > 3. Fix the problem that the basic `podTemplate` in `FlinkDeploymentSpec` will not take effect if JM or TM spec does not specify `podTemplate` on their own.
   > 4. Refactor some codes in `TestUtils` and add unit test for `FlinkUtils#getEffectiveConfig`.
   
   * I am suggesting to factor out No.2 into a separate ticket.
   * Why we have the No.3 issue? IIUC, currently the pod template should work for both JobManager and TaskManager.
   * For No.1, what do you think about introducing a `FlinkConfigBuilder` just like following. It will make things easier to test.
   ```
   /** Build effective flink config from {@link FlinkDeployment} */
   public class FlinkConfigBuilder {
       private final FlinkDeployment deploy;
       private final Configuration effectiveConfig;
   
       public FlinkConfigBuilder(FlinkDeployment deploy) {
           this.deploy = deploy;
           this.effectiveConfig = FlinkUtils.loadConfiguration();
       }
   
       public FlinkConfigBuilder applyImageToConfig() {
           effectiveConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE, deploy.getSpec().getImage());
           return this;
       }
   
       // Apply other fields to the config
       ... ...
   
       public Configuration build() {
           // Set some basic configuration
           final String namespace = deploy.getMetadata().getNamespace();
           final String clusterId = deploy.getMetadata().getName();
           effectiveConfig.setString(KubernetesConfigOptions.NAMESPACE, namespace);
           effectiveConfig.setString(KubernetesConfigOptions.CLUSTER_ID, clusterId);
           return effectiveConfig;
       }
   }
   ```
   
   ```
   final Configuration effectiveConfig = new FlinkConfigBuilder(flinkApp).applyImageToConfig().apply<...>.build();
   ```


-- 
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: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] bgeng777 edited a comment on pull request #9: [FLINK-26163] Refactor FlinkUtils#getEffectiveConfig into smaller pieces

Posted by GitBox <gi...@apache.org>.
bgeng777 edited a comment on pull request #9:
URL: https://github.com/apache/flink-kubernetes-operator/pull/9#issuecomment-1046823084


   Thanks a lot for the detailed review!
   1. Sure. It seems that the fix of JM replicas is just a piece of codes so I did it here. I will move it out and leave the work to the dedicated [jira](https://issues.apache.org/jira/browse/FLINK-26216) for better tracking.
   2. My bad. Check the `KUBERNETES_POD_TEMPLATE` field again and I understood it wrongly before.
   3. Yes, I also think about using Builder pattern when refactoring. Considering not to change the code style too much, I did not use it initially. But I believe it could make codes more neat and flexible. Updated.


-- 
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: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] bgeng777 commented on a change in pull request #9: [FLINK-26163] Refactor FlinkUtils#getEffectiveConfig into smaller pieces

Posted by GitBox <gi...@apache.org>.
bgeng777 commented on a change in pull request #9:
URL: https://github.com/apache/flink-kubernetes-operator/pull/9#discussion_r811551466



##########
File path: flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/FlinkConfigBuilderTest.java
##########
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.kubernetes.operator.utils;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.DeploymentOptions;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.PipelineOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
+import org.apache.flink.kubernetes.configuration.KubernetesDeploymentTarget;
+import org.apache.flink.kubernetes.operator.TestUtils;
+import org.apache.flink.kubernetes.operator.crd.FlinkDeployment;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import io.fabric8.kubernetes.api.model.Container;
+import io.fabric8.kubernetes.api.model.Pod;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import static org.apache.flink.kubernetes.operator.TestUtils.IMAGE;
+import static org.apache.flink.kubernetes.operator.TestUtils.IMAGE_POLICY;
+import static org.apache.flink.kubernetes.operator.TestUtils.SAMPLE_JAR;
+import static org.apache.flink.kubernetes.operator.TestUtils.SERVICE_ACCOUNT;
+
+/** FlinkConfigBuilderTest. */
+public class FlinkConfigBuilderTest {
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(new YAMLFactory());
+    private static FlinkDeployment flinkDeployment;
+
+    @BeforeAll
+    public static void prepareFlinkDeployment() {
+        flinkDeployment = TestUtils.buildApplicationCluster();
+        Container container0 = new Container();
+        container0.setName("container0");
+        Pod pod0 =
+                TestUtils.getTestPod(
+                        "pod0 hostname", "pod0 api version", Arrays.asList(container0));
+        Pod pod1 = TestUtils.getTestPod("pod1 hostname", "pod1 api version", new ArrayList<>());
+        Pod pod2 = TestUtils.getTestPod("pod2 hostname", "pod2 api version", new ArrayList<>());
+
+        flinkDeployment.getSpec().setPodTemplate(pod0);
+        flinkDeployment.getSpec().setIngressDomain("test.com");
+        flinkDeployment.getSpec().getJobManager().setPodTemplate(pod1);
+        flinkDeployment.getSpec().getTaskManager().setPodTemplate(pod2);
+        flinkDeployment.getSpec().getJob().setParallelism(2);
+    }
+
+    @Test
+    public void testApplyImage() {
+        Configuration configuration = new FlinkConfigBuilder(flinkDeployment).applyImage().build();
+        System.out.println(configuration);

Review comment:
       Fixed in a02f7185edfdac9e71e470f46b69d5f89c4dacab

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/FlinkUtils.java
##########
@@ -172,7 +70,7 @@ public static Configuration loadConfiguration(String confDir) {
         return configuration;
     }
 
-    private static String createTempFile(Pod podTemplate) throws IOException {
+    public static String createTempFile(Pod podTemplate) throws IOException {

Review comment:
       Moved in a02f7185edfdac9e71e470f46b69d5f89c4dacab

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/FlinkUtils.java
##########
@@ -55,109 +43,19 @@
     private static final ObjectMapper MAPPER = new ObjectMapper();
 
     public static Configuration getEffectiveConfig(FlinkDeployment flinkApp) {
-        String namespace = flinkApp.getMetadata().getNamespace();
-        String clusterId = flinkApp.getMetadata().getName();
-        FlinkDeploymentSpec spec = flinkApp.getSpec();
-
         try {
-            String flinkConfDir = System.getenv().get(ConfigConstants.ENV_FLINK_CONF_DIR);
-            Configuration effectiveConfig = loadConfiguration(flinkConfDir);
-
-            effectiveConfig.setString(KubernetesConfigOptions.NAMESPACE, namespace);
-            effectiveConfig.setString(KubernetesConfigOptions.CLUSTER_ID, clusterId);
-
-            if (spec.getIngressDomain() != null) {
-                effectiveConfig.set(
-                        KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE,
-                        KubernetesConfigOptions.ServiceExposedType.ClusterIP);
-            }
-
-            if (spec.getJob() != null) {
-                effectiveConfig.set(
-                        DeploymentOptions.TARGET, KubernetesDeploymentTarget.APPLICATION.getName());
-            } else {
-                effectiveConfig.set(
-                        DeploymentOptions.TARGET, KubernetesDeploymentTarget.SESSION.getName());
-            }
-
-            if (!StringUtils.isNullOrWhitespaceOnly(spec.getImage())) {
-                effectiveConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE, spec.getImage());
-            }
-
-            if (!StringUtils.isNullOrWhitespaceOnly(spec.getImagePullPolicy())) {
-                effectiveConfig.set(
-                        KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY,
-                        KubernetesConfigOptions.ImagePullPolicy.valueOf(spec.getImagePullPolicy()));
-            }
-
-            if (spec.getFlinkConfiguration() != null && !spec.getFlinkConfiguration().isEmpty()) {
-                spec.getFlinkConfiguration().forEach(effectiveConfig::setString);
-            }
-
-            // Pod template
-            if (spec.getPodTemplate() != null) {
-                effectiveConfig.set(
-                        KubernetesConfigOptions.KUBERNETES_POD_TEMPLATE,
-                        createTempFile(spec.getPodTemplate()));
-            }
-
-            if (spec.getJobManager() != null) {
-                if (spec.getJobManager().getResource() != null) {
-                    effectiveConfig.setString(
-                            JobManagerOptions.TOTAL_PROCESS_MEMORY.key(),
-                            spec.getJobManager().getResource().getMemory());
-                    effectiveConfig.set(
-                            KubernetesConfigOptions.JOB_MANAGER_CPU,
-                            spec.getJobManager().getResource().getCpu());
-                }
-
-                if (spec.getJobManager().getPodTemplate() != null) {
-                    effectiveConfig.set(
-                            KubernetesConfigOptions.JOB_MANAGER_POD_TEMPLATE,
-                            createTempFile(
-                                    mergePodTemplates(
-                                            spec.getPodTemplate(),
-                                            spec.getJobManager().getPodTemplate())));
-                }
-            }
-
-            if (spec.getTaskManager() != null) {
-                if (spec.getTaskManager().getTaskSlots() > 0) {
-                    effectiveConfig.set(
-                            TaskManagerOptions.NUM_TASK_SLOTS,
-                            spec.getTaskManager().getTaskSlots());
-                }
-
-                if (spec.getTaskManager().getResource() != null) {
-                    effectiveConfig.setString(
-                            TaskManagerOptions.TOTAL_PROCESS_MEMORY.key(),
-                            spec.getTaskManager().getResource().getMemory());
-                    effectiveConfig.set(
-                            KubernetesConfigOptions.TASK_MANAGER_CPU,
-                            spec.getTaskManager().getResource().getCpu());
-                }
-
-                if (spec.getTaskManager().getPodTemplate() != null) {
-                    effectiveConfig.set(
-                            KubernetesConfigOptions.TASK_MANAGER_POD_TEMPLATE,
-                            createTempFile(
-                                    mergePodTemplates(
-                                            spec.getPodTemplate(),
-                                            spec.getTaskManager().getPodTemplate())));
-                }
-            }
-
-            if (spec.getJob() != null) {
-                final URI uri = new URI(spec.getJob().getJarURI());
-                effectiveConfig.set(
-                        PipelineOptions.JARS, Collections.singletonList(uri.toString()));
-
-                if (spec.getJob().getParallelism() > 0) {
-                    effectiveConfig.set(
-                            CoreOptions.DEFAULT_PARALLELISM, spec.getJob().getParallelism());
-                }
-            }
-
+            final Configuration effectiveConfig =
+                    new FlinkConfigBuilder(flinkApp)
+                            .applyFlinkConfiguration()
+                            .applyImage()
+                            .applyImagePullPolicy()
+                            .applyCommonPodTemplate()
+                            .applyIngressDomain()
+                            .applyJobManagerSpec()
+                            .applyTaskManagerSpec()
+                            .applyJobOrSessionSpec()
+                            .build();

Review comment:
       Thanks for the advice. Fixed in a02f7185edfdac9e71e470f46b69d5f89c4dacab




-- 
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: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org