SteNicholas commented on a change in pull request #26:
URL: 
https://github.com/apache/flink-kubernetes-operator/pull/26#discussion_r817845987



##########
File path: 
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/Observer.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.observer;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.kubernetes.operator.crd.FlinkDeployment;
+import org.apache.flink.kubernetes.operator.crd.status.FlinkDeploymentStatus;
+import org.apache.flink.kubernetes.operator.crd.status.JobStatus;
+import org.apache.flink.kubernetes.operator.service.FlinkService;
+import org.apache.flink.runtime.client.JobStatusMessage;
+
+import io.fabric8.kubernetes.api.model.apps.Deployment;
+import io.fabric8.kubernetes.api.model.apps.DeploymentSpec;
+import io.fabric8.kubernetes.api.model.apps.DeploymentStatus;
+import io.javaoperatorsdk.operator.api.reconciler.Context;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+
+/** Observes the actual state of the running jobs on the Flink cluster. */
+public class Observer {
+
+    private static final Logger LOG = LoggerFactory.getLogger(Observer.class);
+
+    private final FlinkService flinkService;
+
+    public Observer(FlinkService flinkService) {
+        this.flinkService = flinkService;
+    }
+
+    public boolean observe(
+            FlinkDeployment flinkApp, Context context, Configuration 
effectiveConfig) {
+        observeJmDeployment(flinkApp, context, effectiveConfig);
+        return isReadyToReconcile(flinkApp, effectiveConfig);
+    }
+
+    private void observeJmDeployment(
+            FlinkDeployment flinkApp, Context context, Configuration 
effectiveConfig) {
+        FlinkDeploymentStatus deploymentStatus = flinkApp.getStatus();
+        JobManagerDeploymentStatus previousJmStatus =
+                deploymentStatus.getJobManagerDeploymentStatus();
+
+        if (JobManagerDeploymentStatus.READY == previousJmStatus) {
+            return;
+        }
+
+        if (JobManagerDeploymentStatus.DEPLOYED_NOT_READY == previousJmStatus) 
{
+            
deploymentStatus.setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY);
+            return;
+        }
+
+        Optional<Deployment> deployment = 
context.getSecondaryResource(Deployment.class);
+        if (deployment.isPresent()) {
+            DeploymentStatus status = deployment.get().getStatus();
+            DeploymentSpec spec = deployment.get().getSpec();
+            if (status != null
+                    && status.getAvailableReplicas() != null
+                    && spec.getReplicas().intValue() == status.getReplicas()
+                    && spec.getReplicas().intValue() == 
status.getAvailableReplicas()
+                    && flinkService.isJobManagerPortReady(effectiveConfig)) {
+
+                // typically it takes a few seconds for the REST server to be 
ready
+                LOG.info(
+                        "JobManager deployment {} in namespace {} port ready, 
waiting for the REST API...",
+                        flinkApp.getMetadata().getName(),
+                        flinkApp.getMetadata().getNamespace());
+                deploymentStatus.setJobManagerDeploymentStatus(
+                        JobManagerDeploymentStatus.DEPLOYED_NOT_READY);
+                return;
+            }
+            LOG.info(
+                    "JobManager deployment {} in namespace {} exists but not 
ready yet, status {}",
+                    flinkApp.getMetadata().getName(),
+                    flinkApp.getMetadata().getNamespace(),
+                    status);
+
+            
deploymentStatus.setJobManagerDeploymentStatus(JobManagerDeploymentStatus.DEPLOYING);
+            return;
+        }
+
+        
deploymentStatus.setJobManagerDeploymentStatus(JobManagerDeploymentStatus.MISSING);
+    }
+
+    private boolean observeFlinkJobStatus(FlinkDeployment flinkApp, 
Configuration effectiveConfig) {
+
+        // No need to observe job status for session clusters
+        if (flinkApp.getSpec().getJob() == null) {
+            return true;
+        }
+
+        LOG.info("Getting job statuses for {}", 
flinkApp.getMetadata().getName());
+        FlinkDeploymentStatus flinkAppStatus = flinkApp.getStatus();
+
+        Collection<JobStatusMessage> clusterJobStatuses;
+        try {
+            clusterJobStatuses = flinkService.listJobs(effectiveConfig);
+        } catch (Exception e) {
+            LOG.error("Exception while listing jobs", e);
+            flinkAppStatus.getJobStatus().setState("UNKNOWN");

Review comment:
       @gyfora, thanks for your answer to my confusion. I got your point here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to