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/03/26 07:53:53 UTC

[GitHub] [flink-kubernetes-operator] gyfora commented on a change in pull request #112: [FLINK-26787] Initial implementation of FlinkSessionJobController and…

gyfora commented on a change in pull request #112:
URL: https://github.com/apache/flink-kubernetes-operator/pull/112#discussion_r835733585



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/BaseObserver.java
##########
@@ -45,7 +45,7 @@
 import java.util.Optional;
 
 /** The base observer. */
-public abstract class BaseObserver implements Observer {
+public abstract class BaseObserver implements Observer<FlinkDeployment> {

Review comment:
       We could rename this BaseDeploymentObserver/AbstractDeploymentObserver and move deployment related stuff under a deployment package , similarly we could have a `job` package for sessionjobs

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/FlinkSessionJobReconciler.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.config.FlinkOperatorConfiguration;
+import org.apache.flink.kubernetes.operator.crd.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.crd.FlinkSessionJob;
+import org.apache.flink.kubernetes.operator.crd.spec.FlinkSessionJobSpec;
+import org.apache.flink.kubernetes.operator.observer.JobManagerDeploymentStatus;
+import org.apache.flink.kubernetes.operator.service.FlinkService;
+import org.apache.flink.kubernetes.operator.utils.FlinkUtils;
+
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.javaoperatorsdk.operator.api.reconciler.Context;
+import io.javaoperatorsdk.operator.api.reconciler.DeleteControl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+
+/** The reconciler for the {@link FlinkSessionJob}. */
+public class FlinkSessionJobReconciler implements Reconciler<FlinkSessionJob> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(JobReconciler.class);
+
+    private final FlinkOperatorConfiguration operatorConfiguration;
+    private final KubernetesClient kubernetesClient;
+    private final FlinkService flinkService;
+
+    public FlinkSessionJobReconciler(
+            KubernetesClient kubernetesClient,
+            FlinkService flinkService,
+            FlinkOperatorConfiguration operatorConfiguration) {
+        this.kubernetesClient = kubernetesClient;
+        this.flinkService = flinkService;
+        this.operatorConfiguration = operatorConfiguration;
+    }
+
+    @Override
+    public void reconcile(
+            FlinkSessionJob flinkSessionJob, Context context, Configuration defaultConfig)
+            throws Exception {
+
+        FlinkSessionJobSpec lastReconciledSpec =
+                flinkSessionJob.getStatus().getReconciliationStatus().getLastReconciledSpec();
+
+        if (lastReconciledSpec == null) {
+            deployFlinkJob(flinkSessionJob, defaultConfig);
+        }
+        // TODO reconcile other spec change.
+    }
+
+    @Override
+    public DeleteControl cleanup(FlinkSessionJob sessionJob, Configuration defaultConfig) {
+        Optional<FlinkDeployment> flinkDepOptional = getFlinkDeployment(sessionJob);
+
+        if (flinkDepOptional.isPresent()) {
+            Configuration effectiveConfig =
+                    FlinkUtils.getEffectiveConfig(flinkDepOptional.get(), defaultConfig);
+            String jobID = sessionJob.getStatus().getJobStatus().getJobId();
+            if (jobID != null) {
+                try {
+                    flinkService.cancelSessionJob(JobID.fromHexString(jobID), effectiveConfig);
+                } catch (Exception e) {
+                    LOG.error("Failed to cancel job.", e);
+                }
+            }
+        } else {
+            LOG.info("Session cluster has disappeared.");
+        }
+        return DeleteControl.defaultDelete();
+    }
+
+    private void deployFlinkJob(FlinkSessionJob sessionJob, Configuration defaultConfig)
+            throws Exception {
+        Optional<FlinkDeployment> flinkDepOptional = getFlinkDeployment(sessionJob);
+        if (flinkDepOptional.isPresent()
+                && (flinkDepOptional.get().getStatus().getJobManagerDeploymentStatus()
+                        == JobManagerDeploymentStatus.READY)) {
+            Configuration effectiveConfig =
+                    FlinkUtils.getEffectiveConfig(flinkDepOptional.get(), defaultConfig);
+            flinkService.submitJobToSessionCluster(sessionJob, effectiveConfig);
+            ReconciliationUtils.updateForSpecReconciliationSuccess(sessionJob);
+        } else {
+            LOG.info("Session cluster has not been ready");

Review comment:
       `Session cluster deployment is not in READY state`

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobController.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.controller;
+
+import org.apache.flink.kubernetes.operator.config.DefaultConfig;
+import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration;
+import org.apache.flink.kubernetes.operator.crd.FlinkSessionJob;
+import org.apache.flink.kubernetes.operator.exception.ReconciliationException;
+import org.apache.flink.kubernetes.operator.observer.Observer;
+import org.apache.flink.kubernetes.operator.reconciler.Reconciler;
+import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils;
+import org.apache.flink.kubernetes.operator.validation.InternalValidator;
+
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.javaoperatorsdk.operator.api.reconciler.Context;
+import io.javaoperatorsdk.operator.api.reconciler.ControllerConfiguration;
+import io.javaoperatorsdk.operator.api.reconciler.DeleteControl;
+import io.javaoperatorsdk.operator.api.reconciler.ErrorStatusHandler;
+import io.javaoperatorsdk.operator.api.reconciler.RetryInfo;
+import io.javaoperatorsdk.operator.api.reconciler.UpdateControl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+
+/** Controller that runs the main reconcile loop for {@link FlinkSessionJob}. */
+@ControllerConfiguration
+public class FlinkSessionJobController
+        implements io.javaoperatorsdk.operator.api.reconciler.Reconciler<FlinkSessionJob>,
+                ErrorStatusHandler<FlinkSessionJob> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(FlinkSessionJobController.class);
+    private final KubernetesClient kubernetesClient;
+
+    private final InternalValidator<FlinkSessionJob> validator;
+    private final Reconciler<FlinkSessionJob> reconciler;
+    private final Observer<FlinkSessionJob> observer;
+    private final DefaultConfig defaultConfig;
+    private final FlinkOperatorConfiguration operatorConfiguration;
+
+    private FlinkControllerConfig<FlinkSessionJob> controllerConfig;
+
+    public FlinkSessionJobController(
+            DefaultConfig defaultConfig,
+            FlinkOperatorConfiguration operatorConfiguration,
+            KubernetesClient kubernetesClient,
+            InternalValidator<FlinkSessionJob> validator,
+            Reconciler<FlinkSessionJob> reconciler,
+            Observer<FlinkSessionJob> observer) {
+        this.defaultConfig = defaultConfig;
+        this.operatorConfiguration = operatorConfiguration;
+        this.kubernetesClient = kubernetesClient;
+        this.validator = validator;
+        this.reconciler = reconciler;
+        this.observer = observer;
+    }
+
+    @Override
+    public UpdateControl<FlinkSessionJob> reconcile(
+            FlinkSessionJob flinkSessionJob, Context context) {
+        FlinkSessionJob originalCopy = ReconciliationUtils.clone(flinkSessionJob);
+        LOG.info("Starting reconciliation");
+        Optional<String> validationError = validator.validate(flinkSessionJob);
+        if (validationError.isPresent()) {
+            LOG.error("Validation failed: " + validationError.get());
+            ReconciliationUtils.updateForReconciliationError(
+                    flinkSessionJob, validationError.get());
+            return ReconciliationUtils.toUpdateControl(originalCopy, flinkSessionJob);
+        }
+
+        try {
+            // TODO refactor the reconciler interface to return UpdateControl directly

Review comment:
       I dont think the reconciler should return the updatecontrol, at least we have changed this a few times already before to the current state where we keep the logic in the ReconciliationUtils.

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/BaseReconciler.java
##########
@@ -28,7 +28,7 @@
 import io.javaoperatorsdk.operator.api.reconciler.DeleteControl;
 
 /** BaseReconciler with functionality that is common to job and session modes. */
-public abstract class BaseReconciler implements Reconciler {
+public abstract class BaseReconciler implements Reconciler<FlinkDeployment> {

Review comment:
       As with the BaseObserver this could be called BaseDeploymentReconcer / AbstractDeploymentReconciler and moved to a separate package

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/FlinkSessionJobReconciler.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.config.FlinkOperatorConfiguration;
+import org.apache.flink.kubernetes.operator.crd.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.crd.FlinkSessionJob;
+import org.apache.flink.kubernetes.operator.crd.spec.FlinkSessionJobSpec;
+import org.apache.flink.kubernetes.operator.observer.JobManagerDeploymentStatus;
+import org.apache.flink.kubernetes.operator.service.FlinkService;
+import org.apache.flink.kubernetes.operator.utils.FlinkUtils;
+
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.javaoperatorsdk.operator.api.reconciler.Context;
+import io.javaoperatorsdk.operator.api.reconciler.DeleteControl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+
+/** The reconciler for the {@link FlinkSessionJob}. */
+public class FlinkSessionJobReconciler implements Reconciler<FlinkSessionJob> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(JobReconciler.class);
+
+    private final FlinkOperatorConfiguration operatorConfiguration;
+    private final KubernetesClient kubernetesClient;
+    private final FlinkService flinkService;
+
+    public FlinkSessionJobReconciler(
+            KubernetesClient kubernetesClient,
+            FlinkService flinkService,
+            FlinkOperatorConfiguration operatorConfiguration) {
+        this.kubernetesClient = kubernetesClient;
+        this.flinkService = flinkService;
+        this.operatorConfiguration = operatorConfiguration;
+    }
+
+    @Override
+    public void reconcile(
+            FlinkSessionJob flinkSessionJob, Context context, Configuration defaultConfig)
+            throws Exception {
+
+        FlinkSessionJobSpec lastReconciledSpec =
+                flinkSessionJob.getStatus().getReconciliationStatus().getLastReconciledSpec();
+
+        if (lastReconciledSpec == null) {
+            deployFlinkJob(flinkSessionJob, defaultConfig);
+        }
+        // TODO reconcile other spec change.
+    }
+
+    @Override
+    public DeleteControl cleanup(FlinkSessionJob sessionJob, Configuration defaultConfig) {
+        Optional<FlinkDeployment> flinkDepOptional = getFlinkDeployment(sessionJob);
+
+        if (flinkDepOptional.isPresent()) {
+            Configuration effectiveConfig =
+                    FlinkUtils.getEffectiveConfig(flinkDepOptional.get(), defaultConfig);
+            String jobID = sessionJob.getStatus().getJobStatus().getJobId();
+            if (jobID != null) {
+                try {
+                    flinkService.cancelSessionJob(JobID.fromHexString(jobID), effectiveConfig);
+                } catch (Exception e) {
+                    LOG.error("Failed to cancel job.", e);
+                }
+            }
+        } else {
+            LOG.info("Session cluster has disappeared.");

Review comment:
       `Session cluster deployment not available`

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/FlinkSessionJobReconciler.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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;

Review comment:
       this could go into a `job`/`sessionjob` package

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/InternalValidator.java
##########
@@ -17,18 +17,20 @@
 
 package org.apache.flink.kubernetes.operator.validation;
 
-import org.apache.flink.kubernetes.operator.crd.FlinkDeployment;
-
 import java.util.Optional;
 
-/** Validator for {@link FlinkDeployment} resources. */
-public interface FlinkDeploymentValidator {
+/**
+ * Validator for custom resources.
+ *
+ * @param <CR> The custom resource to be validated.
+ */
+public interface InternalValidator<CR> {

Review comment:
       Maybe `FlinkResourceValidator` would be a vbetter name 

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/FlinkSessionJobReconciler.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.config.FlinkOperatorConfiguration;
+import org.apache.flink.kubernetes.operator.crd.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.crd.FlinkSessionJob;
+import org.apache.flink.kubernetes.operator.crd.spec.FlinkSessionJobSpec;
+import org.apache.flink.kubernetes.operator.observer.JobManagerDeploymentStatus;
+import org.apache.flink.kubernetes.operator.service.FlinkService;
+import org.apache.flink.kubernetes.operator.utils.FlinkUtils;
+
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.javaoperatorsdk.operator.api.reconciler.Context;
+import io.javaoperatorsdk.operator.api.reconciler.DeleteControl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Optional;
+
+/** The reconciler for the {@link FlinkSessionJob}. */
+public class FlinkSessionJobReconciler implements Reconciler<FlinkSessionJob> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(JobReconciler.class);
+
+    private final FlinkOperatorConfiguration operatorConfiguration;
+    private final KubernetesClient kubernetesClient;
+    private final FlinkService flinkService;
+
+    public FlinkSessionJobReconciler(
+            KubernetesClient kubernetesClient,
+            FlinkService flinkService,
+            FlinkOperatorConfiguration operatorConfiguration) {
+        this.kubernetesClient = kubernetesClient;
+        this.flinkService = flinkService;
+        this.operatorConfiguration = operatorConfiguration;
+    }
+
+    @Override
+    public void reconcile(
+            FlinkSessionJob flinkSessionJob, Context context, Configuration defaultConfig)
+            throws Exception {
+
+        FlinkSessionJobSpec lastReconciledSpec =
+                flinkSessionJob.getStatus().getReconciliationStatus().getLastReconciledSpec();
+
+        if (lastReconciledSpec == null) {
+            deployFlinkJob(flinkSessionJob, defaultConfig);
+        }
+        // TODO reconcile other spec change.

Review comment:
       We could at least log something here for the users for the time being

##########
File path: helm/flink-operator/templates/flink-operator.yaml
##########
@@ -123,6 +127,12 @@ spec:
                 path: flink-conf.yaml
               - key: log4j-console.properties
                 path: log4j-console.properties
+        {{- if .Values.hostPath.create }}
+        - name: {{ .Values.hostPath.name }}
+          hostPath:
+            path: {{ .Values.hostPath.hostPath }}
+            type: DirectoryOrCreate
+        {{- end }}

Review comment:
       This logic is minikube specific right? Maybe it would be better to allow the user to define a volume-mount directly. That way they can use hostPath / persistentvolumeclaim etc.

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/FlinkService.java
##########
@@ -116,6 +136,82 @@ public void submitSessionCluster(FlinkDeployment deployment, Configuration conf)
         LOG.info("Session cluster successfully deployed");
     }
 
+    public void submitJobToSessionCluster(FlinkSessionJob sessionJob, Configuration conf)
+            throws Exception {
+        JarRunResponseBody jarRunResponseBody =
+                jarUpload(sessionJob, conf)
+                        .handle(
+                                (response, throwable) -> {
+                                    if (throwable != null) {
+                                        LOG.error("Failed to upload user jar.", throwable);
+                                        throw new FlinkRuntimeException(throwable);
+                                    } else {
+                                        return jarRun(sessionJob, response, conf);
+                                    }
+                                })
+                        .get();
+
+        String jobID = jarRunResponseBody.getJobId().toHexString();
+        LOG.info("Submitted job: {} to session cluster.", jobID);
+        sessionJob.getStatus().setJobStatus(JobStatus.builder().jobId(jobID).build());
+    }
+
+    private JarRunResponseBody jarRun(
+            FlinkSessionJob sessionJob, JarUploadResponseBody response, Configuration conf) {
+        final String jarId =
+                response.getFilename().substring(response.getFilename().lastIndexOf("/") + 1);
+        // we generate jobID in advance to help deduplicate job submission.
+        JobID jobID = new JobID();
+        try {
+            final RestClusterClient<String> clusterClient =

Review comment:
       should use `try (RestClusterClient<String> clusterClient = ...) { } ..` pattern




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