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/15 11:22:55 UTC

[GitHub] [flink-kubernetes-operator] dannycranmer commented on a change in pull request #1: [FLINK-26078] Kubernetes Operator Prototype

dannycranmer commented on a change in pull request #1:
URL: https://github.com/apache/flink-kubernetes-operator/pull/1#discussion_r806722565



##########
File path: flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/JobReconcilerTest.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.reconciler;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.operator.TestUtils;
+import org.apache.flink.kubernetes.operator.crd.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.crd.spec.UpgradeMode;
+import org.apache.flink.kubernetes.operator.crd.status.JobStatus;
+import org.apache.flink.kubernetes.operator.service.FlinkService;
+import org.apache.flink.kubernetes.operator.utils.FlinkUtils;
+import org.apache.flink.runtime.jobgraph.SavepointConfigOptions;
+
+import io.fabric8.kubernetes.client.KubernetesClient;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+import java.util.Optional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.times;
+
+/** @link JobStatusObserver unit tests */
+public class JobReconcilerTest {
+
+    public static final String JOB_NAME = "test1";
+    public static final String JOB_ID = "fd72014d4c864993a2e5a9287b4a9c5d";
+
+    private FlinkService flinkService = Mockito.mock(FlinkService.class);
+
+    @Test
+    public void testUpgrade() throws Exception {
+        KubernetesClient kubernetesClient = Mockito.mock(KubernetesClient.class);
+        JobReconciler reconciler = new JobReconciler(kubernetesClient, flinkService);
+        FlinkDeployment deployment = TestUtils.buildApplicationCluster();
+        Configuration config = FlinkUtils.getEffectiveConfig(deployment);
+
+        reconciler.reconcile("test", deployment, config);
+        Mockito.verify(flinkService, times(1)).submitApplicationCluster(eq(deployment), eq(config));
+        Mockito.clearInvocations(flinkService);
+        deployment.getStatus().setSpec(deployment.getSpec());
+
+        JobStatus jobStatus = new JobStatus();
+        jobStatus.setJobName(JOB_NAME);
+        jobStatus.setJobId(JOB_ID);
+        jobStatus.setState("RUNNING");
+
+        deployment.getStatus().setJobStatus(jobStatus);
+
+        // Test stateless upgrade
+        FlinkDeployment statelessUpgrade = TestUtils.clone(deployment);
+        statelessUpgrade.getSpec().getJob().setUpgradeMode(UpgradeMode.STATELESS);
+        statelessUpgrade.getSpec().getFlinkConfiguration().put("new", "conf");
+        reconciler.reconcile("test", statelessUpgrade, config);
+        Mockito.verify(flinkService, times(1))
+                .cancelJob(eq(JobID.fromHexString(JOB_ID)), eq(UpgradeMode.STATELESS), eq(config));
+
+        Mockito.verify(flinkService, times(1))

Review comment:
       Flink code style instructs us to [avoid using Mockito](https://flink.apache.org/contributing/code-style-and-quality-common.html#avoid-mockito---use-reusable-test-implementations). I am not sure whether these rules apply to other repos, but I assume so. Let's clarify this

##########
File path: pom.xml
##########
@@ -0,0 +1,285 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <parent>
+      <groupId>org.apache</groupId>
+      <artifactId>apache</artifactId>
+      <version>20</version>
+    </parent>
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <groupId>org.apache.flink</groupId>
+    <artifactId>flink-kubernetes-operator-parent</artifactId>
+    <version>1.0-SNAPSHOT</version>
+
+    <name>Flink Kubernetes: </name>
+    <packaging>pom</packaging>
+    <url>https://flink.apache.org</url>
+    <inceptionYear>2014</inceptionYear>
+
+    <licenses>
+      <license>
+        <name>The Apache Software License, Version 2.0</name>
+        <url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
+        <distribution>repo</distribution>
+      </license>
+    </licenses>
+
+    <modules>
+      <module>flink-kubernetes-operator</module>
+      <module>flink-kubernetes-webhook</module>
+    </modules>
+
+    <properties>
+        <maven.compiler.source>11</maven.compiler.source>
+        <maven.compiler.target>11</maven.compiler.target>
+        <maven-assembly-plugin.version>3.3.0</maven-assembly-plugin.version>
+        <maven-surefire-plugin.version>3.0.0-M4</maven-surefire-plugin.version>
+        <maven-failsafe-plugin.version>3.0.0-M4</maven-failsafe-plugin.version>
+        <maven-resources-plugin.version>3.2.0</maven-resources-plugin.version>
+
+        <operator.sdk.version>2.1.1</operator.sdk.version>
+        <fabric8.version>5.12.1</fabric8.version>
+        <lombok.version>1.18.22</lombok.version>
+
+        <scala.version>2.12</scala.version>
+        <flink.version>1.14.3</flink.version>
+        <flink.shaded.version>15.0</flink.shaded.version>

Review comment:
       Why are we using a newer version for shaded libs?

##########
File path: pom.xml
##########
@@ -0,0 +1,285 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <parent>
+      <groupId>org.apache</groupId>
+      <artifactId>apache</artifactId>
+      <version>20</version>
+    </parent>

Review comment:
       Should we use `flink-parent` here so we can inherit the same quality plugins etc?

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/utils/FlinkUtils.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.DeploymentOptions;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.JobManagerOptions;
+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.crd.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.crd.spec.FlinkDeploymentSpec;
+import org.apache.flink.util.StringUtils;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.fabric8.kubernetes.client.internal.SerializationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.util.Collections;
+import java.util.Iterator;
+
+/** Flink Utility methods used by the operator. */
+public class FlinkUtils {
+
+    private static final Logger LOG = LoggerFactory.getLogger(FlinkUtils.class);
+    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 =
+                    flinkConfDir != null
+                            ? GlobalConfiguration.loadConfiguration(flinkConfDir)
+                            : new Configuration();
+
+            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());
+                }
+            }
+
+            return effectiveConfig;
+        } catch (Exception e) {
+            throw new RuntimeException("Failed to load configuration", e);
+        }
+    }

Review comment:
       This method is very long, suggest splitting it out into smaller methods




-- 
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