Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
17 commits
Select commit Hold shift + click to select a range
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,8 @@
* limitations under the License.
*/

package org.apache.flink.kubernetes.operator.config;
package org.apache.flink.kubernetes.operator.api;

import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec;

/** The mode of {@link FlinkDeployment}. */
Expand All @@ -45,7 +44,7 @@ public static Mode getMode(FlinkDeployment flinkApp) {
: getMode(lastReconciledSpec);
}

private static Mode getMode(FlinkDeploymentSpec spec) {
public static Mode getMode(FlinkDeploymentSpec spec) {
return spec.getJob() != null ? APPLICATION : SESSION;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,14 +21,17 @@
import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec;

import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import io.fabric8.kubernetes.api.model.Condition;
import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.EqualsAndHashCode;
import lombok.NoArgsConstructor;
import lombok.ToString;
import lombok.experimental.SuperBuilder;

import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

/** Last observed status of the Flink deployment. */
Expand All @@ -55,4 +58,7 @@ public class FlinkDeploymentStatus extends CommonStatus<FlinkDeploymentSpec> {

/** Information about the TaskManagers for the scale subresource. */
private TaskManagerInfo taskManager;

/** Condition of the CR . */
private List<Condition> conditions = new ArrayList<>();
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,18 +21,36 @@
public enum JobManagerDeploymentStatus {

/** JobManager is running and ready to receive REST API calls. */
READY,
READY("JobManagerReady", "JobManager is running and ready to receive REST API calls"),

/** JobManager is running but not ready yet to receive REST API calls. */
DEPLOYED_NOT_READY,
DEPLOYED_NOT_READY(
"DeployedNotReady",
"JobManager is running but not yet ready to receive REST API calls"),

/** JobManager process is starting up. */
DEPLOYING,
DEPLOYING("JobManagerIsDeploying", "JobManager process is starting up"),

/** JobManager deployment not found, probably not started or killed by user. */
// TODO: currently a mix of SUSPENDED and ERROR, needs cleanup
MISSING,
MISSING("JobManagerDeploymentMissing", "JobManager deployment not found"),

/** Deployment in terminal error, requires spec change for reconciliation to continue. */
ERROR;
ERROR("Error", "JobManager deployment failed");

private String reason;
private String message;

JobManagerDeploymentStatus(String reason, String message) {
this.reason = reason;
this.message = message;
}

public String getReason() {
return reason;
}

public String getMessage() {
return message;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,144 @@
/*
* 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.api.utils;

import org.apache.flink.api.common.JobStatus;
import org.apache.flink.kubernetes.operator.api.Mode;
import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec;
import org.apache.flink.kubernetes.operator.api.status.FlinkDeploymentReconciliationStatus;
import org.apache.flink.kubernetes.operator.api.status.FlinkDeploymentStatus;
import org.apache.flink.kubernetes.operator.api.status.JobManagerDeploymentStatus;

import io.fabric8.kubernetes.api.model.Condition;
import io.fabric8.kubernetes.api.model.ConditionBuilder;

import java.text.SimpleDateFormat;
import java.util.Date;
import java.util.List;

import static org.apache.flink.api.common.JobStatus.RUNNING;
import static org.apache.flink.kubernetes.operator.api.status.JobManagerDeploymentStatus.READY;

/** Creates a condition object with the type, status, message and reason. */
public class ConditionUtils {
public static final String CONDITION_TYPE_RUNNING = "Running";

/**
* Creates a List of Condition object based on the provided FlinkDeploymentStatus.
*
* @param flinkDeploymentStatus the FlinkDeploymentStatus object containing job status
* information
* @return a list of Condition object representing the current status of the Flink deployment
*/
public static List<Condition> createConditionFromStatus(
FlinkDeploymentStatus flinkDeploymentStatus) {

FlinkDeploymentReconciliationStatus reconciliationStatus =
flinkDeploymentStatus.getReconciliationStatus();
Condition conditionToAdd = null;

if (reconciliationStatus != null) {
FlinkDeploymentSpec deploymentSpec =
reconciliationStatus.deserializeLastReconciledSpec();

if (deploymentSpec != null) {
switch (Mode.getMode(deploymentSpec)) {
case APPLICATION:
conditionToAdd =
getApplicationModeCondition(
flinkDeploymentStatus.getJobStatus().getState());
break;
case SESSION:
conditionToAdd =
getSessionModeCondition(
flinkDeploymentStatus.getJobManagerDeploymentStatus());
}
updateLastTransitionTime(flinkDeploymentStatus.getConditions(), conditionToAdd);
}
}
return conditionToAdd == null ? List.of() : List.of(conditionToAdd);
}

private static void updateLastTransitionTime(List<Condition> conditions, Condition condition) {
if (condition == null) {
return;
}
Condition existingCondition = conditions.isEmpty() ? null : conditions.get(0);
condition.setLastTransitionTime(getLastTransitionTimeStamp(existingCondition, condition));
}

private static Condition getApplicationModeCondition(JobStatus jobStatus) {
return new ConditionBuilder()
.withType(CONDITION_TYPE_RUNNING)
.withStatus(jobStatus == RUNNING ? "True" : "False")
.withReason(toCamelCase(jobStatus.name()))
.withMessage("Job state " + jobStatus.name())

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should the message say Job status - as that is what we are reading?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes,

.build();
}

private static Condition getSessionModeCondition(JobManagerDeploymentStatus jmStatus) {
return new ConditionBuilder()
.withType(CONDITION_TYPE_RUNNING)
.withStatus(jmStatus == READY ? "True" : "False")
.withReason(jmStatus.getReason())
.withMessage(jmStatus.getMessage())
.build();
}

/**
* Reason in the condition object should be a CamelCase string, so need to convert JobStatus as
* all the keywords are one noun, so we only need to upper case the first letter.
*
* @return CamelCase reason as String
*/
private static String toCamelCase(String reason) {
reason = reason.toLowerCase();
return reason.substring(0, 1).toUpperCase() + reason.substring(1);

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Camel case is not just upper casing the first letter. We may need to upper case in the string as well. I suggest mapping the lower case to the appropriate camel cased reason.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right, but not as of now as per here. If we go with mapping, we end up with new method, do we required them?.

}

private static boolean isLastTransitionTimeStampUpdateRequired(
Condition existingCondition, Condition newCondition) {
return existingCondition == null
|| !existingCondition.getStatus().equals(newCondition.getStatus());
}

/**
* get the last transition time for the condition , returns the current time if there is no
* existing condition or if the condition status has changed, otherwise returns existing
* condition LastTransitionTime.
*
* @param existingCondition The current condition object, may be null.
* @param condition The new condition object to compare against the existing one.
* @return A string representing the last transition time in the format
* "yyyy-MM-dd'T'HH:mm:ss'Z'". Returns a new timestamp if the existing condition is null or
* the status has changed, otherwise returns the last transition time of the existing
* condition.
*/
private static String getLastTransitionTimeStamp(
Condition existingCondition, Condition condition) {
String lastTransitionTime;
if (existingCondition == null
|| !existingCondition.getStatus().equals(condition.getStatus())) {
lastTransitionTime =
new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss'Z'").format(new Date());
} else {
lastTransitionTime = existingCondition.getLastTransitionTime();
}
return lastTransitionTime;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.flink.kubernetes.operator.api.lifecycle.ResourceLifecycleState;
import org.apache.flink.kubernetes.operator.api.status.FlinkDeploymentStatus;
import org.apache.flink.kubernetes.operator.api.status.JobManagerDeploymentStatus;
import org.apache.flink.kubernetes.operator.api.utils.ConditionUtils;
import org.apache.flink.kubernetes.operator.config.FlinkConfigManager;
import org.apache.flink.kubernetes.operator.exception.DeploymentFailedException;
import org.apache.flink.kubernetes.operator.exception.ReconciliationException;
Expand Down Expand Up @@ -165,6 +166,8 @@ public UpdateControl<FlinkDeployment> reconcile(FlinkDeployment flinkApp, Contex
throw new ReconciliationException(e);
}

flinkApp.getStatus()
.setConditions(ConditionUtils.createConditionFromStatus(flinkApp.getStatus()));
LOG.debug("End of reconciliation");
statusRecorder.patchAndCacheStatus(flinkApp, ctx.getKubernetesClient());
return ReconciliationUtils.toUpdateControl(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@

import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
import org.apache.flink.kubernetes.operator.api.Mode;
import org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode;
import org.apache.flink.kubernetes.operator.config.Mode;
import org.apache.flink.kubernetes.operator.observer.Observer;
import org.apache.flink.kubernetes.operator.utils.EventRecorder;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,10 @@
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.autoscaler.JobAutoScaler;
import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
import org.apache.flink.kubernetes.operator.api.Mode;
import org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode;
import org.apache.flink.kubernetes.operator.api.status.FlinkDeploymentStatus;
import org.apache.flink.kubernetes.operator.autoscaler.KubernetesJobAutoScalerContext;
import org.apache.flink.kubernetes.operator.config.Mode;
import org.apache.flink.kubernetes.operator.reconciler.Reconciler;
import org.apache.flink.kubernetes.operator.utils.EventRecorder;
import org.apache.flink.kubernetes.operator.utils.StatusRecorder;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,10 +27,10 @@
import org.apache.flink.kubernetes.KubernetesClusterClientFactory;
import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
import org.apache.flink.kubernetes.operator.api.Mode;
import org.apache.flink.kubernetes.operator.api.spec.JobSpec;
import org.apache.flink.kubernetes.operator.artifact.ArtifactManager;
import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration;
import org.apache.flink.kubernetes.operator.config.Mode;
import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext;
import org.apache.flink.kubernetes.operator.kubeclient.Fabric8FlinkStandaloneKubeClient;
import org.apache.flink.kubernetes.operator.kubeclient.FlinkStandaloneKubeClient;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,7 @@ public void verifyBasicReconcileLoop(FlinkVersion flinkVersion) throws Exception
assertEquals(
org.apache.flink.api.common.JobStatus.RUNNING,
appCluster.getStatus().getJobStatus().getState());
assertEquals(7, testController.getInternalStatusUpdateCount());
assertEquals(8, testController.getInternalStatusUpdateCount());
assertFalse(updateControl.isPatchStatus());

FlinkDeploymentReconciliationStatus reconciliationStatus =
Expand Down Expand Up @@ -277,6 +277,8 @@ public void verifyFailedDeployment() throws Exception {

validatingResponseProvider.assertValidated();

validateConditionStatus(appCluster, "Reconciling");

// Validate status
assertNotNull(appCluster.getStatus().getError());

Expand Down Expand Up @@ -359,7 +361,7 @@ public void verifyUpgradeFromSavepointLegacyMode(FlinkVersion flinkVersion) thro
appCluster.getStatus().getJobManagerDeploymentStatus());
assertEquals(
"savepoint_1", appCluster.getStatus().getJobStatus().getUpgradeSavepointPath());

validateConditionStatus(appCluster, "Finished");
// Resume from last savepoint
appCluster.getSpec().getJob().setState(JobState.RUNNING);
testController.reconcile(appCluster, context);
Expand Down Expand Up @@ -618,6 +620,7 @@ public void verifyReconcileWithAChangedOperatorModeToSession() throws Exception
// jobStatus has not been set at this time
assertEquals(org.apache.flink.api.common.JobStatus.RECONCILING, jobStatus.getState());

validateConditionStatus(appCluster, "Reconciling");
// Switches operator mode to SESSION
appCluster.getSpec().setJob(null);
// Validation fails and JobObserver should still be used
Expand All @@ -639,6 +642,7 @@ public void verifyReconcileWithAChangedOperatorModeToSession() throws Exception
assertEquals(expectedJobStatus.getJobId().toHexString(), jobStatus.getJobId());
assertEquals(expectedJobStatus.getJobName(), jobStatus.getJobName());
assertEquals(expectedJobStatus.getJobState(), jobStatus.getState());
validateConditionStatus(appCluster, "Running");
}

@Test
Expand All @@ -652,6 +656,7 @@ public void verifyReconcileWithAChangedOperatorModeToApplication() throws Except
assertEquals(
JobManagerDeploymentStatus.DEPLOYING,
appCluster.getStatus().getJobManagerDeploymentStatus());
validateConditionStatus(appCluster, "JobManagerIsDeploying");

updateControl = testController.reconcile(appCluster, context);
JobStatus jobStatus = appCluster.getStatus().getJobStatus();
Expand All @@ -662,6 +667,8 @@ public void verifyReconcileWithAChangedOperatorModeToApplication() throws Except
// jobStatus has not been set at this time
assertNull(jobStatus.getState());

validateConditionStatus(appCluster, "DeployedNotReady");

// Switches operator mode to APPLICATION
appCluster.getSpec().setJob(TestUtils.buildSessionJob().getSpec().getJob());
// Validation fails and JobObserver should still be used
Expand All @@ -676,6 +683,8 @@ public void verifyReconcileWithAChangedOperatorModeToApplication() throws Except
.getError()
.contains("Cannot switch from session to job cluster"));
assertNull(ReconciliationUtils.getDeployedSpec(appCluster).getJob());

validateConditionStatus(appCluster, "JobManagerReady");
}

private void testUpgradeNotReadyCluster(FlinkDeployment appCluster) throws Exception {
Expand Down Expand Up @@ -1169,7 +1178,7 @@ private void verifyReconcileNormalLifecycle(FlinkDeployment appCluster) throws E
assertEquals(
org.apache.flink.api.common.JobStatus.RUNNING,
appCluster.getStatus().getJobStatus().getState());
assertEquals(6, testController.getInternalStatusUpdateCount());
assertEquals(7, testController.getInternalStatusUpdateCount());
assertFalse(updateControl.isPatchStatus());
assertEquals(
Optional.of(
Expand All @@ -1184,7 +1193,7 @@ private void verifyReconcileNormalLifecycle(FlinkDeployment appCluster) throws E
assertEquals(
org.apache.flink.api.common.JobStatus.RUNNING,
appCluster.getStatus().getJobStatus().getState());
assertEquals(6, testController.getInternalStatusUpdateCount());
assertEquals(7, testController.getInternalStatusUpdateCount());
assertFalse(updateControl.isPatchStatus());
assertEquals(
Optional.of(
Expand Down Expand Up @@ -1254,4 +1263,12 @@ private String getIngressHost(HasMetadata ingress) {
return ingressRuleV1beta1.getHost();
}
}

private void validateConditionStatus(FlinkDeployment appCluster, String reason) {
assertThat(appCluster.getStatus().getConditions()).isNotNull();
assertThat(appCluster.getStatus().getConditions())
.hasSize(1)
.extracting("reason")
.contains(reason);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,11 +24,11 @@
import org.apache.flink.kubernetes.operator.TestUtils;
import org.apache.flink.kubernetes.operator.api.AbstractFlinkResource;
import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
import org.apache.flink.kubernetes.operator.api.Mode;
import org.apache.flink.kubernetes.operator.api.spec.JobSpec;
import org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode;
import org.apache.flink.kubernetes.operator.artifact.ArtifactManager;
import org.apache.flink.kubernetes.operator.config.FlinkConfigManager;
import org.apache.flink.kubernetes.operator.config.Mode;
import org.apache.flink.kubernetes.operator.utils.StandaloneKubernetesUtils;
import org.apache.flink.util.concurrent.Executors;

Expand Down
Loading