diff --git a/docs/content/docs/custom-resource/job-management.md b/docs/content/docs/custom-resource/job-management.md index 231f92a873..cfae67b0d1 100644 --- a/docs/content/docs/custom-resource/job-management.md +++ b/docs/content/docs/custom-resource/job-management.md @@ -84,13 +84,13 @@ Supported values: `stateless`, `savepoint`, `last-state` The `upgradeMode` setting controls both the stop and restore mechanisms as detailed in the following table: -| | Stateless | Last State | Savepoint | -|------------------------|-------------------------|--------------------------------------------|----------------------------------------| -| Config Requirement | None | Checkpointing & HA Enabled | Checkpoint/Savepoint directory defined | -| Job Status Requirement | None | HA metadata available | Job Running* | -| Suspend Mechanism | Cancel / Delete | Delete Flink deployment (keep HA metadata) | Cancel with savepoint | -| Restore Mechanism | Deploy from empty state | Recover last state using HA metadata | Restore From savepoint | -| Production Use | Not recommended | Recommended | Recommended | +| | Stateless | Last State | Savepoint | +|------------------------|-----------------|------------------------------------|----------------------------------------| +| Config Requirement | None | Checkpointing Enabled | Checkpoint/Savepoint directory defined | +| Job Status Requirement | None | Job or HA metadata accessible | Job Running* | +| Suspend Mechanism | Cancel / Delete | Cancel / Delete (keep HA metadata) | Cancel with savepoint | +| Restore Mechanism | Empty state | Use HA metadata or last cp/sp | Restore From savepoint | +| Production Use | Not recommended | Recommended | Recommended | *\* When HA is enabled the `savepoint` upgrade mode may fall back to the `last-state` behaviour in cases where the job is in an unhealthy state.* @@ -149,10 +149,6 @@ spec: state: running ``` -{{< hint warning >}} -Last state upgrade mode is currently only supported for `FlinkDeployments`. -{{< /hint >}} - ### Application restarts without spec change There are cases when users would like to restart the Flink deployments to deal with some transient problem. diff --git a/docs/content/docs/custom-resource/overview.md b/docs/content/docs/custom-resource/overview.md index 6c3aadc580..61beb10af1 100644 --- a/docs/content/docs/custom-resource/overview.md +++ b/docs/content/docs/custom-resource/overview.md @@ -215,10 +215,6 @@ COPY flink-hadoop-fs-1.19-SNAPSHOT.jar $FLINK_PLUGINS_DIR/hadoop-fs/ Alternatively, if you use helm to install flink-kubernetes-operator, it allows you to specify a postStart hook to download the required plugins. -### Limitations - -- Last-state upgradeMode is currently not supported for FlinkSessionJobs - ## Further information - [Snapshots]({{< ref "docs/custom-resource/snapshots" >}}) diff --git a/docs/content/docs/custom-resource/reference.md b/docs/content/docs/custom-resource/reference.md index 12e1d88c7d..c336e1c407 100644 --- a/docs/content/docs/custom-resource/reference.md +++ b/docs/content/docs/custom-resource/reference.md @@ -452,7 +452,7 @@ This serves as a full reference for FlinkDeployment and FlinkSessionJob custom r | Parameter | Type | Docs | | ----------| ---- | ---- | -| lastSavepoint | org.apache.flink.kubernetes.operator.api.status.Savepoint | Last completed savepoint by the operator for manual and periodic snapshots. Only used if FlinkStateSnapshot resources are disabled. | +| lastSavepoint | org.apache.flink.kubernetes.operator.api.status.Savepoint | Last completed savepoint by the operator. | | triggerId | java.lang.String | Trigger id of a pending savepoint operation. | | triggerTimestamp | java.lang.Long | Trigger timestamp of a pending savepoint operation. | | triggerType | org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType | Savepoint trigger mechanism. | diff --git a/docs/layouts/shortcodes/generated/dynamic_section.html b/docs/layouts/shortcodes/generated/dynamic_section.html index d60f078872..54c1eb6946 100644 --- a/docs/layouts/shortcodes/generated/dynamic_section.html +++ b/docs/layouts/shortcodes/generated/dynamic_section.html @@ -122,6 +122,12 @@ Boolean Enables last-state fallback for savepoint upgrade mode. When the job is not running thus savepoint cannot be triggered but HA metadata is available for last state restore the operator can initiate the upgrade process when the flag is enabled. + +
kubernetes.operator.job.upgrade.last-state.job-cancel.enabled
+ false + Boolean + Cancel jobs during last-state upgrade. This config is ignored for session jobs where cancel is the only mechanism to perform this type of upgrade. +
kubernetes.operator.job.upgrade.last-state.max.allowed.checkpoint.age
(none) diff --git a/docs/layouts/shortcodes/generated/kubernetes_operator_config_configuration.html b/docs/layouts/shortcodes/generated/kubernetes_operator_config_configuration.html index 164b188955..164c9bbd3d 100644 --- a/docs/layouts/shortcodes/generated/kubernetes_operator_config_configuration.html +++ b/docs/layouts/shortcodes/generated/kubernetes_operator_config_configuration.html @@ -212,6 +212,12 @@ Boolean Enables last-state fallback for savepoint upgrade mode. When the job is not running thus savepoint cannot be triggered but HA metadata is available for last state restore the operator can initiate the upgrade process when the flag is enabled. + +
kubernetes.operator.job.upgrade.last-state.job-cancel.enabled
+ false + Boolean + Cancel jobs during last-state upgrade. This config is ignored for session jobs where cancel is the only mechanism to perform this type of upgrade. +
kubernetes.operator.job.upgrade.last-state.max.allowed.checkpoint.age
(none) diff --git a/e2e-tests/test_sessionjob_operations.sh b/e2e-tests/test_sessionjob_operations.sh index b1c88fc276..c2d8fdcc86 100755 --- a/e2e-tests/test_sessionjob_operations.sh +++ b/e2e-tests/test_sessionjob_operations.sh @@ -54,6 +54,7 @@ if [ "$location" == "" ];then exit 1 fi +echo "Starting sessionjob savepoint upgrade test" # Testing savepoint mode upgrade # Update the FlinkSessionJob and trigger the savepoint upgrade kubectl patch sessionjob ${SESSION_JOB_NAME} --type merge --patch '{"spec":{"job": {"parallelism": 1 } } }' @@ -67,6 +68,24 @@ assert_available_slots 1 $CLUSTER_ID echo "Successfully run the sessionjob savepoint upgrade test" +flink_version=$(kubectl get $SESSION_CLUSTER_IDENTIFIER -o yaml | yq '.spec.flinkVersion') + +if [ "$flink_version" != "v1_16" ]; then + echo "Starting sessionjob last-state upgrade test" + # Testing last-state mode upgrade + # Update the FlinkSessionJob and trigger the last-state upgrade + kubectl patch sessionjob ${SESSION_JOB_NAME} --type merge --patch '{"spec":{"job": {"parallelism": 2, "upgradeMode": "last-state" } } }' + + # Check the job was restarted with the new parallelism + wait_for_status $SESSION_JOB_IDENTIFIER '.status.jobStatus.state' CANCELLING ${TIMEOUT} || exit 1 + wait_for_status $SESSION_JOB_IDENTIFIER '.status.jobStatus.state' RUNNING ${TIMEOUT} || exit 1 + assert_available_slots 0 $CLUSTER_ID + + echo "Successfully run the sessionjob last-state upgrade test" +else + echo "Skipping last-state test for flink version 1.16" +fi + # Test Operator restart echo "Delete session job " + $SESSION_JOB_NAME kubectl delete flinksessionjob $SESSION_JOB_NAME diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/docs/CrdReferenceDoclet.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/docs/CrdReferenceDoclet.java index 787081120c..f08a5b253a 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/docs/CrdReferenceDoclet.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/docs/CrdReferenceDoclet.java @@ -219,6 +219,9 @@ public Void scan(Element e, Integer depth) { } break; case FIELD: + if (e.getModifiers().contains(Modifier.STATIC)) { + return null; + } out.println( "| " + getNameOrJsonPropValue(e) diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/CommonStatus.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/CommonStatus.java index 6ce5a16201..0e9635cbd0 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/CommonStatus.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/CommonStatus.java @@ -18,12 +18,10 @@ package org.apache.flink.kubernetes.operator.api.status; import org.apache.flink.annotation.Experimental; -import org.apache.flink.annotation.Internal; import org.apache.flink.kubernetes.operator.api.lifecycle.ResourceLifecycleState; import org.apache.flink.kubernetes.operator.api.spec.AbstractFlinkSpec; import org.apache.flink.kubernetes.operator.api.spec.JobState; -import com.fasterxml.jackson.annotation.JsonIgnore; import io.fabric8.crd.generator.annotation.PrinterColumn; import lombok.AllArgsConstructor; import lombok.Data; @@ -98,10 +96,4 @@ public ResourceLifecycleState getLifecycleState() { return ResourceLifecycleState.DEPLOYED; } - - /** - * Internal flag to signal that due to some condition we need to schedule a new reconciliation - * loop immediately. For example autoscaler overrides have changed and we need to apply them. - */ - @JsonIgnore @Internal private boolean immediateReconciliationNeeded = false; } diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/ReconciliationStatus.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/ReconciliationStatus.java index 35363b8a74..668b6a2337 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/ReconciliationStatus.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/ReconciliationStatus.java @@ -20,7 +20,6 @@ import org.apache.flink.annotation.Experimental; import org.apache.flink.kubernetes.operator.api.AbstractFlinkResource; import org.apache.flink.kubernetes.operator.api.spec.AbstractFlinkSpec; -import org.apache.flink.kubernetes.operator.api.spec.JobState; import org.apache.flink.kubernetes.operator.api.utils.SpecUtils; import org.apache.flink.kubernetes.operator.api.utils.SpecWithMeta; @@ -100,22 +99,4 @@ public boolean isLastReconciledSpecStable() { public boolean isBeforeFirstDeployment() { return lastReconciledSpec == null; } - - /** - * This method is only here for backward compatibility reasons. The current version of the - * operator does not leave the resources in UPGRADING state during in-place scaling therefore - * this method will always return false. - * - * @return True if in-place scaling is in progress. - */ - @JsonIgnore - @Deprecated - public boolean scalingInProgress() { - if (isBeforeFirstDeployment() || state != ReconciliationState.UPGRADING) { - return false; - } - var job = deserializeLastReconciledSpec().getJob(); - // For regular full upgrades the jobstate is suspended in UPGRADING state - return job != null && job.getState() == JobState.RUNNING; - } } diff --git a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/SavepointInfo.java b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/SavepointInfo.java index 331bce7838..bdbe8de07d 100644 --- a/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/SavepointInfo.java +++ b/flink-kubernetes-operator-api/src/main/java/org/apache/flink/kubernetes/operator/api/status/SavepointInfo.java @@ -24,6 +24,8 @@ import lombok.Builder; import lombok.Data; import lombok.NoArgsConstructor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; @@ -40,6 +42,9 @@ public class SavepointInfo implements SnapshotInfo { * Last completed savepoint by the operator for manual and periodic snapshots. Only used if * FlinkStateSnapshot resources are disabled. */ + private static final Logger LOG = LoggerFactory.getLogger(SavepointInfo.class); + + /** Last completed savepoint by the operator. */ private Savepoint lastSavepoint; /** Trigger id of a pending savepoint operation. */ @@ -82,7 +87,11 @@ public void resetTrigger() { * @param savepoint Savepoint to be added. */ public void updateLastSavepoint(Savepoint savepoint) { - if (lastSavepoint == null || !lastSavepoint.getLocation().equals(savepoint.getLocation())) { + if (savepoint == null) { + lastSavepoint = null; + } else if (lastSavepoint == null + || !lastSavepoint.getLocation().equals(savepoint.getLocation())) { + LOG.debug("Updating last savepoint to {}", savepoint); lastSavepoint = savepoint; savepointHistory.add(savepoint); if (savepoint.getTriggerType() == SnapshotTriggerType.PERIODIC) { diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilder.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilder.java index 63f65003e2..905ae7d0e0 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilder.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilder.java @@ -148,33 +148,41 @@ protected FlinkConfigBuilder applyFlinkConfiguration() { REST_SERVICE_EXPOSED_TYPE, KubernetesConfigOptions.ServiceExposedType.ClusterIP); // Set 'web.cancel.enable' to false to avoid users accidentally cancelling jobs. setDefaultConf(CANCEL_ENABLE, false); + effectiveConfig.set(FLINK_VERSION, spec.getFlinkVersion()); + return this; + } - if (spec.getJob() != null) { - // Set 'pipeline.name' to resource name by default for application deployments. - setDefaultConf(PipelineOptions.NAME, clusterId); - - // With last-state upgrade mode, set the default value of - // 'execution.checkpointing.interval' - // to 5 minutes when HA is enabled. - if (spec.getJob().getUpgradeMode() == UpgradeMode.LAST_STATE) { - setDefaultConf( - ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL, - DEFAULT_CHECKPOINTING_INTERVAL); - } - - // We need to keep the application clusters around for proper operator behaviour - effectiveConfig.set(SHUTDOWN_ON_APPLICATION_FINISH, false); - if (HighAvailabilityMode.isHighAvailabilityModeActivated(effectiveConfig)) { - setDefaultConf(SUBMIT_FAILED_JOB_ON_APPLICATION_ERROR, true); - } + protected static void applyJobConfig(String name, Configuration conf, JobSpec jobSpec) { + // Set 'pipeline.name' to resource name by default for application deployments. + setDefaultConf(conf, PipelineOptions.NAME, name); + // With last-state upgrade mode, set the default value of + // 'execution.checkpointing.interval' + // to 5 minutes when HA is enabled. + if (jobSpec.getUpgradeMode() == UpgradeMode.LAST_STATE) { setDefaultConf( - ExecutionCheckpointingOptions.EXTERNALIZED_CHECKPOINT, - CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); + conf, + ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL, + DEFAULT_CHECKPOINTING_INTERVAL); } + setDefaultConf( + conf, + ExecutionCheckpointingOptions.EXTERNALIZED_CHECKPOINT, + CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); - effectiveConfig.set(FLINK_VERSION, spec.getFlinkVersion()); - return this; + if (jobSpec.getAllowNonRestoredState() != null) { + conf.set( + SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE, + jobSpec.getAllowNonRestoredState()); + } + + if (jobSpec.getEntryClass() != null) { + conf.set(ApplicationConfiguration.APPLICATION_MAIN_CLASS, jobSpec.getEntryClass()); + } + + if (jobSpec.getArgs() != null) { + conf.set(ApplicationConfiguration.APPLICATION_ARGS, Arrays.asList(jobSpec.getArgs())); + } } protected FlinkConfigBuilder applyLogConfiguration() throws IOException { @@ -304,29 +312,18 @@ protected FlinkConfigBuilder applyJobOrSessionSpec() throws URISyntaxException { DeploymentOptions.TARGET, KubernetesDeploymentTarget.APPLICATION.getName()); if (jobSpec.getJarURI() != null) { - final URI uri = new URI(jobSpec.getJarURI()); effectiveConfig.set( - PipelineOptions.JARS, Collections.singletonList(uri.toString())); + PipelineOptions.JARS, + Collections.singletonList(new URI(jobSpec.getJarURI()).toString())); } - effectiveConfig.set(CoreOptions.DEFAULT_PARALLELISM, getParallelism()); - if (jobSpec.getAllowNonRestoredState() != null) { - effectiveConfig.set( - SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE, - jobSpec.getAllowNonRestoredState()); - } - - if (jobSpec.getEntryClass() != null) { - effectiveConfig.set( - ApplicationConfiguration.APPLICATION_MAIN_CLASS, jobSpec.getEntryClass()); - } + // We need to keep the application clusters around for proper operator behaviour + effectiveConfig.set(SHUTDOWN_ON_APPLICATION_FINISH, false); + setDefaultConf(SUBMIT_FAILED_JOB_ON_APPLICATION_ERROR, true); - if (jobSpec.getArgs() != null) { - effectiveConfig.set( - ApplicationConfiguration.APPLICATION_ARGS, - Arrays.asList(jobSpec.getArgs())); - } + // Generic shared job config logic + applyJobConfig(clusterId, effectiveConfig, jobSpec); } else { effectiveConfig.set( DeploymentOptions.TARGET, KubernetesDeploymentTarget.SESSION.getName()); @@ -423,8 +420,12 @@ public static Configuration buildFrom( } private void setDefaultConf(ConfigOption option, T value) { - if (!effectiveConfig.contains(option)) { - effectiveConfig.set(option, value); + setDefaultConf(effectiveConfig, option, value); + } + + private static void setDefaultConf(Configuration conf, ConfigOption option, T value) { + if (!conf.contains(option)) { + conf.set(option, value); } } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java index 3fa0691eaa..6b5f16a34d 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java @@ -59,6 +59,7 @@ import java.util.function.Consumer; import static org.apache.flink.configuration.CheckpointingOptions.SAVEPOINT_DIRECTORY; +import static org.apache.flink.kubernetes.operator.config.FlinkConfigBuilder.applyJobConfig; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.K8S_OP_CONF_PREFIX; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.NAMESPACE_CONF_PREFIX; import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_DYNAMIC_CONFIG_CHECK_INTERVAL; @@ -292,7 +293,7 @@ private void applyConfigsFromCurrentSpec( * @return Session job config */ public Configuration getSessionJobConfig( - FlinkDeployment deployment, FlinkSessionJobSpec sessionJobSpec) { + String name, FlinkDeployment deployment, FlinkSessionJobSpec sessionJobSpec) { Configuration sessionJobConfig = getObserveConfig(deployment); // merge session job specific config @@ -300,6 +301,7 @@ public Configuration getSessionJobConfig( if (sessionJobFlinkConfiguration != null) { sessionJobFlinkConfiguration.forEach(sessionJobConfig::setString); } + applyJobConfig(name, sessionJobConfig, sessionJobSpec.getJob()); return sessionJobConfig; } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/KubernetesOperatorConfigOptions.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/KubernetesOperatorConfigOptions.java index 5da5d3f855..76d3c4d7c6 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/KubernetesOperatorConfigOptions.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/KubernetesOperatorConfigOptions.java @@ -465,6 +465,14 @@ public static String operatorConfigKey(String key) { .withDescription( "Max allowed checkpoint age for initiating last-state upgrades on running jobs. If a checkpoint is not available within the desired age (and nothing in progress) a savepoint will be triggered."); + @Documentation.Section(SECTION_DYNAMIC) + public static final ConfigOption OPERATOR_JOB_UPGRADE_LAST_STATE_CANCEL_JOB = + operatorConfig("job.upgrade.last-state.job-cancel.enabled") + .booleanType() + .defaultValue(false) + .withDescription( + "Cancel jobs during last-state upgrade. This config is ignored for session jobs where cancel is the only mechanism to perform this type of upgrade."); + @Documentation.Section(SECTION_ADVANCED) public static final ConfigOption OPERATOR_HEALTH_PROBE_ENABLED = operatorConfig("health.probe.enabled") diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java index aa9381fcb5..dbaa6d4866 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java @@ -24,7 +24,7 @@ import org.apache.flink.kubernetes.operator.api.status.JobManagerDeploymentStatus; import org.apache.flink.kubernetes.operator.exception.DeploymentFailedException; import org.apache.flink.kubernetes.operator.exception.ReconciliationException; -import org.apache.flink.kubernetes.operator.exception.RecoveryFailureException; +import org.apache.flink.kubernetes.operator.exception.UpgradeFailureException; import org.apache.flink.kubernetes.operator.health.CanaryResourceManager; import org.apache.flink.kubernetes.operator.observer.deployment.FlinkDeploymentObserverFactory; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; @@ -96,26 +96,29 @@ public DeleteControl cleanup(FlinkDeployment flinkApp, Context josdkContext) { if (canaryResourceManager.handleCanaryResourceDeletion(flinkApp)) { return DeleteControl.defaultDelete(); } - - String msg = "Cleaning up " + FlinkDeployment.class.getSimpleName(); - LOG.info(msg); eventRecorder.triggerEvent( flinkApp, EventRecorder.Type.Normal, EventRecorder.Reason.Cleanup, EventRecorder.Component.Operator, - msg, + "Cleaning up FlinkDeployment", josdkContext.getClient()); statusRecorder.updateStatusFromCache(flinkApp); var ctx = ctxFactory.getResourceContext(flinkApp, josdkContext); try { observerFactory.getOrCreate(flinkApp).observe(ctx); } catch (Exception err) { - // ignore during cleanup + LOG.error("Error while observing for cleanup", err); + } + + var deleteControl = reconcilerFactory.getOrCreate(flinkApp).cleanup(ctx); + if (deleteControl.isRemoveFinalizer()) { + statusRecorder.removeCachedStatus(flinkApp); + ctxFactory.cleanup(flinkApp); + } else { + statusRecorder.patchAndCacheStatus(flinkApp, ctx.getKubernetesClient()); } - statusRecorder.removeCachedStatus(flinkApp); - ctxFactory.cleanup(flinkApp); - return reconcilerFactory.getOrCreate(flinkApp).cleanup(ctx); + return deleteControl; } @Override @@ -147,8 +150,8 @@ public UpdateControl reconcile(FlinkDeployment flinkApp, Contex } statusRecorder.patchAndCacheStatus(flinkApp, ctx.getKubernetesClient()); reconcilerFactory.getOrCreate(flinkApp).reconcile(ctx); - } catch (RecoveryFailureException rfe) { - handleRecoveryFailed(ctx, rfe); + } catch (UpgradeFailureException ufe) { + handleUpgradeFailure(ctx, ufe); } catch (DeploymentFailedException dfe) { handleDeploymentFailed(ctx, dfe); } catch (Exception e) { @@ -184,16 +187,16 @@ private void handleDeploymentFailed( ctx.getKubernetesClient()); } - private void handleRecoveryFailed( - FlinkResourceContext ctx, RecoveryFailureException rfe) { - LOG.error("Flink recovery failed", rfe); + private void handleUpgradeFailure( + FlinkResourceContext ctx, UpgradeFailureException ufe) { + LOG.error("Error while upgrading Flink Deployment", ufe); var flinkApp = ctx.getResource(); - ReconciliationUtils.updateForReconciliationError(ctx, rfe); + ReconciliationUtils.updateForReconciliationError(ctx, ufe); eventRecorder.triggerEvent( flinkApp, EventRecorder.Type.Warning, - rfe.getReason(), - rfe.getMessage(), + ufe.getReason(), + ufe.getMessage(), EventRecorder.Component.JobManagerDeployment, ctx.getKubernetesClient()); } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobContext.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobContext.java index c2b7153b07..5d18f56f29 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobContext.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobContext.java @@ -53,7 +53,10 @@ public Configuration getDeployConfig(AbstractFlinkSpec spec) { var session = getJosdkContext().getSecondaryResource(FlinkDeployment.class); if (sessionClusterReady(session)) { - return configManager.getSessionJobConfig(session.get(), (FlinkSessionJobSpec) spec); + return configManager.getSessionJobConfig( + getResource().getMetadata().getName(), + session.get(), + (FlinkSessionJobSpec) spec); } return null; } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobController.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobController.java index 3c08f00779..a88a2963db 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobController.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobController.java @@ -138,20 +138,28 @@ public DeleteControl cleanup(FlinkSessionJob sessionJob, Context josdkContext) { if (canaryResourceManager.handleCanaryResourceDeletion(sessionJob)) { return DeleteControl.defaultDelete(); } - - String msg = "Cleaning up " + FlinkSessionJob.class.getSimpleName(); - - LOG.info(msg); eventRecorder.triggerEvent( sessionJob, EventRecorder.Type.Normal, EventRecorder.Reason.Cleanup, EventRecorder.Component.Operator, - msg, + "Cleaning up FlinkSessionJob", josdkContext.getClient()); - statusRecorder.removeCachedStatus(sessionJob); var ctx = ctxFactory.getResourceContext(sessionJob, josdkContext); - return reconciler.cleanup(ctx); + try { + observer.observe(ctx); + } catch (Exception err) { + LOG.error("Error while observing for cleanup", err); + } + + var deleteControl = reconciler.cleanup(ctx); + if (deleteControl.isRemoveFinalizer()) { + ctxFactory.cleanup(sessionJob); + statusRecorder.removeCachedStatus(sessionJob); + } else { + statusRecorder.patchAndCacheStatus(sessionJob, ctx.getKubernetesClient()); + } + return deleteControl; } @Override diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/exception/RecoveryFailureException.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/exception/UpgradeFailureException.java similarity index 78% rename from flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/exception/RecoveryFailureException.java rename to flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/exception/UpgradeFailureException.java index 1b0be67ba8..e0becce559 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/exception/RecoveryFailureException.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/exception/UpgradeFailureException.java @@ -18,14 +18,19 @@ package org.apache.flink.kubernetes.operator.exception; /** Exception to signal non-terminal deployment failure. */ -public class RecoveryFailureException extends RuntimeException { +public class UpgradeFailureException extends RuntimeException { private final String reason; - public RecoveryFailureException(String message, String reason) { + public UpgradeFailureException(String message, String reason) { super(message); this.reason = reason; } + public UpgradeFailureException(String message, String reason, Throwable throwable) { + super(message, throwable); + this.reason = reason; + } + public String getReason() { return reason; } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/listener/AuditUtils.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/listener/AuditUtils.java index 33cd5459fd..4433652620 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/listener/AuditUtils.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/listener/AuditUtils.java @@ -44,6 +44,10 @@ public static void logContext(FlinkResourceListener.FlinkStateSnapshotEventConte public static , S extends CommonStatus> void logContext(FlinkResourceListener.StatusUpdateContext ctx) { + if (ctx.getPreviousStatus().getLifecycleState() == ctx.getNewStatus().getLifecycleState()) { + // Unchanged state, nothing to log + return; + } LOG.info(format(ctx.getNewStatus())); } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/AbstractFlinkResourceObserver.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/AbstractFlinkResourceObserver.java index c8360db817..9fd9f59687 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/AbstractFlinkResourceObserver.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/AbstractFlinkResourceObserver.java @@ -61,6 +61,7 @@ public final void observe(FlinkResourceContext ctx) { */ protected boolean isResourceReadyToBeObserved(FlinkResourceContext ctx) { var resource = ctx.getResource(); + var status = resource.getStatus(); var reconciliationStatus = resource.getStatus().getReconciliationStatus(); if (reconciliationStatus.isBeforeFirstDeployment()) { @@ -73,18 +74,18 @@ protected boolean isResourceReadyToBeObserved(FlinkResourceContext ctx) { return false; } + if (ReconciliationUtils.isJobCancelling(status)) { + return true; + } + // We are in the middle or possibly right after an upgrade if (reconciliationStatus.getState() == ReconciliationState.UPGRADING) { // We must check if the upgrade went through without the status upgrade for some reason - - if (reconciliationStatus.scalingInProgress()) { - // Keep this for backward compatibility - reconciliationStatus.setState(ReconciliationState.DEPLOYED); - } else if (checkIfAlreadyUpgraded(ctx)) { + if (checkIfAlreadyUpgraded(ctx)) { ReconciliationUtils.updateStatusForAlreadyUpgraded(resource); } else { ReconciliationUtils.clearLastReconciledSpecIfFirstDeploy(resource); - logger.debug("Skipping observe before resource is deployed during upgrade"); + logger.info("Skipping observe before resource is deployed during upgrade"); return false; } } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserver.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserver.java index 08f74f798a..e299531fc0 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserver.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserver.java @@ -18,14 +18,16 @@ package org.apache.flink.kubernetes.operator.observer; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; import org.apache.flink.kubernetes.operator.api.AbstractFlinkResource; +import org.apache.flink.kubernetes.operator.api.FlinkSessionJob; import org.apache.flink.kubernetes.operator.api.spec.JobState; -import org.apache.flink.kubernetes.operator.api.status.JobStatus; +import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; +import org.apache.flink.kubernetes.operator.api.status.ReconciliationState; import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.utils.EventRecorder; import org.apache.flink.runtime.client.JobStatusMessage; -import org.apache.flink.runtime.rest.NotFoundException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -98,19 +100,30 @@ public boolean observe(FlinkResourceContext ctx) { * @param ctx The Flink resource context. */ protected void onTargetJobNotFound(FlinkResourceContext ctx) { - ctx.getResource() - .getStatus() - .getJobStatus() - .setState(org.apache.flink.api.common.JobStatus.RECONCILING.name()); - ReconciliationUtils.updateForReconciliationError( - ctx, new NotFoundException(JOB_NOT_FOUND_ERR)); + var resource = ctx.getResource(); + var jobStatus = resource.getStatus().getJobStatus(); + eventRecorder.triggerEvent( - ctx.getResource(), + resource, EventRecorder.Type.Warning, EventRecorder.Reason.Missing, EventRecorder.Component.Job, JOB_NOT_FOUND_ERR, ctx.getKubernetesClient()); + + if (resource instanceof FlinkSessionJob + && !ReconciliationUtils.isJobInTerminalState(resource.getStatus()) + && resource.getSpec().getJob().getUpgradeMode() == UpgradeMode.STATELESS) { + // We also mark jobs that were previously not terminated as suspended if + // stateless upgrade mode is used. In these cases we want to simply restart the job. + markSuspended(resource); + } else { + // We could not suspend the job as it was lost during a stateless upgrade, cancel + // upgrading state and retry the upgrade (if possible) + resource.getStatus().getReconciliationStatus().setState(ReconciliationState.DEPLOYED); + } + jobStatus.setState(JobStatus.RECONCILING.name()); + resource.getStatus().setError(JOB_NOT_FOUND_ERR); } /** @@ -120,9 +133,11 @@ protected void onTargetJobNotFound(FlinkResourceContext ctx) { * @param jobStatus JobStatus object. * @param previousJobStatus Last observed job state. */ - private void ifRunningMoveToReconciling(JobStatus jobStatus, String previousJobStatus) { - if (org.apache.flink.api.common.JobStatus.RUNNING.name().equals(previousJobStatus)) { - jobStatus.setState(org.apache.flink.api.common.JobStatus.RECONCILING.name()); + private void ifRunningMoveToReconciling( + org.apache.flink.kubernetes.operator.api.status.JobStatus jobStatus, + String previousJobStatus) { + if (JobStatus.RUNNING.name().equals(previousJobStatus)) { + jobStatus.setState(JobStatus.RECONCILING.name()); } } @@ -143,6 +158,7 @@ private void updateJobStatus(FlinkResourceContext ctx, JobStatusMessage clust var resource = ctx.getResource(); var jobStatus = resource.getStatus().getJobStatus(); var previousJobStatus = jobStatus.getState(); + var currentJobStatus = clusterJobStatus.getJobState(); jobStatus.setState(clusterJobStatus.getJobState().name()); jobStatus.setJobName(clusterJobStatus.getJobName()); @@ -159,6 +175,13 @@ private void updateJobStatus(FlinkResourceContext ctx, JobStatusMessage clust "Job status changed from %s to %s", previousJobStatus, jobStatus.getState()); + if (JobStatus.CANCELED == currentJobStatus + || (currentJobStatus.isGloballyTerminalState() + && JobStatus.CANCELLING.name().equals(previousJobStatus))) { + // The job was cancelled + markSuspended(resource); + } + setErrorIfPresent(ctx, clusterJobStatus); eventRecorder.triggerEvent( resource, @@ -170,8 +193,18 @@ private void updateJobStatus(FlinkResourceContext ctx, JobStatusMessage clust } } + private static void markSuspended(AbstractFlinkResource resource) { + LOG.info("Marking suspended"); + ReconciliationUtils.updateLastReconciledSpec( + resource, + (s, m) -> { + s.getJob().setState(JobState.SUSPENDED); + m.setFirstDeployment(false); + }); + } + private void setErrorIfPresent(FlinkResourceContext ctx, JobStatusMessage clusterJobStatus) { - if (clusterJobStatus.getJobState() == org.apache.flink.api.common.JobStatus.FAILED) { + if (clusterJobStatus.getJobState() == JobStatus.FAILED) { try { var result = ctx.getFlinkService() diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/SnapshotObserver.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/SnapshotObserver.java index 59272af6ff..4255b09a1b 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/SnapshotObserver.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/SnapshotObserver.java @@ -26,16 +26,13 @@ import org.apache.flink.kubernetes.operator.api.status.Checkpoint; import org.apache.flink.kubernetes.operator.api.status.CheckpointInfo; import org.apache.flink.kubernetes.operator.api.status.CommonStatus; -import org.apache.flink.kubernetes.operator.api.status.JobStatus; import org.apache.flink.kubernetes.operator.api.status.Savepoint; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext; -import org.apache.flink.kubernetes.operator.exception.ReconciliationException; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.reconciler.SnapshotType; -import org.apache.flink.kubernetes.operator.service.FlinkService; import org.apache.flink.kubernetes.operator.utils.ConfigOptionUtils; import org.apache.flink.kubernetes.operator.utils.EventRecorder; import org.apache.flink.kubernetes.operator.utils.FlinkStateSnapshotUtils; @@ -81,7 +78,7 @@ public SnapshotObserver(EventRecorder eventRecorder) { } public void observeSavepointStatus(FlinkResourceContext ctx) { - + LOG.debug("Observing savepoint status"); var resource = ctx.getResource(); var jobStatus = resource.getStatus().getJobStatus(); var jobId = jobStatus.getJobId(); @@ -93,8 +90,7 @@ public void observeSavepointStatus(FlinkResourceContext ctx) { // If job is in globally terminal state, observe last savepoint if (ReconciliationUtils.isJobInTerminalState(resource.getStatus())) { - observeLatestCheckpoint( - ctx.getFlinkService(), jobStatus, jobId, ctx.getObserveConfig()); + observeLatestCheckpoint(ctx, jobId); } cleanupSavepointHistory(ctx); @@ -119,7 +115,7 @@ private void observeTriggeredSavepoint(FlinkResourceContext ctx, String jobI var savepointInfo = resource.getStatus().getJobStatus().getSavepointInfo(); - LOG.info("Observing savepoint status."); + LOG.debug("Observing in-progress savepoint"); var savepointFetchResult = ctx.getFlinkService() .fetchSavepointInfo( @@ -444,19 +440,22 @@ private long getMaxCountForSnapshotType( } } - private void observeLatestCheckpoint( - FlinkService flinkService, - JobStatus jobStatus, - String jobID, - Configuration observeConfig) { - try { - flinkService - .getLastCheckpoint(JobID.fromHexString(jobID), observeConfig) - .ifPresent( - snapshot -> jobStatus.setUpgradeSavepointPath(snapshot.getLocation())); - } catch (Exception e) { - LOG.error("Could not observe latest checkpoint information.", e); - throw new ReconciliationException(e); - } + private void observeLatestCheckpoint(FlinkResourceContext ctx, String jobId) { + + var status = ctx.getResource().getStatus(); + var jobStatus = status.getJobStatus(); + + ctx.getFlinkService() + .getLastCheckpoint(JobID.fromHexString(jobId), ctx.getObserveConfig()) + .ifPresentOrElse( + snapshot -> jobStatus.setUpgradeSavepointPath(snapshot.getLocation()), + () -> { + if (ReconciliationUtils.isJobCancelled(status)) { + // For cancelled jobs the observed savepoint is always definite, + // so if empty we know the job doesn't have any + // checkpoints/savepoints + jobStatus.setUpgradeSavepointPath(null); + } + }); } } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/sessionjob/FlinkSessionJobObserver.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/sessionjob/FlinkSessionJobObserver.java index d02e677be1..5455954278 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/sessionjob/FlinkSessionJobObserver.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/sessionjob/FlinkSessionJobObserver.java @@ -18,6 +18,7 @@ package org.apache.flink.kubernetes.operator.observer.sessionjob; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; import org.apache.flink.kubernetes.operator.api.FlinkSessionJob; import org.apache.flink.kubernetes.operator.api.status.FlinkSessionJobStatus; import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext; @@ -67,7 +68,16 @@ protected boolean checkIfAlreadyUpgraded(FlinkResourceContext c return false; } var jobId = JobID.fromHexString(jobStatus.getJobId()); - if (ctx.getFlinkService().getJobStatus(ctx.getObserveConfig(), jobId).isPresent()) { + var deployed = + ctx.getFlinkService() + .getJobStatus(ctx.getObserveConfig(), jobId) + .map( + jsm -> + !JobStatus.CANCELLING.equals(jsm.getJobState()) + && !JobStatus.CANCELED.equals( + jsm.getJobState())) + .orElse(false); + if (deployed) { LOG.info("Job with id {} is already deployed.", jobId); return true; } else { diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/ReconciliationUtils.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/ReconciliationUtils.java index 66659df853..15e15141ef 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/ReconciliationUtils.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/ReconciliationUtils.java @@ -26,7 +26,6 @@ import org.apache.flink.kubernetes.operator.api.spec.AbstractFlinkSpec; import org.apache.flink.kubernetes.operator.api.spec.JobSpec; import org.apache.flink.kubernetes.operator.api.spec.JobState; -import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.api.status.CommonStatus; import org.apache.flink.kubernetes.operator.api.status.FlinkDeploymentStatus; import org.apache.flink.kubernetes.operator.api.status.FlinkStateSnapshotStatus; @@ -43,7 +42,6 @@ import org.apache.flink.kubernetes.operator.exception.ValidationException; import org.apache.flink.kubernetes.operator.utils.FlinkUtils; import org.apache.flink.kubernetes.operator.utils.StatusRecorder; -import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import io.fabric8.kubernetes.client.CustomResource; import io.javaoperatorsdk.operator.api.reconciler.ErrorStatusUpdateControl; @@ -55,6 +53,7 @@ import java.time.Clock; import java.time.Duration; import java.time.Instant; +import java.util.function.BiConsumer; import static org.apache.flink.api.common.JobStatus.FINISHED; import static org.apache.flink.api.common.JobStatus.RUNNING; @@ -272,10 +271,12 @@ UpdateControl toUpdateControl( return updateControl; } + if (isJobCancelling(status)) { + return updateControl.rescheduleAfter(operatorConfiguration.getProgressCheckInterval()); + } + if (upgradeStarted( - status.getReconciliationStatus(), - previous.getStatus().getReconciliationStatus()) - || current.getStatus().isImmediateReconciliationNeeded()) { + status.getReconciliationStatus(), previous.getStatus().getReconciliationStatus())) { return updateControl.rescheduleAfter(0); } @@ -293,6 +294,7 @@ UpdateControl toUpdateControl( } } + @VisibleForTesting public static Duration rescheduleAfter( JobManagerDeploymentStatus status, FlinkDeployment flinkDeployment, @@ -325,18 +327,6 @@ private static boolean savepointInProgress(JobStatus jobStatus) { return StringUtils.isNotEmpty(jobStatus.getSavepointInfo().getTriggerId()); } - public static boolean isUpgradeModeChangedToLastStateAndHADisabledPreviously( - AbstractFlinkResource flinkApp, Configuration observeConfig) { - - var deployedSpec = getDeployedSpec(flinkApp); - UpgradeMode previousUpgradeMode = deployedSpec.getJob().getUpgradeMode(); - UpgradeMode currentUpgradeMode = flinkApp.getSpec().getJob().getUpgradeMode(); - - return previousUpgradeMode != UpgradeMode.LAST_STATE - && currentUpgradeMode == UpgradeMode.LAST_STATE - && !HighAvailabilityMode.isHighAvailabilityModeActivated(observeConfig); - } - public static SPEC getDeployedSpec( AbstractFlinkResource deployment) { var reconciliationStatus = deployment.getStatus().getReconciliationStatus(); @@ -356,15 +346,15 @@ private static boolean upgradeStarted( if (currentReconState == previousReconState) { return false; } - if (currentStatus.scalingInProgress()) { - return false; - } return currentReconState == ReconciliationState.ROLLING_BACK || currentReconState == ReconciliationState.UPGRADING; } public static boolean isJobInTerminalState(CommonStatus status) { var jobState = status.getJobStatus().getState(); + if (jobState == null) { + jobState = org.apache.flink.api.common.JobStatus.RECONCILING.name(); + } return org.apache.flink.api.common.JobStatus.valueOf(jobState).isGloballyTerminalState(); } @@ -374,6 +364,25 @@ public static boolean isJobRunning(CommonStatus status) { .equals(status.getJobStatus().getState()); } + public static boolean isJobCancelled(CommonStatus status) { + return org.apache.flink.api.common.JobStatus.CANCELED + .name() + .equals(status.getJobStatus().getState()); + } + + public static boolean isJobCancellable(CommonStatus status) { + return !org.apache.flink.api.common.JobStatus.RECONCILING + .name() + .equals(status.getJobStatus().getState()); + } + + public static boolean isJobCancelling(CommonStatus status) { + return status.getJobStatus() != null + && org.apache.flink.api.common.JobStatus.CANCELLING + .name() + .equals(status.getJobStatus().getState()); + } + /** * In case of validation errors we need to (temporarily) reset the old spec so that we can * reconcile other outstanding changes, instead of simply blocking. @@ -396,7 +405,6 @@ public static boolean applyValidationErrorAndRe var deployment = ctx.getResource(); var status = deployment.getStatus(); if (!validationError.equals(status.getError())) { - LOG.error("Validation failed: " + validationError); ReconciliationUtils.updateForReconciliationError( ctx, new ValidationException(validationError)); } @@ -482,6 +490,7 @@ public static void clearLastReconciledSpecIfFirstDeploy(AbstractFlinkResource void updateReconciliationMetadata SpecUtils.writeSpecWithMeta(lastSpecWithMeta.getSpec(), newMeta)); resource.getStatus().setObservedGeneration(resource.getMetadata().getGeneration()); } + + public static void updateLastReconciledSpec( + AbstractFlinkResource resource, BiConsumer update) { + var reconStatus = resource.getStatus().getReconciliationStatus(); + var specWithMeta = reconStatus.deserializeLastReconciledSpecWithMeta(); + var spec = specWithMeta.getSpec(); + var meta = specWithMeta.getMeta(); + update.accept(spec, meta); + reconStatus.setLastReconciledSpec(SpecUtils.writeSpecWithMeta(spec, meta)); + } } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/AbstractJobReconciler.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/AbstractJobReconciler.java index babb05f88c..594632b29d 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/AbstractJobReconciler.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/AbstractJobReconciler.java @@ -23,27 +23,31 @@ import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.kubernetes.operator.api.AbstractFlinkResource; +import org.apache.flink.kubernetes.operator.api.FlinkSessionJob; import org.apache.flink.kubernetes.operator.api.diff.DiffType; import org.apache.flink.kubernetes.operator.api.spec.AbstractFlinkSpec; import org.apache.flink.kubernetes.operator.api.spec.JobState; import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.api.status.CommonStatus; import org.apache.flink.kubernetes.operator.api.status.JobStatus; +import org.apache.flink.kubernetes.operator.api.status.ReconciliationState; import org.apache.flink.kubernetes.operator.api.status.SavepointFormatType; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; import org.apache.flink.kubernetes.operator.autoscaler.KubernetesJobAutoScalerContext; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext; -import org.apache.flink.kubernetes.operator.exception.RecoveryFailureException; +import org.apache.flink.kubernetes.operator.exception.UpgradeFailureException; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.reconciler.SnapshotTriggerTimestampStore; import org.apache.flink.kubernetes.operator.reconciler.SnapshotType; import org.apache.flink.kubernetes.operator.service.CheckpointHistoryWrapper; +import org.apache.flink.kubernetes.operator.service.SuspendMode; import org.apache.flink.kubernetes.operator.utils.EventRecorder; import org.apache.flink.kubernetes.operator.utils.FlinkStateSnapshotUtils; import org.apache.flink.kubernetes.operator.utils.SnapshotUtils; import org.apache.flink.kubernetes.operator.utils.StatusRecorder; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; import io.javaoperatorsdk.operator.processing.event.ResourceID; import lombok.Value; @@ -124,34 +128,61 @@ protected boolean reconcileSpecChange( } if (currentJobState == JobState.RUNNING) { - if (desiredJobState == JobState.RUNNING) { - LOG.info("Upgrading/Restarting running job, suspending first..."); - } - AvailableUpgradeMode availableUpgradeMode = getAvailableUpgradeMode(ctx, deployConfig); - if (!availableUpgradeMode.isAvailable()) { - return false; + var jobUpgrade = getJobUpgrade(ctx, deployConfig); + if (!jobUpgrade.isAvailable()) { + // If job upgrade is currently not available for some reason we must still check if + // other reconciliation action may be taken while we wait... + LOG.info( + "Job is not running and checkpoint information is not available for executing the upgrade, waiting for upgradeable state"); + return !jobUpgrade.allowOtherReconcileActions; } + LOG.debug("Job upgrade available: {}", jobUpgrade); - eventRecorder.triggerEvent( - resource, - EventRecorder.Type.Normal, - EventRecorder.Reason.Suspended, - EventRecorder.Component.JobManagerDeployment, - MSG_SUSPENDED, - ctx.getKubernetesClient()); + var suspendMode = jobUpgrade.getSuspendMode(); + if (suspendMode != SuspendMode.NOOP) { + eventRecorder.triggerEvent( + resource, + EventRecorder.Type.Normal, + EventRecorder.Reason.Suspended, + EventRecorder.Component.JobManagerDeployment, + MSG_SUSPENDED, + ctx.getKubernetesClient()); + } - UpgradeMode upgradeMode = availableUpgradeMode.getUpgradeMode().get(); + boolean async = cancelJob(ctx, suspendMode); + if (async) { + // Async cancellation will be completed in the background, so we must exit + // reconciliation early and wait until it completes to finish the upgrade. + resource.getStatus() + .getReconciliationStatus() + .setState(ReconciliationState.UPGRADING); + ReconciliationUtils.updateLastReconciledSpec( + resource, + (s, m) -> { + s.getJob().setUpgradeMode(jobUpgrade.getRestoreMode()); + m.setFirstDeployment(false); + }); + return true; + } // We must record the upgrade mode used to the status later - currentDeploySpec.getJob().setUpgradeMode(upgradeMode); + currentDeploySpec.getJob().setUpgradeMode(jobUpgrade.getRestoreMode()); - cancelJob(ctx, upgradeMode); if (desiredJobState == JobState.RUNNING) { ReconciliationUtils.updateStatusBeforeDeploymentAttempt( resource, deployConfig, clock); } else { ReconciliationUtils.updateStatusForDeployedSpec(resource, deployConfig, clock); } + + if (suspendMode == SuspendMode.NOOP) { + // If already cancelled we want to restore immediately so we modify the current + // state + // We don't do this when we actually performed a potentially lengthy cancel action + // to allow reconciling the spec + lastReconciledSpec.getJob().setUpgradeMode(jobUpgrade.getRestoreMode()); + currentJobState = JobState.SUSPENDED; + } } if (currentJobState == JobState.SUSPENDED && desiredJobState == JobState.RUNNING) { @@ -177,65 +208,105 @@ protected boolean reconcileSpecChange( return true; } - protected AvailableUpgradeMode getAvailableUpgradeMode( - FlinkResourceContext ctx, Configuration deployConfig) throws Exception { + protected JobUpgrade getJobUpgrade(FlinkResourceContext ctx, Configuration deployConfig) + throws Exception { var resource = ctx.getResource(); var status = resource.getStatus(); var upgradeMode = resource.getSpec().getJob().getUpgradeMode(); + boolean terminal = ReconciliationUtils.isJobInTerminalState(status); if (upgradeMode == UpgradeMode.STATELESS) { LOG.info("Stateless job, ready for upgrade"); - return AvailableUpgradeMode.of(UpgradeMode.STATELESS); + return JobUpgrade.stateless(terminal); } var flinkService = ctx.getFlinkService(); - if (ReconciliationUtils.isJobInTerminalState(status) - && !flinkService.isHaMetadataAvailable(ctx.getObserveConfig())) { + if (ReconciliationUtils.isJobCancelled(status) + || (terminal && !flinkService.isHaMetadataAvailable(ctx.getObserveConfig()))) { if (!SnapshotUtils.lastSavepointKnown(status)) { - throw new RecoveryFailureException( + throw new UpgradeFailureException( "Job is in terminal state but last checkpoint is unknown, possibly due to an unrecoverable restore error. Manual restore required.", "UpgradeFailed"); } - LOG.info( - "Job is in terminal state, ready for upgrade from observed latest checkpoint/savepoint"); - return AvailableUpgradeMode.of(UpgradeMode.SAVEPOINT); + LOG.info("Job is in terminal state, ready for upgrade from observed latest state"); + return JobUpgrade.savepoint(true); } - if (ReconciliationUtils.isJobRunning(status)) { - LOG.info("Job is in running state, ready for upgrade with {}", upgradeMode); - var changedToLastStateWithoutHa = - ReconciliationUtils.isUpgradeModeChangedToLastStateAndHADisabledPreviously( - resource, ctx.getObserveConfig()); - if (changedToLastStateWithoutHa) { - LOG.info( - "Using savepoint upgrade mode when switching to last-state without HA previously enabled"); - return AvailableUpgradeMode.of(UpgradeMode.SAVEPOINT); + if (ReconciliationUtils.isJobCancelling(status)) { + LOG.info("Cancellation is in progress. Waiting for cancelled state."); + return JobUpgrade.pendingCancellation(); + } + + boolean running = ReconciliationUtils.isJobRunning(status); + boolean versionChanged = + flinkVersionChanged( + ReconciliationUtils.getDeployedSpec(resource), resource.getSpec()); + + if (upgradeMode == UpgradeMode.SAVEPOINT) { + if (running) { + LOG.info("Job is in running state, ready for upgrade with savepoint"); + return JobUpgrade.savepoint(false); + } else if (versionChanged + || deployConfig.get( + KubernetesOperatorConfigOptions + .OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED)) { + LOG.info("Falling back to last-state upgrade mode from savepoint"); + ctx.getResource() + .getSpec() + .getJob() + .setUpgradeMode(upgradeMode = UpgradeMode.LAST_STATE); + } else { + LOG.info("Last-state fallback is disabled, waiting for upgradable state"); + return JobUpgrade.pendingUpgrade(); } + } - if (flinkVersionChanged( - ReconciliationUtils.getDeployedSpec(resource), resource.getSpec())) { - LOG.info("Using savepoint upgrade mode when upgrading Flink version"); - return AvailableUpgradeMode.of(UpgradeMode.SAVEPOINT); + if (upgradeMode == UpgradeMode.LAST_STATE) { + if (versionChanged) { + // We need some special handling in case of version upgrades where HA based + // last-state upgrade is not possible + boolean savepointPossible = + !StringUtils.isNullOrWhitespaceOnly( + ctx.getObserveConfig() + .getString(CheckpointingOptions.SAVEPOINT_DIRECTORY)); + if (running && savepointPossible) { + LOG.info("Using savepoint to upgrade Flink version"); + return JobUpgrade.savepoint(false); + } else if (ReconciliationUtils.isJobCancellable(resource.getStatus())) { + LOG.info("Using last-state upgrade with cancellation to upgrade Flink version"); + return JobUpgrade.lastStateUsingCancel(); + } else { + LOG.info( + "Neither savepoint nor cancellation is possible, cannot perform stateful version upgrade"); + return JobUpgrade.pendingUpgrade(); + } } - if (upgradeMode == UpgradeMode.LAST_STATE) { - return changeLastStateIfCheckpointTooOld(ctx, deployConfig); + boolean cancellable = allowLastStateCancel(ctx); + if (running) { + return getUpgradeModeBasedOnStateAge(ctx, deployConfig, cancellable); } - return AvailableUpgradeMode.of(UpgradeMode.SAVEPOINT); + if (cancellable) { + LOG.info("Using cancel to perform last-state upgrade"); + return JobUpgrade.lastStateUsingCancel(); + } } - return AvailableUpgradeMode.unavailable(); + return JobUpgrade.unavailable(); } @VisibleForTesting - protected AvailableUpgradeMode changeLastStateIfCheckpointTooOld( - FlinkResourceContext ctx, Configuration deployConfig) throws Exception { + protected JobUpgrade getUpgradeModeBasedOnStateAge( + FlinkResourceContext ctx, Configuration deployConfig, boolean cancellable) + throws Exception { + var defaultMode = + cancellable ? JobUpgrade.lastStateUsingCancel() : JobUpgrade.lastStateUsingHaMeta(); var maxAge = deployConfig.get(OPERATOR_JOB_UPGRADE_LAST_STATE_CHECKPOINT_MAX_AGE); if (maxAge == null) { - return AvailableUpgradeMode.of(UpgradeMode.LAST_STATE); + return defaultMode; } var jobStatus = ctx.getResource().getStatus().getJobStatus(); @@ -247,7 +318,7 @@ protected AvailableUpgradeMode changeLastStateIfCheckpointTooOld( // If job started recently, no need to query checkpoint if (withinMaxAge.test(startTime)) { - return AvailableUpgradeMode.of(UpgradeMode.LAST_STATE); + return defaultMode; } var chkInfo = ctx.getFlinkService().getCheckpointInfo(jobId, ctx.getObserveConfig()); @@ -264,15 +335,29 @@ protected AvailableUpgradeMode changeLastStateIfCheckpointTooOld( if (withinMaxAge.test(completedTs)) { // We have a recent enough checkpoint - return AvailableUpgradeMode.of(UpgradeMode.LAST_STATE); + return defaultMode; } else if (withinMaxAge.test(pendingTs)) { LOG.info("Waiting for pending checkpoint to complete before upgrading."); - return AvailableUpgradeMode.pendingUpgrade(); + return JobUpgrade.pendingUpgrade(); } else { LOG.info( "Using savepoint upgrade mode because latest checkpoint is too old for last-state upgrade"); - return AvailableUpgradeMode.of(UpgradeMode.SAVEPOINT); + return JobUpgrade.savepoint(false); + } + } + + private boolean allowLastStateCancel(FlinkResourceContext ctx) { + var resource = ctx.getResource(); + if (!ReconciliationUtils.isJobCancellable(resource.getStatus())) { + return false; + } + if (resource instanceof FlinkSessionJob) { + return true; } + + var conf = ctx.getObserveConfig(); + return conf.get(KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_CANCEL_JOB) + || !ctx.getFlinkService().isHaMetadataAvailable(conf); } protected void restoreJob( @@ -483,7 +568,7 @@ private void redeployWithSavepoint( JobState desiredJobState) throws Exception { LOG.info("Redeploying from savepoint"); - cancelJob(ctx, UpgradeMode.STATELESS); + cancelJob(ctx, SuspendMode.STATELESS); currentDeploySpec.getJob().setUpgradeMode(UpgradeMode.SAVEPOINT); Optional savepointPath = @@ -506,10 +591,11 @@ private void redeployWithSavepoint( * Cancel the job for the given resource using the specified upgrade mode. * * @param ctx Reconciler context. - * @param upgradeMode Upgrade mode used during cancel. + * @param suspendMode Suspend mode used during cancel. * @throws Exception Error during cancellation. + * @return True if this is an async cancellation */ - protected abstract void cancelJob(FlinkResourceContext ctx, UpgradeMode upgradeMode) + protected abstract boolean cancelJob(FlinkResourceContext ctx, SuspendMode suspendMode) throws Exception; /** @@ -522,24 +608,50 @@ protected abstract void cancelJob(FlinkResourceContext ctx, UpgradeMode upgr /** Object to capture available upgrade mode. */ @Value - public static class AvailableUpgradeMode { - Optional upgradeMode; + public static class JobUpgrade { + SuspendMode suspendMode; + UpgradeMode restoreMode; + boolean available; boolean allowFallback; + boolean allowOtherReconcileActions; + + static JobUpgrade stateless(boolean terminal) { + return new JobUpgrade( + terminal ? SuspendMode.NOOP : SuspendMode.STATELESS, + UpgradeMode.STATELESS, + true, + false, + false); + } + + static JobUpgrade savepoint(boolean terminal) { + return new JobUpgrade( + terminal ? SuspendMode.NOOP : SuspendMode.SAVEPOINT, + UpgradeMode.SAVEPOINT, + true, + false, + false); + } + + static JobUpgrade lastStateUsingHaMeta() { + return new JobUpgrade( + SuspendMode.LAST_STATE, UpgradeMode.LAST_STATE, true, false, false); + } - public boolean isAvailable() { - return upgradeMode.isPresent(); + static JobUpgrade lastStateUsingCancel() { + return new JobUpgrade(SuspendMode.CANCEL, UpgradeMode.SAVEPOINT, true, false, false); } - static AvailableUpgradeMode of(UpgradeMode upgradeMode) { - return new AvailableUpgradeMode(Optional.of(upgradeMode), false); + static JobUpgrade pendingCancellation() { + return new JobUpgrade(null, null, false, false, false); } - static AvailableUpgradeMode unavailable() { - return new AvailableUpgradeMode(Optional.empty(), true); + static JobUpgrade pendingUpgrade() { + return new JobUpgrade(null, null, false, false, true); } - static AvailableUpgradeMode pendingUpgrade() { - return new AvailableUpgradeMode(Optional.empty(), false); + static JobUpgrade unavailable() { + return new JobUpgrade(null, null, false, true, true); } } } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconciler.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconciler.java index ac6c69e937..a078a5934f 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconciler.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconciler.java @@ -32,11 +32,12 @@ import org.apache.flink.kubernetes.operator.autoscaler.KubernetesJobAutoScalerContext; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext; -import org.apache.flink.kubernetes.operator.exception.RecoveryFailureException; +import org.apache.flink.kubernetes.operator.exception.UpgradeFailureException; import org.apache.flink.kubernetes.operator.health.ClusterHealthInfo; import org.apache.flink.kubernetes.operator.observer.ClusterHealthEvaluator; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.service.FlinkService; +import org.apache.flink.kubernetes.operator.service.SuspendMode; import org.apache.flink.kubernetes.operator.utils.EventRecorder; import org.apache.flink.kubernetes.operator.utils.FlinkUtils; import org.apache.flink.kubernetes.operator.utils.IngressUtils; @@ -50,7 +51,6 @@ import io.javaoperatorsdk.operator.api.reconciler.DeleteControl; import io.javaoperatorsdk.operator.processing.event.ResourceID; import lombok.SneakyThrows; -import org.apache.commons.lang3.ObjectUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,36 +76,25 @@ public ApplicationReconciler( } @Override - protected AvailableUpgradeMode getAvailableUpgradeMode( + protected JobUpgrade getJobUpgrade( FlinkResourceContext ctx, Configuration deployConfig) throws Exception { var deployment = ctx.getResource(); var status = deployment.getStatus(); - var availableUpgradeMode = super.getAvailableUpgradeMode(ctx, deployConfig); + var availableUpgradeMode = super.getJobUpgrade(ctx, deployConfig); if (availableUpgradeMode.isAvailable() || !availableUpgradeMode.isAllowFallback()) { return availableUpgradeMode; } var flinkService = ctx.getFlinkService(); - boolean lastStateAllowed = - deployment.getSpec().getJob().getUpgradeMode() == UpgradeMode.LAST_STATE - || deployConfig.getBoolean( - KubernetesOperatorConfigOptions - .OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED); - - if (lastStateAllowed - && HighAvailabilityMode.isHighAvailabilityModeActivated(deployConfig) + if (HighAvailabilityMode.isHighAvailabilityModeActivated(deployConfig) && HighAvailabilityMode.isHighAvailabilityModeActivated(ctx.getObserveConfig()) - && !flinkVersionChanged( - ReconciliationUtils.getDeployedSpec(deployment), deployment.getSpec())) { - - if (flinkService.isHaMetadataAvailable(deployConfig)) { - LOG.info( - "Job is not running but HA metadata is available for last state restore, ready for upgrade"); - return AvailableUpgradeMode.of(UpgradeMode.LAST_STATE); - } + && flinkService.isHaMetadataAvailable(deployConfig)) { + LOG.info( + "Job is not running but HA metadata is available for last state restore, ready for upgrade"); + return JobUpgrade.lastStateUsingHaMeta(); } var jmDeployStatus = status.getJobManagerDeploymentStatus(); @@ -117,22 +106,20 @@ protected AvailableUpgradeMode getAvailableUpgradeMode( != UpgradeMode.LAST_STATE && FlinkUtils.jmPodNeverStarted(ctx.getJosdkContext())) { deleteJmThatNeverStarted(flinkService, deployment, deployConfig); - return getAvailableUpgradeMode(ctx, deployConfig); + return getJobUpgrade(ctx, deployConfig); } if ((jmDeployStatus == JobManagerDeploymentStatus.MISSING || jmDeployStatus == JobManagerDeploymentStatus.ERROR) && !flinkService.isHaMetadataAvailable(deployConfig)) { - throw new RecoveryFailureException( + throw new UpgradeFailureException( "JobManager deployment is missing and HA data is not available to make stateful upgrades. " + "It is possible that the job has finished or terminally failed, or the configmaps have been deleted. " + "Manual restore required.", "UpgradeFailed"); } - LOG.info( - "Job is not running and HA metadata is not available or usable for executing the upgrade, waiting for upgradeable state"); - return AvailableUpgradeMode.unavailable(); + return JobUpgrade.unavailable(); } private void deleteJmThatNeverStarted( @@ -234,12 +221,13 @@ private void setJobIdIfNecessary( } @Override - protected void cancelJob(FlinkResourceContext ctx, UpgradeMode upgradeMode) + protected boolean cancelJob(FlinkResourceContext ctx, SuspendMode suspendMode) throws Exception { - var conf = ObjectUtils.firstNonNull(ctx.getObserveConfig(), new Configuration()); - ctx.getFlinkService() - .cancelJob(ctx.getResource(), upgradeMode, conf) - .ifPresent(location -> setUpgradeSavepointPath(ctx, location)); + var result = + ctx.getFlinkService() + .cancelJob(ctx.getResource(), suspendMode, ctx.getObserveConfig()); + result.getSavepointPath().ifPresent(location -> setUpgradeSavepointPath(ctx, location)); + return result.isPending(); } @Override @@ -379,16 +367,17 @@ protected DeleteControl cleanupInternal(FlinkResourceContext ct var deployment = ctx.getResource(); var status = deployment.getStatus(); var conf = ctx.getDeployConfig(ctx.getResource().getSpec()); - if (status.getReconciliationStatus().isBeforeFirstDeployment()) { + if (status.getReconciliationStatus().isBeforeFirstDeployment() + || ReconciliationUtils.isJobInTerminalState(status)) { ctx.getFlinkService() .deleteClusterDeployment(deployment.getMetadata(), status, conf, true); } else { var observeConfig = ctx.getObserveConfig(); - UpgradeMode upgradeMode = + var suspendMode = observeConfig.getBoolean(KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION) - ? UpgradeMode.SAVEPOINT - : UpgradeMode.STATELESS; - cancelJob(ctx, upgradeMode); + ? SuspendMode.SAVEPOINT + : SuspendMode.STATELESS; + cancelJob(ctx, suspendMode); } return DeleteControl.defaultDelete(); } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconciler.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconciler.java index 3f54a8c73f..af227a9324 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconciler.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconciler.java @@ -23,25 +23,25 @@ import org.apache.flink.kubernetes.operator.api.FlinkDeployment; import org.apache.flink.kubernetes.operator.api.FlinkSessionJob; import org.apache.flink.kubernetes.operator.api.spec.FlinkSessionJobSpec; -import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; +import org.apache.flink.kubernetes.operator.api.spec.JobState; import org.apache.flink.kubernetes.operator.api.status.FlinkSessionJobStatus; import org.apache.flink.kubernetes.operator.api.status.JobManagerDeploymentStatus; import org.apache.flink.kubernetes.operator.autoscaler.KubernetesJobAutoScalerContext; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext; +import org.apache.flink.kubernetes.operator.observer.JobStatusObserver; +import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.reconciler.deployment.AbstractJobReconciler; -import org.apache.flink.kubernetes.operator.service.AbstractFlinkService; +import org.apache.flink.kubernetes.operator.service.SuspendMode; import org.apache.flink.kubernetes.operator.utils.EventRecorder; import org.apache.flink.kubernetes.operator.utils.StatusRecorder; import io.javaoperatorsdk.operator.api.reconciler.DeleteControl; import io.javaoperatorsdk.operator.processing.event.ResourceID; -import org.apache.commons.lang3.ObjectUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Optional; -import java.util.concurrent.ExecutionException; /** The reconciler for the {@link FlinkSessionJob}. */ public class SessionJobReconciler @@ -98,14 +98,13 @@ public void deploy( } @Override - protected void cancelJob(FlinkResourceContext ctx, UpgradeMode upgradeMode) + protected boolean cancelJob(FlinkResourceContext ctx, SuspendMode suspendMode) throws Exception { - var conf = ObjectUtils.firstNonNull(ctx.getObserveConfig(), new Configuration()); - - ctx.getFlinkService() - .cancelSessionJob(ctx.getResource(), upgradeMode, conf) - .ifPresent(location -> setUpgradeSavepointPath(ctx, location)); - ctx.getResource().getStatus().getJobStatus().setJobId(null); + var result = + ctx.getFlinkService() + .cancelSessionJob(ctx.getResource(), suspendMode, ctx.getObserveConfig()); + result.getSavepointPath().ifPresent(location -> setUpgradeSavepointPath(ctx, location)); + return result.isPending(); } @Override @@ -115,6 +114,25 @@ protected void cleanupAfterFailedJob(FlinkResourceContext ctx) @Override public DeleteControl cleanupInternal(FlinkResourceContext ctx) { + var status = ctx.getResource().getStatus(); + long delay = ctx.getOperatorConfig().getProgressCheckInterval().toMillis(); + if (status.getReconciliationStatus().isBeforeFirstDeployment() + || ReconciliationUtils.isJobInTerminalState(status) + || status.getReconciliationStatus() + .deserializeLastReconciledSpec() + .getJob() + .getState() + == JobState.SUSPENDED + || JobStatusObserver.JOB_NOT_FOUND_ERR.equals(status.getError())) { + // Job is not running, nothing to do... + return DeleteControl.defaultDelete(); + } + + if (ReconciliationUtils.isJobCancelling(status)) { + LOG.info("Waiting for pending cancellation"); + return DeleteControl.noFinalizerRemoval().rescheduleAfter(delay); + } + Optional flinkDepOptional = ctx.getJosdkContext().getSecondaryResource(FlinkDeployment.class); @@ -123,25 +141,21 @@ public DeleteControl cleanupInternal(FlinkResourceContext ctx) if (jobID != null) { try { var observeConfig = ctx.getObserveConfig(); - UpgradeMode upgradeMode = + var suspendMode = observeConfig.getBoolean( KubernetesOperatorConfigOptions.SAVEPOINT_ON_DELETION) - ? UpgradeMode.SAVEPOINT - : UpgradeMode.STATELESS; - cancelJob(ctx, upgradeMode); - } catch (ExecutionException e) { - if (AbstractFlinkService.isJobMissingOrTerminated(e)) { - return DeleteControl.defaultDelete(); + ? SuspendMode.SAVEPOINT + : SuspendMode.STATELESS; + if (cancelJob(ctx, suspendMode)) { + LOG.info("Waiting for pending cancellation"); + return DeleteControl.noFinalizerRemoval().rescheduleAfter(delay); } - long delay = ctx.getOperatorConfig().getProgressCheckInterval().toMillis(); + } catch (Exception e) { LOG.error( - "Failed to cancel job {}, will reschedule after {} milliseconds.", - jobID, + "Failed to cancel job, will reschedule after {} milliseconds.", delay, e); return DeleteControl.noFinalizerRemoval().rescheduleAfter(delay); - } catch (Exception e) { - LOG.error("Failed to cancel job {}.", jobID, e); } } } else { diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java index 8e4c9ebd0a..c89baf8968 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkService.java @@ -34,7 +34,7 @@ import org.apache.flink.kubernetes.operator.api.spec.FlinkSessionJobSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; import org.apache.flink.kubernetes.operator.api.spec.JobSpec; -import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; +import org.apache.flink.kubernetes.operator.api.status.CommonStatus; 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.status.Savepoint; @@ -42,12 +42,14 @@ import org.apache.flink.kubernetes.operator.artifact.ArtifactManager; import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; -import org.apache.flink.kubernetes.operator.exception.RecoveryFailureException; +import org.apache.flink.kubernetes.operator.exception.ReconciliationException; +import org.apache.flink.kubernetes.operator.exception.UpgradeFailureException; import org.apache.flink.kubernetes.operator.observer.CheckpointFetchResult; import org.apache.flink.kubernetes.operator.observer.CheckpointStatsResult; import org.apache.flink.kubernetes.operator.observer.SavepointFetchResult; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.utils.EnvUtils; +import org.apache.flink.kubernetes.operator.utils.EventRecorder; import org.apache.flink.kubernetes.operator.utils.FlinkUtils; import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneClientHAServices; @@ -96,7 +98,6 @@ import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FileUtils; -import org.apache.flink.util.FlinkException; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; @@ -180,8 +181,6 @@ public AbstractFlinkService( protected abstract PodList getJmPodList(String namespace, String clusterId); - protected abstract PodList getTmPodList(String namespace, String clusterId); - protected abstract void deployApplicationCluster(JobSpec jobSpec, Configuration conf) throws Exception; @@ -311,224 +310,174 @@ public JobResult requestJobResult(Configuration conf, JobID jobID) throws Except } } - protected Optional cancelJob( + protected CancelResult cancelJob( FlinkDeployment deployment, - UpgradeMode upgradeMode, + SuspendMode suspendMode, Configuration conf, - boolean deleteClusterAfterSavepoint) + boolean deleteCluster) throws Exception { - var deploymentStatus = deployment.getStatus(); - var jobIdString = deploymentStatus.getJobStatus().getJobId(); - var jobId = jobIdString != null ? JobID.fromHexString(jobIdString) : null; - - Optional savepointOpt = Optional.empty(); - var savepointFormatType = - conf.get(KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_FORMAT_TYPE); + var status = deployment.getStatus(); + String savepointPath = null; try (var clusterClient = getClusterClient(conf)) { - var clusterId = clusterClient.getClusterId(); - switch (upgradeMode) { - case STATELESS: - if (ReconciliationUtils.isJobRunning(deployment.getStatus())) { - LOG.info("Job is running, cancelling job."); - try { - clusterClient - .cancel(Preconditions.checkNotNull(jobId)) - .get( - operatorConfig.getFlinkCancelJobTimeout().toSeconds(), - TimeUnit.SECONDS); - LOG.info("Job successfully cancelled."); - } catch (Exception e) { - LOG.error("Could not shut down cluster gracefully, deleting...", e); - } - } - deleteClusterDeployment(deployment.getMetadata(), deploymentStatus, conf, true); - break; + switch (suspendMode) { case SAVEPOINT: - final String savepointDirectory = - Preconditions.checkNotNull( - conf.get(CheckpointingOptions.SAVEPOINT_DIRECTORY)); - final long timeout = - conf.get(ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT) - .getSeconds(); - if (ReconciliationUtils.isJobRunning(deploymentStatus)) { + savepointPath = savepointJobOrError(clusterClient, status, conf); + break; + case STATELESS: + if (ReconciliationUtils.isJobCancellable(status)) { try { - LOG.info("Suspending job with savepoint."); - String savepoint = - clusterClient - .stopWithSavepoint( - Preconditions.checkNotNull(jobId), - conf.getBoolean( - KubernetesOperatorConfigOptions - .DRAIN_ON_SAVEPOINT_DELETION), - savepointDirectory, - savepointFormatType) - .get(timeout, TimeUnit.SECONDS); - savepointOpt = Optional.of(savepoint); - LOG.info("Job successfully suspended with savepoint {}.", savepoint); - } catch (TimeoutException exception) { - throw new FlinkException( - String.format( - "Timed out stopping the job %s in Flink cluster %s with savepoint, " - + "please configure a larger timeout via '%s'", - jobId, - clusterId, - ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT - .key()), - exception); - } catch (Exception e) { - var stopWithSavepointException = - ExceptionUtils.findThrowableSerializedAware( - e, - StopWithSavepointStoppingException.class, - getClass().getClassLoader()); - if (stopWithSavepointException.isPresent()) { - // Handle edge case where the savepoint completes but the job fails - // right afterward. - savepointOpt = - Optional.of( - stopWithSavepointException - .get() - .getSavepointPath()); - } else { - // Rethrow if savepoint was not completed successfully. - throw e; - } + cancelJobOrError(clusterClient, status, true); + } catch (Exception ex) { + // We can simply delete the deployment for stateless } - } else if (ReconciliationUtils.isJobInTerminalState(deploymentStatus)) { - LOG.info( - "Job is already in terminal state skipping cancel-with-savepoint operation."); - } else { - throw new RuntimeException( - "Unexpected non-terminal status: " + deploymentStatus); - } - if (deleteClusterAfterSavepoint) { - LOG.info("Cleaning up deployment after stop-with-savepoint"); - - deleteClusterDeployment( - deployment.getMetadata(), deploymentStatus, conf, true); } break; - case LAST_STATE: - deleteClusterDeployment( - deployment.getMetadata(), deploymentStatus, conf, false); - break; - default: - throw new RuntimeException("Unsupported upgrade mode " + upgradeMode); + case CANCEL: + cancelJobOrError(clusterClient, status, false); + // This is async we need to return + return CancelResult.pending(); } } - deploymentStatus.getJobStatus().setState(JobStatus.FINISHED.name()); - - // Unless we leave the jm around after savepoint, we should wait until it has finished - // shutting down - if (deleteClusterAfterSavepoint || upgradeMode != UpgradeMode.SAVEPOINT) { - deploymentStatus.setJobManagerDeploymentStatus(JobManagerDeploymentStatus.MISSING); + if (suspendMode.deleteCluster() || deleteCluster) { + deleteClusterDeployment( + deployment.getMetadata(), status, conf, suspendMode.deleteHaMeta()); } - return savepointOpt; + + status.getJobStatus().setState(JobStatus.FINISHED.name()); + return CancelResult.completed(savepointPath); } @Override - public Optional cancelSessionJob( - FlinkSessionJob sessionJob, UpgradeMode upgradeMode, Configuration conf) + public CancelResult cancelSessionJob( + FlinkSessionJob sessionJob, SuspendMode suspendMode, Configuration conf) throws Exception { - var sessionJobStatus = sessionJob.getStatus(); - var jobStatus = sessionJobStatus.getJobStatus(); - var jobIdString = jobStatus.getJobId(); - Preconditions.checkNotNull(jobIdString, "The job to be suspend should not be null"); - var jobId = JobID.fromHexString(jobIdString); - Optional savepointOpt = Optional.empty(); + var status = sessionJob.getStatus(); + String savepointPath = null; + try (var clusterClient = getClusterClient(conf)) { + switch (suspendMode) { + case STATELESS: + case CANCEL: + cancelJobOrError(clusterClient, status, suspendMode == SuspendMode.STATELESS); + // This is async we need to return and re-observe + return CancelResult.pending(); + case SAVEPOINT: + savepointPath = savepointJobOrError(clusterClient, status, conf); + break; + } + } + status.getJobStatus().setState(JobStatus.FINISHED.name()); + status.getJobStatus().setJobId(null); + return CancelResult.completed(savepointPath); + } + public void cancelJobOrError( + RestClusterClient clusterClient, + CommonStatus status, + boolean ignoreMissing) { + var jobID = JobID.fromHexString(status.getJobStatus().getJobId()); + if (ReconciliationUtils.isJobCancelling(status)) { + LOG.info("Job already cancelling"); + return; + } + LOG.info("Cancelling job"); + try { + clusterClient + .cancel(jobID) + .get(operatorConfig.getFlinkCancelJobTimeout().toSeconds(), TimeUnit.SECONDS); + LOG.info("Cancellation successfully initiated"); + } catch (Exception e) { + if (isJobMissing(e)) { + if (ignoreMissing) { + LOG.info("Job already missing"); + } else { + throw new UpgradeFailureException( + "Cannot find job when trying to cancel", + EventRecorder.Reason.CleanupFailed.name(), + e); + } + } else if (isJobTerminated(e)) { + LOG.info("Job already terminated"); + } else { + LOG.warn("Error while cancelling job", e); + throw new UpgradeFailureException( + "Cancellation Error", EventRecorder.Reason.CleanupFailed.name(), e); + } + } + status.getJobStatus().setState(JobStatus.CANCELLING.name()); + } + + public String savepointJobOrError( + RestClusterClient clusterClient, CommonStatus status, Configuration conf) { + var jobID = JobID.fromHexString(status.getJobStatus().getJobId()); + String savepointDirectory = conf.get(CheckpointingOptions.SAVEPOINT_DIRECTORY); var savepointFormatType = conf.get(KubernetesOperatorConfigOptions.OPERATOR_SAVEPOINT_FORMAT_TYPE); - - LOG.debug("Current job state: {}", jobStatus.getState()); - - if (!ReconciliationUtils.isJobInTerminalState(sessionJobStatus)) { - LOG.debug("Job is not in terminal state, cancelling it"); - - try (var clusterClient = getClusterClient(conf)) { - switch (upgradeMode) { - case STATELESS: - LOG.info("Cancelling job."); - try { - clusterClient - .cancel(jobId) - .get( - operatorConfig.getFlinkCancelJobTimeout().toSeconds(), - TimeUnit.SECONDS); - LOG.info("Job successfully cancelled."); - } catch (Exception e) { - if (isJobMissingOrTerminated(e)) { - LOG.info("Job already missing or terminated"); - } else { - throw e; - } - } - break; - case SAVEPOINT: - if (ReconciliationUtils.isJobRunning(sessionJobStatus)) { - LOG.info("Suspending job with savepoint."); - final String savepointDirectory = - Preconditions.checkNotNull( - conf.get(CheckpointingOptions.SAVEPOINT_DIRECTORY)); - final long timeout = - conf.get(ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT) - .getSeconds(); - try { - String savepoint = - clusterClient - .stopWithSavepoint( - jobId, - conf.getBoolean( - KubernetesOperatorConfigOptions - .DRAIN_ON_SAVEPOINT_DELETION), - savepointDirectory, - savepointFormatType) - .get(timeout, TimeUnit.SECONDS); - savepointOpt = Optional.of(savepoint); - LOG.info( - "Job successfully suspended with savepoint {}.", savepoint); - } catch (TimeoutException exception) { - throw new FlinkException( - String.format( - "Timed out stopping the job %s with savepoint, " - + "please configure a larger timeout via '%s'", - jobId, - ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT - .key()), - exception); - } - } else { - throw new RuntimeException( - "Unexpected non-terminal status: " + jobStatus.getState()); - } - break; - case LAST_STATE: - default: - throw new RuntimeException("Unsupported upgrade mode " + upgradeMode); + long timeout = conf.get(ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT).getSeconds(); + String savepointPath; + if (ReconciliationUtils.isJobRunning(status)) { + LOG.info("Suspending job with savepoint"); + try { + savepointPath = + clusterClient + .stopWithSavepoint( + jobID, + conf.getBoolean( + KubernetesOperatorConfigOptions + .DRAIN_ON_SAVEPOINT_DELETION), + savepointDirectory, + savepointFormatType) + .get(timeout, TimeUnit.SECONDS); + } catch (TimeoutException exception) { + throw new UpgradeFailureException( + String.format( + "Timed out stopping the job %s with savepoint, " + + "please configure a larger timeout via '%s'", + jobID, ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT.key()), + EventRecorder.Reason.SavepointError.name(), + exception); + } catch (Exception e) { + var stopWithSavepointException = + ExceptionUtils.findThrowableSerializedAware( + e, + StopWithSavepointStoppingException.class, + getClass().getClassLoader()); + if (stopWithSavepointException.isPresent()) { + // Handle edge case where the savepoint completes but the job fails + // right afterward. + savepointPath = stopWithSavepointException.get().getSavepointPath(); + } else { + // Rethrow if savepoint was not completed successfully. + throw new UpgradeFailureException( + "Savepoint Error", EventRecorder.Reason.SavepointError.name(), e); } } } else { - LOG.debug("Job is in terminal state, skipping cancel"); + throw new RuntimeException("Unexpected job status: " + status); + } + LOG.info("Job successfully suspended with savepoint {}", savepointPath); + return savepointPath; + } + + public static boolean isJobMissing(Exception e) { + if (findThrowable(e, FlinkJobNotFoundException.class).isPresent()) { + return true; } - jobStatus.setState(JobStatus.FINISHED.name()); - return savepointOpt; + return findThrowable(e, RestClientException.class) + .map(RestClientException::getHttpResponseStatus) + .map(respCode -> HttpResponseStatus.NOT_FOUND == respCode) + .orElse(false); } - public static boolean isJobMissingOrTerminated(Exception e) { - if (findThrowable(e, FlinkJobNotFoundException.class).isPresent() - || findThrowable(e, FlinkJobTerminatedWithoutCancellationException.class) - .isPresent()) { + public static boolean isJobTerminated(Exception e) { + if (findThrowable(e, FlinkJobTerminatedWithoutCancellationException.class).isPresent()) { return true; } return findThrowable(e, RestClientException.class) .map(RestClientException::getHttpResponseStatus) - .map( - respCode -> - HttpResponseStatus.NOT_FOUND == respCode - || HttpResponseStatus.CONFLICT == respCode) + .map(respCode -> HttpResponseStatus.CONFLICT == respCode) .orElse(false); } @@ -589,15 +538,20 @@ public String triggerCheckpoint( } @Override - public Optional getLastCheckpoint(JobID jobId, Configuration conf) throws Exception { - var latestCheckpointOpt = getCheckpointInfo(jobId, conf).f0; + public Optional getLastCheckpoint(JobID jobId, Configuration conf) { + Optional latestCheckpointOpt; + try { + latestCheckpointOpt = getCheckpointInfo(jobId, conf).f0; + } catch (Exception e) { + throw new ReconciliationException("Could not observe latest savepoint information", e); + } if (latestCheckpointOpt.isPresent() && latestCheckpointOpt .get() .getExternalPointer() .equals(NonPersistentMetadataCheckpointStorageLocation.EXTERNAL_POINTER)) { - throw new RecoveryFailureException( + throw new UpgradeFailureException( "Latest checkpoint not externally addressable, manual recovery required.", "CheckpointNotFound"); } @@ -837,7 +791,6 @@ public RestClusterClient getClusterClient(Configuration conf) throws Exc ExternalServiceDecorator.getNamespacedExternalServiceName( clusterId, namespace)); final String restServerAddress = String.format("http://%s:%s", host, port); - LOG.debug("Creating RestClusterClient({})", restServerAddress); return new RestClusterClient<>( operatorRestConf, clusterId, @@ -980,7 +933,7 @@ protected static Configuration removeOperatorConfigs(Configuration config) { private void validateHaMetadataExists(Configuration conf) { if (!isHaMetadataAvailable(conf)) { - throw new RecoveryFailureException( + throw new UpgradeFailureException( "HA metadata not available to restore from last state. " + "It is possible that the job has finished or terminally failed, or the configmaps have been deleted. ", "RestoreFailed"); diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/FlinkService.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/FlinkService.java index 71b1476999..7707e38156 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/FlinkService.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/FlinkService.java @@ -25,7 +25,6 @@ import org.apache.flink.kubernetes.operator.api.FlinkSessionJob; import org.apache.flink.kubernetes.operator.api.spec.FlinkSessionJobSpec; import org.apache.flink.kubernetes.operator.api.spec.JobSpec; -import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.api.status.FlinkDeploymentStatus; import org.apache.flink.kubernetes.operator.api.status.Savepoint; import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext; @@ -38,6 +37,8 @@ import io.fabric8.kubernetes.api.model.ObjectMeta; import io.fabric8.kubernetes.api.model.PodList; import io.fabric8.kubernetes.client.KubernetesClient; +import lombok.AllArgsConstructor; +import lombok.Getter; import javax.annotation.Nullable; @@ -73,8 +74,7 @@ JobID submitJobToSessionCluster( JobResult requestJobResult(Configuration conf, JobID jobID) throws Exception; - Optional cancelJob( - FlinkDeployment deployment, UpgradeMode upgradeMode, Configuration conf) + CancelResult cancelJob(FlinkDeployment deployment, SuspendMode suspendMode, Configuration conf) throws Exception; void deleteClusterDeployment( @@ -83,8 +83,8 @@ void deleteClusterDeployment( Configuration conf, boolean deleteHaData); - Optional cancelSessionJob( - FlinkSessionJob sessionJob, UpgradeMode upgradeMode, Configuration conf) + CancelResult cancelSessionJob( + FlinkSessionJob sessionJob, SuspendMode suspendMode, Configuration conf) throws Exception; String triggerSavepoint( @@ -100,7 +100,7 @@ String triggerCheckpoint( Configuration conf) throws Exception; - Optional getLastCheckpoint(JobID jobId, Configuration conf) throws Exception; + Optional getLastCheckpoint(JobID jobId, Configuration conf); SavepointFetchResult fetchSavepointInfo(String triggerId, String jobId, Configuration conf); @@ -126,4 +126,23 @@ Map getMetrics(Configuration conf, String jobId, List me throws Exception; RestClusterClient getClusterClient(Configuration conf) throws Exception; + + /** Result of a cancel operation. */ + @AllArgsConstructor + class CancelResult { + @Getter boolean pending; + String savepointPath; + + public static CancelResult completed(String path) { + return new CancelResult(false, path); + } + + public static CancelResult pending() { + return new CancelResult(true, null); + } + + public Optional getSavepointPath() { + return Optional.ofNullable(savepointPath); + } + } } diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/NativeFlinkService.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/NativeFlinkService.java index 56fe53b74f..ffd071e662 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/NativeFlinkService.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/NativeFlinkService.java @@ -35,7 +35,6 @@ import org.apache.flink.kubernetes.operator.api.FlinkDeployment; import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; import org.apache.flink.kubernetes.operator.api.spec.JobSpec; -import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.artifact.ArtifactManager; import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; @@ -67,7 +66,6 @@ import java.time.Duration; import java.util.HashMap; import java.util.Map; -import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; @@ -118,10 +116,10 @@ public void deploySessionCluster(Configuration conf) throws Exception { } @Override - public Optional cancelJob( - FlinkDeployment deployment, UpgradeMode upgradeMode, Configuration configuration) + public CancelResult cancelJob( + FlinkDeployment deployment, SuspendMode suspendMode, Configuration configuration) throws Exception { - return cancelJob(deployment, upgradeMode, configuration, false); + return cancelJob(deployment, suspendMode, configuration, false); } @Override @@ -133,12 +131,6 @@ protected PodList getJmPodList(String namespace, String clusterId) { .list(); } - @Override - protected PodList getTmPodList(String namespace, String clusterId) { - // Native mode does not manage TaskManager - return new PodList(); - } - protected void submitClusterInternal(Configuration conf) throws Exception { LOG.info("Deploying session cluster"); final ClusterClientServiceLoader clusterClientServiceLoader = diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkService.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkService.java index 833f915884..c91341f213 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkService.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/StandaloneFlinkService.java @@ -28,7 +28,6 @@ import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.operator.api.FlinkDeployment; import org.apache.flink.kubernetes.operator.api.spec.JobSpec; -import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.artifact.ArtifactManager; import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration; import org.apache.flink.kubernetes.operator.config.Mode; @@ -46,7 +45,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -81,10 +79,10 @@ public void deploySessionCluster(Configuration conf) throws Exception { } @Override - public Optional cancelJob( - FlinkDeployment deployment, UpgradeMode upgradeMode, Configuration conf) + public CancelResult cancelJob( + FlinkDeployment deployment, SuspendMode suspendMode, Configuration conf) throws Exception { - return cancelJob(deployment, upgradeMode, conf, true); + return cancelJob(deployment, suspendMode, conf, true); } @Override @@ -96,15 +94,6 @@ protected PodList getJmPodList(String namespace, String clusterId) { .list(); } - @Override - protected PodList getTmPodList(String namespace, String clusterId) { - return kubernetesClient - .pods() - .inNamespace(namespace) - .withLabels(StandaloneKubernetesUtils.getTaskManagerSelectors(clusterId)) - .list(); - } - @VisibleForTesting protected FlinkStandaloneKubeClient createNamespacedKubeClient(Configuration configuration) { final int poolSize = diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/SuspendMode.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/SuspendMode.java new file mode 100644 index 0000000000..e29ec9b3f5 --- /dev/null +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/service/SuspendMode.java @@ -0,0 +1,38 @@ +/* + * 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.service; + +import lombok.RequiredArgsConstructor; + +/** Suspend mode. */ +@RequiredArgsConstructor +public enum SuspendMode { + NOOP, + STATELESS, + SAVEPOINT, + CANCEL, + LAST_STATE; + + public boolean deleteCluster() { + return this == STATELESS || this == LAST_STATE; + } + + public boolean deleteHaMeta() { + return this != LAST_STATE; + } +} diff --git a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/DefaultValidator.java b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/DefaultValidator.java index b326e96b4b..fad4553d6f 100644 --- a/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/DefaultValidator.java +++ b/flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/validation/DefaultValidator.java @@ -31,7 +31,6 @@ import org.apache.flink.kubernetes.operator.api.FlinkSessionJob; import org.apache.flink.kubernetes.operator.api.FlinkStateSnapshot; import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; -import org.apache.flink.kubernetes.operator.api.spec.FlinkSessionJobSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; import org.apache.flink.kubernetes.operator.api.spec.IngressSpec; import org.apache.flink.kubernetes.operator.api.spec.JobManagerSpec; @@ -42,12 +41,10 @@ import org.apache.flink.kubernetes.operator.api.spec.TaskManagerSpec; import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.api.status.FlinkStateSnapshotStatus; -import org.apache.flink.kubernetes.operator.api.status.JobManagerDeploymentStatus; import org.apache.flink.kubernetes.operator.config.FlinkConfigBuilder; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.exception.ReconciliationException; -import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.utils.FlinkStateSnapshotUtils; import org.apache.flink.kubernetes.operator.utils.IngressUtils; import org.apache.flink.kubernetes.utils.Constants; @@ -242,10 +239,6 @@ private Optional validateJobSpec( } Configuration configuration = Configuration.fromMap(confMap); - if (job.getUpgradeMode() == UpgradeMode.LAST_STATE - && !HighAvailabilityMode.isHighAvailabilityModeActivated(configuration)) { - return Optional.of("Job could not be upgraded with last-state while HA disabled"); - } if (job.getUpgradeMode() != UpgradeMode.STATELESS) { if (StringUtils.isNullOrWhitespaceOnly( @@ -453,18 +446,6 @@ private Optional validateSpecChange( JobSpec oldJob = oldSpec.getJob(); JobSpec newJob = newSpec.getJob(); if (oldJob != null && newJob != null) { - if (StringUtils.isNullOrWhitespaceOnly( - effectiveConfig.get(CheckpointingOptions.SAVEPOINT_DIRECTORY.key())) - && deployment.getStatus().getJobManagerDeploymentStatus() - != JobManagerDeploymentStatus.MISSING - && ReconciliationUtils.isUpgradeModeChangedToLastStateAndHADisabledPreviously( - deployment, configManager.getObserveConfig(deployment))) { - return Optional.of( - String.format( - "Job could not be upgraded to last-state while config key[%s] is not set", - CheckpointingOptions.SAVEPOINT_DIRECTORY.key())); - } - if (newJob.getSavepointRedeployNonce() != null && !newJob.getSavepointRedeployNonce() .equals(oldJob.getSavepointRedeployNonce())) { @@ -533,7 +514,6 @@ private Optional validateSessionJobOnly(FlinkSessionJob sessionJob) { return firstPresent( validateDeploymentName(sessionJob.getSpec().getDeploymentName()), validateJobNotEmpty(sessionJob), - validateNotLastStateUpgradeMode(sessionJob), validateSpecChange(sessionJob)); } @@ -586,19 +566,7 @@ private Optional validateSessionClusterId( } } - private Optional validateNotLastStateUpgradeMode(FlinkSessionJob sessionJob) { - if (sessionJob.getSpec().getJob().getUpgradeMode() == UpgradeMode.LAST_STATE) { - return Optional.of( - String.format( - "The %s upgrade mode is not supported in session job now.", - UpgradeMode.LAST_STATE)); - } - return Optional.empty(); - } - private Optional validateSpecChange(FlinkSessionJob sessionJob) { - FlinkSessionJobSpec newSpec = sessionJob.getSpec(); - if (sessionJob.getStatus().getReconciliationStatus().isBeforeFirstDeployment()) { return Optional.empty(); } else { diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestingFlinkService.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestingFlinkService.java index 73c65a074c..7aef5ce583 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestingFlinkService.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/TestingFlinkService.java @@ -17,6 +17,7 @@ package org.apache.flink.kubernetes.operator; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.java.tuple.Tuple2; @@ -35,7 +36,6 @@ import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; import org.apache.flink.kubernetes.operator.api.spec.JobSpec; import org.apache.flink.kubernetes.operator.api.spec.KubernetesDeploymentMode; -import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.api.status.Savepoint; import org.apache.flink.kubernetes.operator.api.status.SavepointFormatType; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; @@ -43,12 +43,13 @@ import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext; -import org.apache.flink.kubernetes.operator.exception.RecoveryFailureException; +import org.apache.flink.kubernetes.operator.exception.UpgradeFailureException; import org.apache.flink.kubernetes.operator.observer.CheckpointFetchResult; import org.apache.flink.kubernetes.operator.observer.CheckpointStatsResult; import org.apache.flink.kubernetes.operator.observer.SavepointFetchResult; import org.apache.flink.kubernetes.operator.service.AbstractFlinkService; import org.apache.flink.kubernetes.operator.service.CheckpointHistoryWrapper; +import org.apache.flink.kubernetes.operator.service.SuspendMode; import org.apache.flink.kubernetes.operator.standalone.StandaloneKubernetesConfigOptionsInternal; import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.execution.ExecutionState; @@ -232,7 +233,7 @@ protected void deployApplicationCluster(JobSpec jobSpec, Configuration conf) thr protected void validateHaMetadataExists(Configuration conf) { if (!isHaMetadataAvailable(conf)) { - throw new RecoveryFailureException( + throw new UpgradeFailureException( "HA metadata not available to restore from last state. " + "It is possible that the job has finished or terminally failed, or the configmaps have been deleted. " + "Manual restore required.", @@ -403,6 +404,7 @@ public CheckpointStatsResult fetchCheckpointStats( @Override public RestClusterClient getClusterClient(Configuration config) throws Exception { + TestingClusterClient clusterClient = new TestingClusterClient<>(config); FlinkVersion flinkVersion = config.get(FlinkConfigBuilder.FLINK_VERSION); clusterClient.setListJobsFunction( @@ -421,8 +423,7 @@ public RestClusterClient getClusterClient(Configuration config) throws E clusterClient.setStopWithSavepointFunction( (jobID, advanceEventTime, savepointDir) -> { try { - return CompletableFuture.completedFuture( - cancelJob(flinkVersion, jobID, true)); + return CompletableFuture.completedFuture(cancelJob(jobID, true)); } catch (Exception e) { return CompletableFuture.failedFuture(e); } @@ -431,7 +432,7 @@ public RestClusterClient getClusterClient(Configuration config) throws E clusterClient.setCancelFunction( jobID -> { try { - cancelJob(flinkVersion, jobID, false); + cancelJob(jobID, false); } catch (Exception e) { return CompletableFuture.failedFuture(e); } @@ -486,14 +487,14 @@ private static JobDetails toJobDetails(JobStatusMessage jobStatus) { } @Override - public Optional cancelJob( - FlinkDeployment deployment, UpgradeMode upgradeMode, Configuration configuration) + public CancelResult cancelJob( + FlinkDeployment deployment, SuspendMode upgradeMode, Configuration configuration) throws Exception { return cancelJob(deployment, upgradeMode, configuration, false); } - private String cancelJob(FlinkVersion flinkVersion, JobID jobID, boolean savepoint) - throws Exception { + @VisibleForTesting + public String cancelJob(JobID jobID, boolean savepoint) throws Exception { cancelJobCallCount++; if (!isPortReady) { @@ -537,7 +538,7 @@ private String cancelJob(FlinkVersion flinkVersion, JobID jobID, boolean savepoi new JobStatusMessage( oldStatus.getJobId(), oldStatus.getJobName(), - JobStatus.FINISHED, + savepoint ? JobStatus.FINISHED : JobStatus.CANCELED, oldStatus.getStartTime()); jobOpt.get().f0 = sp; @@ -572,7 +573,7 @@ public void disposeSavepoint(String savepointPath, Configuration conf) throws Ex } @Override - public Optional getLastCheckpoint(JobID jobId, Configuration conf) throws Exception { + public Optional getLastCheckpoint(JobID jobId, Configuration conf) { jobs.stream() .filter(js -> js.f1.getJobId().equals(jobId)) .findAny() @@ -631,11 +632,6 @@ protected PodList getJmPodList(String namespace, String clusterId) { return podList; } - @Override - protected PodList getTmPodList(String namespace, String clusterId) { - return new PodList(); - } - public void markApplicationJobFailedWithError(JobID jobID, String error) throws Exception { var job = jobs.stream().filter(tuple -> tuple.f1.getJobId().equals(jobID)).findFirst(); if (job.isEmpty()) { diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java index 4f7d9351fa..4ba1367d7b 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigBuilderTest.java @@ -23,7 +23,6 @@ import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.configuration.DeploymentOptionsInternal; -import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.PipelineOptions; @@ -31,7 +30,6 @@ import org.apache.flink.configuration.WebOptions; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.configuration.KubernetesDeploymentTarget; -import org.apache.flink.kubernetes.highavailability.KubernetesHaServicesFactory; import org.apache.flink.kubernetes.operator.TestUtils; import org.apache.flink.kubernetes.operator.api.CrdConstants; import org.apache.flink.kubernetes.operator.api.FlinkDeployment; @@ -56,8 +54,6 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.MethodSource; import java.io.File; import java.io.IOException; @@ -126,8 +122,6 @@ public void testApplyFlinkConfiguration() { KubernetesConfigOptions.ServiceExposedType.ClusterIP, configuration.get(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE)); assertEquals(false, configuration.get(WebOptions.CANCEL_ENABLE)); - assertEquals( - flinkDeployment.getMetadata().getName(), configuration.get(PipelineOptions.NAME)); FlinkDeployment deployment = ReconciliationUtils.clone(flinkDeployment); deployment @@ -145,21 +139,6 @@ public void testApplyFlinkConfiguration() { KubernetesConfigOptions.ServiceExposedType.LoadBalancer, configuration.get(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE)); - deployment.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); - configuration = - new FlinkConfigBuilder( - deployment, - new Configuration() - .set( - HighAvailabilityOptions.HA_MODE, - KubernetesHaServicesFactory.class - .getCanonicalName())) - .applyFlinkConfiguration() - .build(); - assertEquals( - DEFAULT_CHECKPOINTING_INTERVAL, - configuration.get(ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL)); - deployment = TestUtils.buildSessionCluster(); configuration = new FlinkConfigBuilder(deployment, new Configuration()) @@ -168,18 +147,6 @@ public void testApplyFlinkConfiguration() { assertEquals(false, configuration.get(WebOptions.CANCEL_ENABLE)); } - @ParameterizedTest - @MethodSource("org.apache.flink.kubernetes.operator.TestUtils#flinkVersions") - public void testApplyFlinkConfigurationShouldSetShutdownOnFinishBasedOnFlinkVersion( - FlinkVersion flinkVersion) { - flinkDeployment.getSpec().setFlinkVersion(flinkVersion); - Configuration configuration = - new FlinkConfigBuilder(flinkDeployment, new Configuration()) - .applyFlinkConfiguration() - .build(); - Assertions.assertFalse(configuration.getBoolean(SHUTDOWN_ON_APPLICATION_FINISH)); - } - @Test public void testApplyLogConfiguration() throws IOException { Configuration configuration = @@ -767,6 +734,22 @@ public void testApplyJobOrSessionSpec() throws Exception { .build(); assertEquals(12, configuration.get(CoreOptions.DEFAULT_PARALLELISM)); + assertEquals( + true, configuration.get(DeploymentOptions.SUBMIT_FAILED_JOB_ON_APPLICATION_ERROR)); + Assertions.assertFalse(configuration.getBoolean(SHUTDOWN_ON_APPLICATION_FINISH)); + assertEquals( + flinkDeployment.getMetadata().getName(), configuration.get(PipelineOptions.NAME)); + + dep = ReconciliationUtils.clone(deploymentClone); + dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); + configuration = + new FlinkConfigBuilder(dep, new Configuration()) + .applyFlinkConfiguration() + .applyJobOrSessionSpec() + .build(); + assertEquals( + DEFAULT_CHECKPOINTING_INTERVAL, + configuration.get(ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL)); } @Test diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java index ba6af301fb..5b3578f386 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/FlinkSessionJobControllerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.kubernetes.operator.controller; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.kubernetes.operator.TestUtils; import org.apache.flink.kubernetes.operator.TestingFlinkService; @@ -30,6 +31,8 @@ import org.apache.flink.kubernetes.operator.api.status.ReconciliationState; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; +import org.apache.flink.kubernetes.operator.observer.JobStatusObserver; +import org.apache.flink.kubernetes.operator.service.CheckpointHistoryWrapper; import org.apache.flink.kubernetes.operator.utils.EventRecorder; import org.apache.flink.runtime.client.JobStatusMessage; @@ -250,6 +253,115 @@ public void verifyUpgradeFromSavepointLegacy() throws Exception { assertEquals(0, jobs.size()); } + @Test + public void verifyLastStateUpgrade() throws Exception { + sessionJob.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); + testController.reconcile(sessionJob, context); + + // Simulate completed checkpoints + flinkService.setCheckpointInfo( + Tuple2.of( + Optional.of( + new CheckpointHistoryWrapper.CompletedCheckpointInfo( + 0, "cp1", System.currentTimeMillis())), + Optional.empty())); + + // Trigger Update + sessionJob.getSpec().setRestartNonce(3L); + testController.reconcile(sessionJob, context); + + // Make sure we are cancelling + assertEquals("CANCELLING", sessionJob.getStatus().getJobStatus().getState()); + + // Once cancelling completed make sure that last reconciled spec is correctly upgraded and + // job was started from cp + testController.reconcile(sessionJob, context); + assertEquals("cp1", sessionJob.getStatus().getJobStatus().getUpgradeSavepointPath()); + assertEquals( + UpgradeMode.SAVEPOINT, + sessionJob + .getStatus() + .getReconciliationStatus() + .deserializeLastReconciledSpec() + .getJob() + .getUpgradeMode()); + assertEquals("RECONCILING", sessionJob.getStatus().getJobStatus().getState()); + assertEquals( + ReconciliationState.DEPLOYED, + sessionJob.getStatus().getReconciliationStatus().getState()); + + flinkService.clearJobsInTerminalState(); + var jobs = flinkService.listJobs(); + assertEquals(1, jobs.size()); + assertEquals("cp1", jobs.get(0).f0); + + testController.reconcile(sessionJob, context); + assertEquals("RUNNING", sessionJob.getStatus().getJobStatus().getState()); + + // Suspend job + flinkService.setCheckpointInfo( + Tuple2.of( + Optional.of( + new CheckpointHistoryWrapper.CompletedCheckpointInfo( + 0, "cp2", System.currentTimeMillis())), + Optional.empty())); + sessionJob.getSpec().getJob().setState(JobState.SUSPENDED); + testController.reconcile(sessionJob, context); + testController.reconcile(sessionJob, context); + assertEquals("cp2", sessionJob.getStatus().getJobStatus().getUpgradeSavepointPath()); + } + + @Test + public void verifyLastStateUpgradeFailure() throws Exception { + sessionJob.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); + testController.reconcile(sessionJob, context); + + // Simulate completed checkpoints + flinkService.setCheckpointInfo( + Tuple2.of( + Optional.of( + new CheckpointHistoryWrapper.CompletedCheckpointInfo( + 0, "cp1", System.currentTimeMillis())), + Optional.empty())); + + // Trigger Update + sessionJob.getSpec().setRestartNonce(3L); + testController.events().clear(); + testController.reconcile(sessionJob, context); + + // Make sure we are cancelling + assertEquals("CANCELLING", sessionJob.getStatus().getJobStatus().getState()); + testController.events().poll(); + assertEquals( + testController.events().poll().getReason(), + EventRecorder.Reason.SpecChanged.name()); + testController.events().clear(); + + // Remove all jobs to trigger not found error + flinkService.clear(); + + testController.reconcile(sessionJob, context); + assertEquals(JobStatusObserver.JOB_NOT_FOUND_ERR, sessionJob.getStatus().getError()); + assertEquals("RECONCILING", sessionJob.getStatus().getJobStatus().getState()); + assertEquals( + ReconciliationState.DEPLOYED, + sessionJob.getStatus().getReconciliationStatus().getState()); + + testController.events().clear(); + testController.reconcile(sessionJob, context); + assertEquals( + testController.events().poll().getReason(), EventRecorder.Reason.Missing.name()); + assertTrue(testController.events().isEmpty()); + testController.reconcile(sessionJob, context); + assertEquals( + testController.events().poll().getReason(), EventRecorder.Reason.Missing.name()); + assertTrue(testController.events().isEmpty()); + + // Deletion should still work + var deleteControl = testController.cleanup(sessionJob, context); + assertTrue(deleteControl.isRemoveFinalizer()); + } + @Test public void verifyStatelessUpgrade() throws Exception { UpdateControl updateControl; @@ -275,7 +387,6 @@ public void verifyStatelessUpgrade() throws Exception { // Upgrade job sessionJob.getSpec().getJob().setParallelism(100); updateControl = testController.reconcile(sessionJob, context); - assertEquals(2, testController.events().size()); assertEquals( EventRecorder.Reason.SpecChanged, @@ -284,9 +395,11 @@ public void verifyStatelessUpgrade() throws Exception { EventRecorder.Reason.Suspended, EventRecorder.Reason.valueOf(testController.events().poll().getReason())); - assertEquals(0, updateControl.getScheduleDelay().get()); assertEquals( - JobState.SUSPENDED, + configManager.getOperatorConfiguration().getProgressCheckInterval().toMillis(), + updateControl.getScheduleDelay().get()); + assertEquals( + JobState.RUNNING, sessionJob .getStatus() .getReconciliationStatus() @@ -294,9 +407,8 @@ public void verifyStatelessUpgrade() throws Exception { .getJob() .getState()); - flinkService.clearJobsInTerminalState(); - updateControl = testController.reconcile(sessionJob, context); + flinkService.clearJobsInTerminalState(); assertEquals( Optional.of( @@ -308,7 +420,10 @@ public void verifyStatelessUpgrade() throws Exception { assertEquals(1, jobs.size()); assertNull(jobs.get(0).f0); - assertEquals(2, testController.events().size()); + assertEquals(3, testController.events().size()); + assertEquals( + EventRecorder.Reason.JobStatusChanged, + EventRecorder.Reason.valueOf(testController.events().poll().getReason())); assertEquals( EventRecorder.Reason.Submit, EventRecorder.Reason.valueOf(testController.events().poll().getReason())); @@ -327,6 +442,8 @@ public void verifyStatelessUpgrade() throws Exception { assertEquals( EventRecorder.Reason.Suspended, EventRecorder.Reason.valueOf(testController.events().poll().getReason())); + testController.reconcile(sessionJob, context); + testController.events().clear(); // Resume from empty state sessionJob.getSpec().getJob().setState(JobState.RUNNING); @@ -358,31 +475,26 @@ public void verifyStatelessUpgrade() throws Exception { assertEquals( EventRecorder.Reason.Suspended, EventRecorder.Reason.valueOf(testController.events().poll().getReason())); - assertEquals( - JobState.SUSPENDED, - sessionJob - .getStatus() - .getReconciliationStatus() - .deserializeLastReconciledSpec() - .getJob() - .getState()); sessionJob.getSpec().getJob().setParallelism(-1); testController.reconcile(sessionJob, context); flinkService.clearJobsInTerminalState(); - assertEquals(2, testController.events().size()); + assertEquals(3, testController.events().size()); testController.reconcile(sessionJob, context); var statusEvents = testController.events().stream() .filter(e -> !e.getReason().equals(ValidationError.name())) .collect(Collectors.toList()); - assertEquals(2, statusEvents.size()); + assertEquals(3, statusEvents.size()); assertEquals( - EventRecorder.Reason.Submit, + EventRecorder.Reason.JobStatusChanged, EventRecorder.Reason.valueOf(statusEvents.get(0).getReason())); assertEquals( - EventRecorder.Reason.JobStatusChanged, + EventRecorder.Reason.Submit, EventRecorder.Reason.valueOf(statusEvents.get(1).getReason())); + assertEquals( + EventRecorder.Reason.JobStatusChanged, + EventRecorder.Reason.valueOf(statusEvents.get(2).getReason())); assertEquals(JobStatus.RUNNING.name(), sessionJob.getStatus().getJobStatus().getState()); assertEquals( @@ -531,6 +643,24 @@ public void testUnsupportedVersions(FlinkVersion version) throws Exception { } } + @Test + public void testCancelJobNotFound() throws Exception { + testController.reconcile(sessionJob, context); + + var deleteControl = testController.cleanup(sessionJob, context); + + assertEquals("CANCELLING", sessionJob.getStatus().getJobStatus().getState()); + assertFalse(deleteControl.isRemoveFinalizer()); + assertEquals( + configManager.getOperatorConfiguration().getProgressCheckInterval().toMillis(), + deleteControl.getScheduleDelay().get()); + + flinkService.clear(); + flinkService.setFlinkJobNotFound(true); + deleteControl = testController.cleanup(sessionJob, context); + assertTrue(deleteControl.isRemoveFinalizer()); + } + private void verifyReconcileInitialSuspendedDeployment(FlinkSessionJob sessionJob) throws Exception { UpdateControl updateControl = diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java index b5518f9148..753a241e60 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/controller/RollbackTest.java @@ -159,7 +159,7 @@ public void testSavepointRollbackWithoutHaMetadata() throws Exception { dep.getSpec().setRestartNonce(10L); testController.reconcile(dep, context); }, - true); + false); } @Test @@ -265,7 +265,7 @@ public void testRollbackFailureWithLastState() throws Exception { testController.reconcile(dep, context); flinkService.setPortReady(true); }, - false); + true); } @Test @@ -352,7 +352,7 @@ public void testRollback( FlinkDeployment deployment, ThrowingRunnable triggerRollback, ThrowingRunnable validateAndRecover, - boolean injectValidationError) + boolean expectTwoStepRollback) throws Exception { var flinkConfiguration = deployment.getSpec().getFlinkConfiguration(); @@ -376,12 +376,12 @@ public void testRollback( assertFalse(deployment.getStatus().getReconciliationStatus().isLastReconciledSpecStable()); assertEquals( - deployment.getSpec().getJob() != null + expectTwoStepRollback ? ReconciliationState.ROLLING_BACK : ReconciliationState.ROLLED_BACK, deployment.getStatus().getReconciliationStatus().getState()); - if (injectValidationError) { + if (expectTwoStepRollback) { deployment.getSpec().setLogConfiguration(Map.of("invalid", "entry")); } flinkService.setJobManagerReady(true); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java new file mode 100644 index 0000000000..f5b388871b --- /dev/null +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/JobStatusObserverTest.java @@ -0,0 +1,161 @@ +/* + * 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.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.configuration.PipelineOptionsInternal; +import org.apache.flink.kubernetes.operator.OperatorTestBase; +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.FlinkSessionJob; +import org.apache.flink.kubernetes.operator.api.spec.JobState; +import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; +import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext; +import org.apache.flink.kubernetes.operator.utils.EventRecorder; + +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; +import lombok.Getter; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.ArrayList; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** Tests for the {@link JobStatusObserver}. */ +@EnableKubernetesMockClient(crud = true) +public class JobStatusObserverTest extends OperatorTestBase { + + @Getter private KubernetesClient kubernetesClient; + private JobStatusObserver> observer; + + @Override + protected void setup() { + observer = new JobStatusObserver<>(eventRecorder); + } + + @ParameterizedTest + @MethodSource("cancellingArgs") + void testCancellingToMissing( + JobStatus fromStatus, UpgradeMode upgradeMode, JobState expectedAfter) { + var job = initSessionJob(); + job.getSpec().getJob().setUpgradeMode(upgradeMode); + var status = job.getStatus(); + var jobStatus = status.getJobStatus(); + jobStatus.setState(fromStatus.name()); + assertEquals( + JobState.RUNNING, + status.getReconciliationStatus() + .deserializeLastReconciledSpec() + .getJob() + .getState()); + observer.observe( + getResourceContext( + job, TestUtils.createContextWithReadyFlinkDeployment(kubernetesClient))); + assertEquals( + JobStatusObserver.JOB_NOT_FOUND_ERR, + flinkResourceEventCollector.events.poll().getMessage()); + assertEquals( + expectedAfter, + status.getReconciliationStatus() + .deserializeLastReconciledSpec() + .getJob() + .getState()); + } + + @ParameterizedTest + @EnumSource(value = JobStatus.class, mode = EnumSource.Mode.EXCLUDE, names = "CANCELED") + void testCancellingToTerminal(JobStatus fromStatus) throws Exception { + var observer = new JobStatusObserver<>(eventRecorder); + var deployment = initDeployment(); + var status = deployment.getStatus(); + var jobStatus = status.getJobStatus(); + jobStatus.setState(fromStatus.name()); + assertEquals( + JobState.RUNNING, + status.getReconciliationStatus() + .deserializeLastReconciledSpec() + .getJob() + .getState()); + FlinkResourceContext> ctx = getResourceContext(deployment); + flinkService.submitApplicationCluster( + deployment.getSpec().getJob(), ctx.getDeployConfig(deployment.getSpec()), false); + flinkService.cancelJob(JobID.fromHexString(jobStatus.getJobId()), false); + observer.observe(ctx); + assertEquals( + EventRecorder.Reason.JobStatusChanged.name(), + flinkResourceEventCollector.events.poll().getReason()); + assertEquals( + JobState.SUSPENDED, + status.getReconciliationStatus() + .deserializeLastReconciledSpec() + .getJob() + .getState()); + } + + private static Stream cancellingArgs() { + var args = new ArrayList(); + for (var status : JobStatus.values()) { + for (var upgradeMode : UpgradeMode.values()) { + args.add( + Arguments.of( + status, + upgradeMode, + upgradeMode == UpgradeMode.STATELESS + && !status.isGloballyTerminalState() + ? JobState.SUSPENDED + : JobState.RUNNING)); + } + } + return args.stream(); + } + + private static FlinkDeployment initDeployment() { + FlinkDeployment deployment = TestUtils.buildApplicationCluster(); + var jobId = new JobID().toHexString(); + deployment + .getSpec() + .getFlinkConfiguration() + .put(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); + deployment.getStatus().getJobStatus().setJobId(jobId); + deployment + .getStatus() + .getReconciliationStatus() + .serializeAndSetLastReconciledSpec(deployment.getSpec(), deployment); + return deployment; + } + + private static FlinkSessionJob initSessionJob() { + var job = TestUtils.buildSessionJob(); + var jobId = new JobID().toHexString(); + job.getSpec() + .getFlinkConfiguration() + .put(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID.key(), jobId); + job.getStatus().getJobStatus().setJobId(jobId); + job.getStatus() + .getReconciliationStatus() + .serializeAndSetLastReconciledSpec(job.getSpec(), job); + return job; + } +} diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java index 9f79c407fc..2f06504b45 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/deployment/ApplicationObserverTest.java @@ -19,7 +19,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.configuration.PipelineOptionsInternal; import org.apache.flink.core.execution.CheckpointType; import org.apache.flink.kubernetes.operator.OperatorTestBase; @@ -41,7 +40,6 @@ import org.apache.flink.kubernetes.operator.utils.FlinkUtils; import org.apache.flink.kubernetes.operator.utils.SnapshotUtils; import org.apache.flink.runtime.client.JobStatusMessage; -import org.apache.flink.runtime.jobgraph.JobVertexID; import io.fabric8.kubernetes.api.model.Event; import io.fabric8.kubernetes.client.KubernetesClient; @@ -51,7 +49,6 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.time.Clock; import java.time.Duration; import java.time.Instant; import java.util.HashMap; @@ -859,35 +856,6 @@ public void observeAlreadyUpgraded() { assertEquals(5, specWithMeta.getSpec().getJob().getParallelism()); } - @Test - public void observeAlreadyScaled() { - // Update status for for running job - ReconciliationUtils.updateStatusBeforeDeploymentAttempt( - deployment, - new FlinkConfigManager(new Configuration()) - .getDeployConfig(deployment.getMetadata(), deployment.getSpec())); - ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); - assertEquals( - ReconciliationState.DEPLOYED, - deployment.getStatus().getReconciliationStatus().getState()); - - var conf = new Configuration(); - var v1 = new JobVertexID(); - conf.set(PipelineOptions.PARALLELISM_OVERRIDES, Map.of(v1.toHexString(), "2")); - deployment.getSpec().setFlinkConfiguration(conf.toMap()); - - // Assert that we move to deployed when in deprecated scaling UPGRADING state - ReconciliationUtils.updateStatusForSpecReconciliation( - deployment, JobState.RUNNING, conf, true, Clock.systemDefaultZone()); - assertEquals( - ReconciliationState.UPGRADING, - deployment.getStatus().getReconciliationStatus().getState()); - observer.observe(deployment, context); - assertEquals( - ReconciliationState.DEPLOYED, - deployment.getStatus().getReconciliationStatus().getState()); - } - @Test public void validateLastReconciledClearedOnInitialFailure() { deployment.getMetadata().setGeneration(123L); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/sessionjob/FlinkSessionJobObserverTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/sessionjob/FlinkSessionJobObserverTest.java index 62b05366c0..1b6bdb0b53 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/sessionjob/FlinkSessionJobObserverTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/observer/sessionjob/FlinkSessionJobObserverTest.java @@ -27,6 +27,7 @@ import org.apache.flink.kubernetes.operator.TestUtils; import org.apache.flink.kubernetes.operator.api.FlinkSessionJob; import org.apache.flink.kubernetes.operator.api.spec.FlinkSessionJobSpec; +import org.apache.flink.kubernetes.operator.api.spec.JobState; import org.apache.flink.kubernetes.operator.api.status.FlinkSessionJobStatus; import org.apache.flink.kubernetes.operator.api.status.ReconciliationState; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; @@ -114,7 +115,19 @@ public void testBasicObserve() throws Exception { observer.observe(sessionJob, readyContext); Assertions.assertEquals( JobStatus.RECONCILING.name(), sessionJob.getStatus().getJobStatus().getState()); + Assertions.assertEquals( + JobState.SUSPENDED, + sessionJob + .getStatus() + .getReconciliationStatus() + .deserializeLastReconciledSpec() + .getJob() + .getState()); + + // reset sessionJob.getStatus().getJobStatus().setJobId(jobID); + ReconciliationUtils.updateLastReconciledSpec( + sessionJob, (s, m) -> s.getJob().setState(JobState.RUNNING)); // testing multi job diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java index 6a5f60ad90..e00305d3cb 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerTest.java @@ -60,13 +60,14 @@ import org.apache.flink.kubernetes.operator.autoscaler.KubernetesJobAutoScalerContext; import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; -import org.apache.flink.kubernetes.operator.exception.RecoveryFailureException; +import org.apache.flink.kubernetes.operator.exception.UpgradeFailureException; import org.apache.flink.kubernetes.operator.health.ClusterHealthInfo; import org.apache.flink.kubernetes.operator.observer.ClusterHealthEvaluator; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.reconciler.SnapshotType; import org.apache.flink.kubernetes.operator.reconciler.TestReconcilerAdapter; import org.apache.flink.kubernetes.operator.service.NativeFlinkService; +import org.apache.flink.kubernetes.operator.service.SuspendMode; import org.apache.flink.kubernetes.operator.utils.EventRecorder; import org.apache.flink.kubernetes.operator.utils.SnapshotStatus; import org.apache.flink.kubernetes.operator.utils.SnapshotUtils; @@ -328,14 +329,14 @@ public void testUpgrade(FlinkVersion flinkVersion) throws Exception { .setJobManagerDeploymentStatus(JobManagerDeploymentStatus.MISSING); reconciler.reconcile(deployment, context); fail(); - } catch (RecoveryFailureException expected) { + } catch (UpgradeFailureException expected) { } try { deployment.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.ERROR); reconciler.reconcile(deployment, context); fail(); - } catch (RecoveryFailureException expected) { + } catch (UpgradeFailureException expected) { } flinkService.clear(); @@ -864,11 +865,11 @@ protected void updateVertexResources( } @Override - public Optional cancelJob( + public CancelResult cancelJob( FlinkDeployment deployment, - UpgradeMode upgradeMode, + SuspendMode upgradeMode, Configuration conf) { - return Optional.empty(); + return CancelResult.completed(null); } }; @@ -944,7 +945,6 @@ public void scale(KubernetesJobAutoScalerContext ctx) { var deployment = TestUtils.buildApplicationCluster(); appReconciler.reconcile(ctxFactory.getResourceContext(deployment, context)); verifyAndSetRunningJobsToStatus(deployment, flinkService.listJobs()); - assertFalse(deployment.getStatus().isImmediateReconciliationNeeded()); // Job running verify no upgrades if overrides are empty appReconciler.reconcile(ctxFactory.getResourceContext(deployment, context)); @@ -1280,10 +1280,8 @@ public void testRollbackUpgradeModeHandling(boolean jmStarted) throws Exception reconciler.reconcile(deployment, context); assertEquals( - ReconciliationState.ROLLING_BACK, + jmStarted ? ReconciliationState.ROLLING_BACK : ReconciliationState.ROLLED_BACK, deployment.getStatus().getReconciliationStatus().getState()); - assertEquals(0, flinkService.listJobs().size()); - assertEquals("FINISHED", deployment.getStatus().getJobStatus().getState()); assertEquals( jmStarted ? UpgradeMode.LAST_STATE : UpgradeMode.SAVEPOINT, deployment diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java index 465c3cf61d..93a25e8297 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconcilerUpgradeModeTest.java @@ -40,7 +40,7 @@ import org.apache.flink.kubernetes.operator.api.status.SavepointFormatType; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; -import org.apache.flink.kubernetes.operator.exception.RecoveryFailureException; +import org.apache.flink.kubernetes.operator.exception.UpgradeFailureException; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.reconciler.TestReconcilerAdapter; import org.apache.flink.kubernetes.operator.service.CheckpointHistoryWrapper; @@ -213,7 +213,7 @@ private void testUpgradeToLastState(FlinkVersion flinkVersion, UpgradeMode fromU deployment.getStatus().getJobStatus().setState("RECONCILING"); Assertions.assertThrows( - RecoveryFailureException.class, + UpgradeFailureException.class, () -> { deployment .getStatus() @@ -223,7 +223,7 @@ private void testUpgradeToLastState(FlinkVersion flinkVersion, UpgradeMode fromU }); Assertions.assertThrows( - RecoveryFailureException.class, + UpgradeFailureException.class, () -> { deployment .getStatus() @@ -367,9 +367,13 @@ public void testUpgradeJmDeployCannotStart(UpgradeMode fromMode, UpgradeMode toM assertEquals(JobState.RUNNING, lastReconciledSpec.getJob().getState()); } else { assertEquals( - JobManagerDeploymentStatus.MISSING, + toMode == UpgradeMode.STATELESS + ? JobManagerDeploymentStatus.MISSING + : JobManagerDeploymentStatus.DEPLOYING, deployment.getStatus().getJobManagerDeploymentStatus()); - assertEquals(JobState.SUSPENDED, lastReconciledSpec.getJob().getState()); + assertEquals( + toMode == UpgradeMode.STATELESS ? JobState.SUSPENDED : JobState.RUNNING, + lastReconciledSpec.getJob().getState()); assertEquals( toMode == UpgradeMode.STATELESS ? UpgradeMode.STATELESS : UpgradeMode.SAVEPOINT, lastReconciledSpec.getJob().getUpgradeMode()); @@ -392,6 +396,16 @@ public void testUpgradeJmDeployCannotStart(UpgradeMode fromMode, UpgradeMode toM } } + private static Stream testInitialJmDeployCannotStartParams() { + return Stream.of( + Arguments.of(UpgradeMode.LAST_STATE, true), + Arguments.of(UpgradeMode.LAST_STATE, false), + Arguments.of(UpgradeMode.SAVEPOINT, true), + Arguments.of(UpgradeMode.SAVEPOINT, false), + Arguments.of(UpgradeMode.STATELESS, true), + Arguments.of(UpgradeMode.STATELESS, false)); + } + @ParameterizedTest @MethodSource("testInitialJmDeployCannotStartParams") public void testInitialJmDeployCannotStartLegacy(UpgradeMode upgradeMode, boolean initSavepoint) @@ -437,7 +451,9 @@ public void testInitialJmDeployCannotStartLegacy(UpgradeMode upgradeMode, boolea deployment.getSpec().setImage(newImage); reconciler.reconcile(deployment, context); assertEquals( - ReconciliationState.UPGRADING, + upgradeMode == UpgradeMode.STATELESS + ? ReconciliationState.UPGRADING + : ReconciliationState.DEPLOYED, deployment.getStatus().getReconciliationStatus().getState()); lastReconciledSpec = deployment.getStatus().getReconciliationStatus().deserializeLastReconciledSpec(); @@ -464,12 +480,25 @@ public void testInitialJmDeployCannotStartLegacy(UpgradeMode upgradeMode, boolea flinkService.listJobs().get(0).f0); } - @Test - public void testLastStateMaxCheckpointAge() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testLastStateMaxCheckpointAge(boolean cancellable) throws Exception { var deployment = TestUtils.buildApplicationCluster(); + deployment + .getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_CANCEL_JOB + .key(), + Boolean.toString(cancellable)); deployment.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); + var expectedWhenNoSavepoint = + cancellable + ? AbstractJobReconciler.JobUpgrade.lastStateUsingCancel() + : AbstractJobReconciler.JobUpgrade.lastStateUsingHaMeta(); + // Set job status to running var jobStatus = deployment.getStatus().getJobStatus(); long now = System.currentTimeMillis(); @@ -482,9 +511,7 @@ public void testLastStateMaxCheckpointAge() throws Exception { var ctx = getResourceContext(deployment); var deployConf = ctx.getDeployConfig(deployment.getSpec()); - assertEquals( - AbstractJobReconciler.AvailableUpgradeMode.of(UpgradeMode.LAST_STATE), - jobReconciler.getAvailableUpgradeMode(ctx, deployConf)); + assertEquals(expectedWhenNoSavepoint, jobReconciler.getJobUpgrade(ctx, deployConf)); deployConf.set( KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_CHECKPOINT_MAX_AGE, @@ -495,15 +522,13 @@ public void testLastStateMaxCheckpointAge() throws Exception { // Job started just now jobStatus.setStartTime(Long.toString(now)); - assertEquals( - AbstractJobReconciler.AvailableUpgradeMode.of(UpgradeMode.LAST_STATE), - jobReconciler.getAvailableUpgradeMode(ctx, deployConf)); + assertEquals(expectedWhenNoSavepoint, jobReconciler.getJobUpgrade(ctx, deployConf)); // Job started more than a minute ago jobStatus.setStartTime(Long.toString(now - 61000)); assertEquals( - AbstractJobReconciler.AvailableUpgradeMode.of(UpgradeMode.SAVEPOINT), - jobReconciler.getAvailableUpgradeMode(ctx, deployConf)); + AbstractJobReconciler.JobUpgrade.savepoint(false), + jobReconciler.getJobUpgrade(ctx, deployConf)); // If we have a pending savepoint within the max age, wait flinkService.setCheckpointInfo( @@ -513,8 +538,8 @@ public void testLastStateMaxCheckpointAge() throws Exception { new CheckpointHistoryWrapper.PendingCheckpointInfo( 0, now - 30000)))); assertEquals( - AbstractJobReconciler.AvailableUpgradeMode.pendingUpgrade(), - jobReconciler.getAvailableUpgradeMode(ctx, deployConf)); + AbstractJobReconciler.JobUpgrade.pendingUpgrade(), + jobReconciler.getJobUpgrade(ctx, deployConf)); // If pending savepoint triggered before max age, use savepoint flinkService.setCheckpointInfo( @@ -524,14 +549,12 @@ public void testLastStateMaxCheckpointAge() throws Exception { new CheckpointHistoryWrapper.PendingCheckpointInfo( 0, now - 61000)))); assertEquals( - AbstractJobReconciler.AvailableUpgradeMode.of(UpgradeMode.SAVEPOINT), - jobReconciler.getAvailableUpgradeMode(ctx, deployConf)); + AbstractJobReconciler.JobUpgrade.savepoint(false), + jobReconciler.getJobUpgrade(ctx, deployConf)); // Allow fallback to job start even with pending savepoint jobStatus.setStartTime(Long.toString(now - 30000)); - assertEquals( - AbstractJobReconciler.AvailableUpgradeMode.of(UpgradeMode.LAST_STATE), - jobReconciler.getAvailableUpgradeMode(ctx, deployConf)); + assertEquals(expectedWhenNoSavepoint, jobReconciler.getJobUpgrade(ctx, deployConf)); // Recent completed checkpoint jobStatus.setStartTime(Long.toString(now - 61000)); @@ -543,9 +566,7 @@ public void testLastStateMaxCheckpointAge() throws Exception { Optional.of( new CheckpointHistoryWrapper.PendingCheckpointInfo( 0, now - 61000)))); - assertEquals( - AbstractJobReconciler.AvailableUpgradeMode.of(UpgradeMode.LAST_STATE), - jobReconciler.getAvailableUpgradeMode(ctx, deployConf)); + assertEquals(expectedWhenNoSavepoint, jobReconciler.getJobUpgrade(ctx, deployConf)); // Job start and checkpoint too old, trigger savepoint jobStatus.setStartTime(Long.toString(now - 61000)); @@ -558,18 +579,132 @@ public void testLastStateMaxCheckpointAge() throws Exception { new CheckpointHistoryWrapper.PendingCheckpointInfo( 0, now - 61000)))); assertEquals( - AbstractJobReconciler.AvailableUpgradeMode.of(UpgradeMode.SAVEPOINT), - jobReconciler.getAvailableUpgradeMode(ctx, deployConf)); + AbstractJobReconciler.JobUpgrade.savepoint(false), + jobReconciler.getJobUpgrade(ctx, deployConf)); } - private static Stream testInitialJmDeployCannotStartParams() { + private static Stream testVersionUpgradeTestParams() { + return Stream.of( + Arguments.of(UpgradeMode.LAST_STATE, true, true), + Arguments.of(UpgradeMode.LAST_STATE, true, false), + Arguments.of(UpgradeMode.LAST_STATE, false, true), + Arguments.of(UpgradeMode.LAST_STATE, false, false), + Arguments.of(UpgradeMode.SAVEPOINT, true, true), + Arguments.of(UpgradeMode.SAVEPOINT, true, false)); + } + + @ParameterizedTest + @MethodSource("testVersionUpgradeTestParams") + public void testFlinkVersionSwitching( + UpgradeMode upgradeMode, boolean savepointsEnabled, boolean allowFallback) + throws Exception { + var jobReconciler = (ApplicationReconciler) this.reconciler.getReconciler(); + var deployment = TestUtils.buildApplicationCluster(FlinkVersion.v1_18); + if (!savepointsEnabled) { + deployment + .getSpec() + .getFlinkConfiguration() + .remove(CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); + } + deployment + .getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions + .OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED + .key(), + Boolean.toString(allowFallback)); + deployment.getSpec().getJob().setUpgradeMode(upgradeMode); + ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); + deployment.getSpec().setFlinkVersion(FlinkVersion.v1_19); + + // Set job status to running + var jobStatus = deployment.getStatus().getJobStatus(); + long now = System.currentTimeMillis(); + + jobStatus.setStartTime(Long.toString(now)); + jobStatus.setJobId(new JobID().toString()); + + // Running state, savepoint if possible + jobStatus.setState(org.apache.flink.api.common.JobStatus.RUNNING.name()); + var ctx = getResourceContext(deployment); + var deployConf = ctx.getDeployConfig(deployment.getSpec()); + + assertEquals( + savepointsEnabled + ? AbstractJobReconciler.JobUpgrade.savepoint(false) + : AbstractJobReconciler.JobUpgrade.lastStateUsingCancel(), + jobReconciler.getJobUpgrade(ctx, deployConf)); + + // Not running (but cancellable) + jobStatus.setState(org.apache.flink.api.common.JobStatus.RESTARTING.name()); + assertEquals( + AbstractJobReconciler.JobUpgrade.lastStateUsingCancel(), + jobReconciler.getJobUpgrade(ctx, deployConf)); + + // Unknown / reconciling + jobStatus.setState(org.apache.flink.api.common.JobStatus.RECONCILING.name()); + assertEquals( + AbstractJobReconciler.JobUpgrade.pendingUpgrade(), + jobReconciler.getJobUpgrade(ctx, deployConf)); + } + + private static Stream testLastStateCancelParams() { return Stream.of( Arguments.of(UpgradeMode.LAST_STATE, true), Arguments.of(UpgradeMode.LAST_STATE, false), Arguments.of(UpgradeMode.SAVEPOINT, true), - Arguments.of(UpgradeMode.SAVEPOINT, false), - Arguments.of(UpgradeMode.STATELESS, true), - Arguments.of(UpgradeMode.STATELESS, false)); + Arguments.of(UpgradeMode.SAVEPOINT, false)); + } + + @ParameterizedTest + @MethodSource("testLastStateCancelParams") + public void testLastStateNoHaMeta(UpgradeMode upgradeMode, boolean allowFallback) + throws Exception { + var jobReconciler = (ApplicationReconciler) this.reconciler.getReconciler(); + var deployment = TestUtils.buildApplicationCluster(); + deployment + .getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions + .OPERATOR_JOB_UPGRADE_LAST_STATE_FALLBACK_ENABLED + .key(), + Boolean.toString(allowFallback)); + deployment.getSpec().getFlinkConfiguration().remove(HighAvailabilityOptions.HA_MODE.key()); + deployment + .getSpec() + .getFlinkConfiguration() + .put( + KubernetesOperatorConfigOptions.OPERATOR_JOB_UPGRADE_LAST_STATE_CANCEL_JOB + .key(), + Boolean.toString(false)); + deployment.getSpec().getJob().setUpgradeMode(upgradeMode); + ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); + + // Set job status to running + var jobStatus = deployment.getStatus().getJobStatus(); + long now = System.currentTimeMillis(); + + jobStatus.setStartTime(Long.toString(now)); + jobStatus.setJobId(new JobID().toString()); + + // Running state, savepoint if possible + jobStatus.setState(org.apache.flink.api.common.JobStatus.FAILING.name()); + var ctx = getResourceContext(deployment); + var deployConf = ctx.getDeployConfig(deployment.getSpec()); + + if (upgradeMode == UpgradeMode.LAST_STATE) { + assertEquals( + AbstractJobReconciler.JobUpgrade.lastStateUsingCancel(), + jobReconciler.getJobUpgrade(ctx, deployConf)); + } else { + assertEquals( + allowFallback + ? AbstractJobReconciler.JobUpgrade.lastStateUsingCancel() + : AbstractJobReconciler.JobUpgrade.pendingUpgrade(), + jobReconciler.getJobUpgrade(ctx, deployConf)); + } } private static Stream testUpgradeJmDeployCannotStartParams() { @@ -681,11 +816,10 @@ public void testUpgradeModeChangeFromSavepointToLastState() throws Exception { } @Test - public void testUpgradeModeChangedToLastStateShouldTriggerSavepointWhileHADisabled() - throws Exception { + public void testUpgradeModeChangedToLastStateShouldCancelWhileHADisabled() throws Exception { flinkService.setHaDataAvailable(false); - final FlinkDeployment deployment = TestUtils.buildApplicationCluster(); + var deployment = TestUtils.buildApplicationCluster(); deployment.getSpec().getFlinkConfiguration().remove(HighAvailabilityOptions.HA_MODE.key()); reconciler.reconcile(deployment, context); @@ -694,7 +828,7 @@ public void testUpgradeModeChangedToLastStateShouldTriggerSavepointWhileHADisabl deployment.getStatus().getJobManagerDeploymentStatus()); // Not ready for spec changes, the reconciliation is not performed - final String newImage = "new-image-1"; + String newImage = "new-image-1"; deployment.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); deployment.getSpec().setImage(newImage); reconciler.reconcile(deployment, context); @@ -711,6 +845,15 @@ public void testUpgradeModeChangedToLastStateShouldTriggerSavepointWhileHADisabl // Ready for spec changes, the reconciliation should be performed verifyAndSetRunningJobsToStatus(deployment, flinkService.listJobs()); reconciler.reconcile(deployment, context); + assertEquals("CANCELLING", deployment.getStatus().getJobStatus().getState()); + + String expectedSavepointPath = "savepoint_0"; + var jobStatus = deployment.getStatus().getJobStatus(); + jobStatus.setState("CANCELED"); + jobStatus + .getSavepointInfo() + .setLastSavepoint(Savepoint.of(expectedSavepointPath, SnapshotTriggerType.UNKNOWN)); + reconciler.reconcile(deployment, context); assertEquals( newImage, @@ -720,10 +863,8 @@ public void testUpgradeModeChangedToLastStateShouldTriggerSavepointWhileHADisabl .deserializeLastReconciledSpec() .getImage()); // Upgrade mode changes from stateless to last-state should trigger a savepoint - final String expectedSavepointPath = "savepoint_0"; - var snapshots = TestUtils.getFlinkStateSnapshotsForResource(kubernetesClient, deployment); - assertThat(snapshots).isNotEmpty(); - assertEquals(expectedSavepointPath, snapshots.get(0).getSpec().getSavepoint().getPath()); + var runningJobs = flinkService.listJobs(); + assertEquals(expectedSavepointPath, runningJobs.get(0).f0); } @Test diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java index 6ab77705b6..0185b765e8 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/reconciler/sessionjob/SessionJobReconcilerTest.java @@ -36,6 +36,7 @@ import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; import org.apache.flink.kubernetes.operator.reconciler.TestReconcilerAdapter; +import org.apache.flink.kubernetes.operator.service.SuspendMode; import org.apache.flink.kubernetes.operator.utils.SnapshotUtils; import org.apache.flink.runtime.client.JobStatusMessage; @@ -204,7 +205,7 @@ public void testSubmitAndCleanUp() throws Exception { // clean up reconciler.cleanup( sessionJob, TestUtils.createContextWithReadyFlinkDeployment(kubernetesClient)); - assertEquals(FINISHED, flinkService.listJobs().get(0).f1.getJobState()); + assertEquals(CANCELED, flinkService.listJobs().get(0).f1.getJobState()); } @Test @@ -227,36 +228,15 @@ public void testCancelJobRescheduled() throws Exception { assertEquals(RUNNING, flinkService.listJobs().get(0).f1.getJobState()); flinkService.setPortReady(true); - deleteControl = - reconciler.cleanup( - sessionJob, - TestUtils.createContextWithReadyFlinkDeployment(kubernetesClient)); - assertEquals(true, deleteControl.isRemoveFinalizer()); - assertEquals(FINISHED, flinkService.listJobs().get(0).f1.getJobState()); - } - - @Test - public void testCancelJobNotFound() throws Exception { - FlinkSessionJob sessionJob = TestUtils.buildSessionJob(); - - // session ready - reconciler.reconcile( + reconciler.cleanup( sessionJob, TestUtils.createContextWithReadyFlinkDeployment(kubernetesClient)); - assertEquals(1, flinkService.listJobs().size()); - verifyAndSetRunningJobsToStatus( - sessionJob, JobState.RUNNING, RECONCILING.name(), null, flinkService.listJobs()); - // clean up - flinkService.setFlinkJobNotFound(true); - var deleteControl = - reconciler.cleanup( - sessionJob, - TestUtils.createContextWithReadyFlinkDeployment(kubernetesClient)); - + sessionJob.getStatus().getJobStatus().setState("CANCELED"); deleteControl = reconciler.cleanup( sessionJob, TestUtils.createContextWithReadyFlinkDeployment(kubernetesClient)); - assertEquals(true, deleteControl.isRemoveFinalizer()); + assertTrue(deleteControl.isRemoveFinalizer()); + assertEquals(CANCELED, flinkService.listJobs().get(0).f1.getJobState()); } @Test @@ -275,6 +255,7 @@ public void testCancelJobTerminatedWithoutCancellation() throws Exception { reconciler.cleanup( sessionJob, TestUtils.createContextWithReadyFlinkDeployment(kubernetesClient)); + sessionJob.getStatus().getJobStatus().setState("CANCELED"); deleteControl = reconciler.cleanup( @@ -296,7 +277,8 @@ public void testRestart() throws Exception { sessionJob.getSpec().setRestartNonce(2L); reconciler.reconcile( sessionJob, TestUtils.createContextWithReadyFlinkDeployment(kubernetesClient)); - assertEquals(FINISHED, flinkService.listJobs().get(0).f1.getJobState()); + assertEquals(CANCELED, flinkService.listJobs().get(0).f1.getJobState()); + sessionJob.getStatus().getJobStatus().setState("CANCELED"); reconciler.reconcile( sessionJob, TestUtils.createContextWithReadyFlinkDeployment(kubernetesClient)); verifyAndSetRunningJobsToStatus( @@ -351,8 +333,9 @@ public void testStatelessUpgrade() throws Exception { statelessSessionJob.getSpec().getJob().setParallelism(2); // job suspended first reconciler.reconcile(statelessSessionJob, readyContext); - assertEquals(FINISHED, flinkService.listJobs().get(0).f1.getJobState()); - verifyJobState(statelessSessionJob, JobState.SUSPENDED, "FINISHED"); + assertEquals(CANCELED, flinkService.listJobs().get(0).f1.getJobState()); + verifyJobState(statelessSessionJob, JobState.RUNNING, "CANCELLING"); + statelessSessionJob.getStatus().getJobStatus().setState("CANCELED"); flinkService.clear(); reconciler.reconcile(statelessSessionJob, readyContext); @@ -522,6 +505,7 @@ public void testTriggerSavepointLegacyLegacy() throws Exception { // running -> suspended reconciler.reconcile(sp1SessionJob, readyContext); + sp1SessionJob.getStatus().getJobStatus().setState("CANCELED"); // suspended -> running reconciler.reconcile(sp1SessionJob, readyContext); // parallelism changed @@ -616,33 +600,17 @@ public void testTriggerCheckpoint() throws Exception { assertFalse(SnapshotUtils.checkpointInProgress(getJobStatus(sp1SessionJob))); } - private static Stream cancelStatelessSessionJobParams() { - return Stream.of( - Arguments.of(INITIALIZING, true), - Arguments.of(CREATED, true), - Arguments.of(RUNNING, true), - Arguments.of(FAILING, true), - Arguments.of(FAILED, false), - Arguments.of(CANCELLING, true), - Arguments.of(CANCELED, false), - Arguments.of(FINISHED, false), - Arguments.of(RESTARTING, true), - Arguments.of(SUSPENDED, true), - Arguments.of(RECONCILING, true)); + @Test + public void testCancelStatelessSessionJob() throws Exception { + testCancelStatelessSessionJob(SuspendMode.STATELESS); } @Test - public void testCancelStatelessSessionJobParams() { - assertEquals( - org.apache.flink.api.common.JobStatus.values().length, - cancelStatelessSessionJobParams().count()); + public void testCancelSessionJob() throws Exception { + testCancelStatelessSessionJob(SuspendMode.CANCEL); } - @ParameterizedTest - @MethodSource("cancelStatelessSessionJobParams") - public void testCancelStatelessSessionJob( - org.apache.flink.api.common.JobStatus fromJobStatus, boolean shouldCallCancel) - throws Exception { + public void testCancelStatelessSessionJob(SuspendMode suspendMode) throws Exception { FlinkSessionJob sessionJob = TestUtils.buildSessionJob(); var readyContext = TestUtils.createContextWithReadyFlinkDeployment(kubernetesClient); @@ -663,21 +631,16 @@ public void testCancelStatelessSessionJob( new JobStatusMessage( jobStatusMessage.getJobId(), jobStatusMessage.getJobName(), - fromJobStatus, + RUNNING, jobStatusMessage.getStartTime()); // Set state which must be overwritten by cancelSessionJob - sessionJob.getStatus().getJobStatus().setState(fromJobStatus.name()); + sessionJob.getStatus().getJobStatus().setState("RUNNING"); - flinkService.cancelSessionJob(sessionJob, UpgradeMode.STATELESS, jobConfig); + flinkService.cancelSessionJob(sessionJob, suspendMode, jobConfig); - if (!shouldCallCancel) { - assertEquals(0, flinkService.getCancelJobCallCount()); - assertEquals(fromJobStatus, job.f1.getJobState()); - } else { - assertEquals(1, flinkService.getCancelJobCallCount()); - assertEquals(FINISHED, job.f1.getJobState()); - } - assertEquals(FINISHED.name(), sessionJob.getStatus().getJobStatus().getState()); + assertEquals(1, flinkService.getCancelJobCallCount()); + assertEquals(CANCELED, job.f1.getJobState()); + assertEquals(CANCELLING.name(), sessionJob.getStatus().getJobStatus().getState()); } private static Stream cancelSavepointSessionJobParams() { @@ -686,10 +649,10 @@ private static Stream cancelSavepointSessionJobParams() { Arguments.of(CREATED, true, false), Arguments.of(RUNNING, false, true), Arguments.of(FAILING, true, false), - Arguments.of(FAILED, false, false), Arguments.of(CANCELLING, true, false), - Arguments.of(CANCELED, false, false), - Arguments.of(FINISHED, false, false), + Arguments.of(CANCELED, true, false), + Arguments.of(FINISHED, true, false), + Arguments.of(FAILED, true, false), Arguments.of(RESTARTING, true, false), Arguments.of(SUSPENDED, true, false), Arguments.of(RECONCILING, true, false)); @@ -739,15 +702,15 @@ public void testCancelSavepointSessionJob( sessionJob.getStatus().getJobStatus().setState(fromJobStatus.name()); if (!shouldThrowException) { - flinkService.cancelSessionJob(sessionJob, UpgradeMode.SAVEPOINT, jobConfig); + flinkService.cancelSessionJob(sessionJob, SuspendMode.SAVEPOINT, jobConfig); } else { var e = assertThrows( RuntimeException.class, () -> flinkService.cancelSessionJob( - sessionJob, UpgradeMode.SAVEPOINT, jobConfig)); - Assertions.assertTrue(e.getMessage().contains("Unexpected non-terminal status")); + sessionJob, SuspendMode.SAVEPOINT, jobConfig)); + Assertions.assertTrue(e.getMessage().contains("Unexpected job status")); } if (!shouldCallCancel) { @@ -839,7 +802,7 @@ public void testJobUpgradeIgnorePendingSavepoint() throws Exception { assertEquals( "savepoint_trigger_0", spSessionJob.getStatus().getJobStatus().getSavepointInfo().getTriggerId()); - assertEquals("FINISHED", spSessionJob.getStatus().getJobStatus().getState()); + assertEquals("CANCELLING", spSessionJob.getStatus().getJobStatus().getState()); } @Test @@ -868,6 +831,7 @@ public void testJobIdGeneration() throws Exception { () -> { // suspend reconciler.reconcile(sessionJob, readyContext); + sessionJob.getStatus().getJobStatus().setState("CANCELED"); // upgrade reconciler.reconcile(sessionJob, readyContext); }); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java index 3cfa562b21..514acbd405 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/AbstractFlinkServiceTest.java @@ -36,7 +36,6 @@ import org.apache.flink.kubernetes.operator.api.FlinkDeployment; import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; import org.apache.flink.kubernetes.operator.api.spec.JobSpec; -import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.api.status.CheckpointType; import org.apache.flink.kubernetes.operator.api.status.FlinkDeploymentStatus; import org.apache.flink.kubernetes.operator.api.status.JobManagerDeploymentStatus; @@ -47,7 +46,7 @@ import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; import org.apache.flink.kubernetes.operator.controller.FlinkResourceContext; -import org.apache.flink.kubernetes.operator.exception.RecoveryFailureException; +import org.apache.flink.kubernetes.operator.exception.UpgradeFailureException; import org.apache.flink.kubernetes.operator.observer.CheckpointFetchResult; import org.apache.flink.kubernetes.operator.observer.SavepointFetchResult; import org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils; @@ -127,7 +126,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutorService; @@ -209,7 +207,9 @@ public void sessionJobSubmissionTest(FlinkVersion flinkVersion) throws Exception .serializeAndSetLastReconciledSpec(session.getSpec(), session); var job = TestUtils.buildSessionJob(); - var deployConf = configManager.getSessionJobConfig(session, job.getSpec()); + var deployConf = + configManager.getSessionJobConfig( + job.getMetadata().getName(), session, job.getSpec()); flinkService.submitJobToSessionCluster( job.getMetadata(), job.getSpec(), JobID.generate(), deployConf, null); @@ -296,13 +296,26 @@ public void cancelJobWithStatelessUpgradeModeTest() throws Exception { deployment.getStatus().getJobStatus().setState("RUNNING"); flinkService.cancelJob( deployment, - UpgradeMode.STATELESS, + SuspendMode.STATELESS, configManager.getObserveConfig(deployment), false); assertTrue(cancelFuture.isDone()); assertEquals(jobID, cancelFuture.get()); assertNull(jobStatus.getSavepointInfo().getLastSavepoint()); assertNull(jobStatus.getUpgradeSavepointPath()); + assertEquals("FINISHED", jobStatus.getState()); + assertEquals( + List.of( + Tuple2.of( + deployment.getMetadata().getNamespace(), + deployment.getMetadata().getName())), + flinkService.deleted); + assertEquals( + List.of( + Tuple2.of( + deployment.getMetadata().getNamespace(), + deployment.getMetadata().getName())), + flinkService.haDeleted); } @ParameterizedTest @@ -331,11 +344,11 @@ public void cancelErrorHandling(int statusCode) throws Exception { Exception.class, () -> flinkService.cancelSessionJob( - job, UpgradeMode.STATELESS, new Configuration())); + job, SuspendMode.STATELESS, new Configuration())); assertEquals("RUNNING", jobStatus.getState()); } else { - flinkService.cancelSessionJob(job, UpgradeMode.STATELESS, new Configuration()); - assertEquals("FINISHED", jobStatus.getState()); + flinkService.cancelSessionJob(job, SuspendMode.STATELESS, new Configuration()); + assertEquals("CANCELLING", jobStatus.getState()); } } @@ -370,13 +383,13 @@ public void cancelJobWithSavepointUpgradeModeTest(boolean deleteAfterSavepoint) jobStatus.setState(org.apache.flink.api.common.JobStatus.RUNNING.name()); ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); - var savepointPathOpt = + var result = flinkService.cancelJob( deployment, - UpgradeMode.SAVEPOINT, + SuspendMode.SAVEPOINT, configManager.getObserveConfig(deployment), deleteAfterSavepoint); - assertEquals(savepointPath, savepointPathOpt.get()); + assertEquals(savepointPath, result.getSavepointPath().get()); assertTrue(stopWithSavepointFuture.isDone()); assertEquals(jobID, stopWithSavepointFuture.get().f0); @@ -444,13 +457,13 @@ public void cancelJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnSavepoin .put(KubernetesOperatorConfigOptions.DRAIN_ON_SAVEPOINT_DELETION.key(), "true"); } - var savepointOpt = + var result = flinkService.cancelJob( deployment, - UpgradeMode.SAVEPOINT, + SuspendMode.SAVEPOINT, configManager.getObserveConfig(deployment), true); - assertEquals(savepointPath, savepointOpt.get()); + assertEquals(savepointPath, result.getSavepointPath().get()); assertTrue(stopWithSavepointFuture.isDone()); assertEquals(jobID, stopWithSavepointFuture.get().f0); @@ -507,14 +520,16 @@ public void cancelSessionJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnS .getFlinkConfiguration() .put(KubernetesOperatorConfigOptions.DRAIN_ON_SAVEPOINT_DELETION.key(), "true"); } - var deployConf = configManager.getSessionJobConfig(session, job.getSpec()); + var deployConf = + configManager.getSessionJobConfig( + job.getMetadata().getName(), session, job.getSpec()); - var savepointOpt = flinkService.cancelSessionJob(job, UpgradeMode.SAVEPOINT, deployConf); + var result = flinkService.cancelSessionJob(job, SuspendMode.SAVEPOINT, deployConf); assertTrue(stopWithSavepointFuture.isDone()); assertEquals(jobID, stopWithSavepointFuture.get().f0); assertEquals(jobStatus.getState(), org.apache.flink.api.common.JobStatus.FINISHED.name()); - assertEquals(savepointPath, savepointOpt.get()); + assertEquals(savepointPath, result.getSavepointPath().get()); if (drainOnSavepoint) { assertTrue(stopWithSavepointFuture.get().f1); @@ -524,7 +539,7 @@ public void cancelSessionJobWithDrainOnSavepointUpgradeModeTest(boolean drainOnS } @Test - public void cancelJobWithLastStateUpgradeModeTest() throws Exception { + public void jobCancelTest() throws Exception { var deployment = TestUtils.buildApplicationCluster(); ReconciliationUtils.updateStatusForDeployedSpec(deployment, new Configuration()); var testingClusterClient = @@ -534,12 +549,13 @@ public void cancelJobWithLastStateUpgradeModeTest() throws Exception { JobID jobID = JobID.generate(); JobStatus jobStatus = deployment.getStatus().getJobStatus(); jobStatus.setJobId(jobID.toHexString()); + jobStatus.setState("FAILING"); flinkService.cancelJob( - deployment, - UpgradeMode.LAST_STATE, - configManager.getObserveConfig(deployment), - false); + deployment, SuspendMode.CANCEL, configManager.getObserveConfig(deployment), false); + assertTrue(flinkService.haDeleted.isEmpty()); + assertTrue(flinkService.deleted.isEmpty()); + assertEquals("CANCELLING", jobStatus.getState()); assertNull(jobStatus.getSavepointInfo().getLastSavepoint()); assertNull(jobStatus.getUpgradeSavepointPath()); } @@ -755,12 +771,12 @@ private void runNativeSavepointFormatTest(boolean failAfterSavepointCompletes) var conf = new Configuration(configManager.getObserveConfig(deployment)) .set(OPERATOR_SAVEPOINT_FORMAT_TYPE, SavepointFormatType.NATIVE); - var savepointOpt = flinkService.cancelJob(deployment, UpgradeMode.SAVEPOINT, conf, false); + var result = flinkService.cancelJob(deployment, SuspendMode.SAVEPOINT, conf, false); assertTrue(stopWithSavepointFuture.isDone()); assertEquals( failAfterSavepointCompletes, stopWithSavepointFuture.isCompletedExceptionally()); - assertEquals(savepointPath, savepointOpt.get()); + assertEquals(savepointPath, result.getSavepointPath().get()); } @Test @@ -806,7 +822,7 @@ public void getLastCheckpointTest() throws Exception { try { flinkService.getLastCheckpoint(new JobID(), new Configuration()); fail(); - } catch (RecoveryFailureException dpe) { + } catch (UpgradeFailureException dpe) { } } @@ -1204,6 +1220,7 @@ class TestingService extends AbstractFlinkService { RestClusterClient clusterClient; RestClient restClient; List> deleted = new ArrayList<>(); + List> haDeleted = new ArrayList<>(); Map, PodList> jmPods = new HashMap<>(); Map, PodList> tmPods = new HashMap<>(); @@ -1237,11 +1254,6 @@ protected PodList getJmPodList(String namespace, String clusterId) { return jmPods.getOrDefault(Tuple2.of(namespace, clusterId), new PodList()); } - @Override - protected PodList getTmPodList(String namespace, String clusterId) { - return tmPods.getOrDefault(Tuple2.of(namespace, clusterId), new PodList()); - } - @Override protected void deployApplicationCluster(JobSpec jobSpec, Configuration conf) { throw new UnsupportedOperationException(); @@ -1253,8 +1265,8 @@ public void deploySessionCluster(Configuration conf) { } @Override - public Optional cancelJob( - FlinkDeployment deployment, UpgradeMode upgradeMode, Configuration conf) { + public CancelResult cancelJob( + FlinkDeployment deployment, SuspendMode upgradeMode, Configuration conf) { throw new UnsupportedOperationException(); } @@ -1266,10 +1278,15 @@ public boolean scale(FlinkResourceContext resourceContext, Configuration conf @Override protected void deleteClusterInternal( String namespace, - String cluserId, + String clusterId, Configuration conf, DeletionPropagation deletionPropagation) { - deleted.add(Tuple2.of(namespace, cluserId)); + deleted.add(Tuple2.of(namespace, clusterId)); + } + + @Override + protected void deleteHAData(String namespace, String clusterId, Configuration conf) { + haDeleted.add(Tuple2.of(namespace, clusterId)); } } } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java index f463511a05..23f179b2f1 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/service/NativeFlinkServiceTest.java @@ -30,7 +30,6 @@ import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec; import org.apache.flink.kubernetes.operator.api.spec.FlinkVersion; import org.apache.flink.kubernetes.operator.api.spec.JobSpec; -import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; import org.apache.flink.kubernetes.operator.config.FlinkConfigManager; import org.apache.flink.kubernetes.operator.config.FlinkOperatorConfiguration; import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; @@ -44,8 +43,6 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.JobVertexResourceRequirements; import org.apache.flink.runtime.rest.messages.JobMessageParameters; -import org.apache.flink.runtime.rest.messages.JobPlanInfo; -import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo; import org.apache.flink.runtime.rest.messages.job.JobResourceRequirementsBody; import org.apache.flink.runtime.rest.messages.job.JobResourceRequirementsHeaders; import org.apache.flink.util.concurrent.Executors; @@ -64,7 +61,6 @@ import java.time.Duration; import java.time.Instant; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -206,20 +202,20 @@ public void testDeleteOnSavepointBefore1_15(FlinkVersion flinkVersion) throws Ex new NativeFlinkService( client, null, executorService, operatorConfig, eventRecorder) { @Override - protected Optional cancelJob( + protected CancelResult cancelJob( FlinkDeployment deployment, - UpgradeMode upgradeMode, + SuspendMode upgradeMode, Configuration conf, - boolean deleteClusterAfterSavepoint) { - assertEquals(false, deleteClusterAfterSavepoint); + boolean deleteCluster) { + assertFalse(deleteCluster); tested.set(true); - return Optional.empty(); + return CancelResult.completed(null); } }; flinkService.cancelJob( TestUtils.buildApplicationCluster(flinkVersion), - UpgradeMode.SAVEPOINT, + SuspendMode.SAVEPOINT, new Configuration()); assertTrue(tested.get()); } @@ -554,24 +550,6 @@ public void resourceRestApiTest() throws Exception { testingClusterClient, deployment, reqs.getJobVertexParallelisms()); } - public static JobDetailsInfo createJobDetailsFor( - List vertexInfos) { - return new JobDetailsInfo( - new JobID(), - "", - false, - org.apache.flink.api.common.JobStatus.RUNNING, - 0, - 0, - 0, - 0, - 0, - Map.of(), - vertexInfos, - Map.of(), - new JobPlanInfo.RawJson("")); - } - class TestingNativeFlinkService extends NativeFlinkService { private Configuration runtimeConfig; @@ -590,7 +568,7 @@ protected void deployApplicationCluster(JobSpec jobSpec, Configuration conf) { } @Override - protected void submitClusterInternal(Configuration conf) throws Exception { + protected void submitClusterInternal(Configuration conf) { this.runtimeConfig = conf; } @@ -610,9 +588,6 @@ public RestClusterClient getClusterClient(Configuration config) { } private Configuration createOperatorConfig() { - Map configMap = Map.of(OPERATOR_HEALTH_PROBE_PORT.key(), "80"); - Configuration deployConfig = Configuration.fromMap(configMap); - - return deployConfig; + return Configuration.fromMap(Map.of(OPERATOR_HEALTH_PROBE_PORT.key(), "80")); } } diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/ReconciliationUtilsTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/ReconciliationUtilsTest.java index 75ab9322f3..2957a75816 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/ReconciliationUtilsTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/utils/ReconciliationUtilsTest.java @@ -32,7 +32,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertTrue; /** Test for {@link org.apache.flink.kubernetes.operator.reconciler.ReconciliationUtils}. */ public class ReconciliationUtilsTest { @@ -65,20 +64,6 @@ public void testRescheduleUpgradeImmediately() { assertNotEquals(0, updateControl.getScheduleDelay().get()); } - @Test - public void testRescheduleIfImmediateFlagSet() { - var previous = BaseTestUtils.buildApplicationCluster(); - var current = BaseTestUtils.buildApplicationCluster(); - var updateControl = - ReconciliationUtils.toUpdateControl(operatorConfiguration, current, previous, true); - assertTrue(updateControl.getScheduleDelay().get() > 0); - - current.getStatus().setImmediateReconciliationNeeded(true); - updateControl = - ReconciliationUtils.toUpdateControl(operatorConfiguration, current, previous, true); - assertEquals(0, updateControl.getScheduleDelay().get()); - } - @Test public void testObservedGenerationStatus() throws Exception { FlinkDeployment app = BaseTestUtils.buildApplicationCluster(); diff --git a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java index f5d0712f10..5bf5033afb 100644 --- a/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java +++ b/flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/validation/DefaultValidatorTest.java @@ -43,7 +43,6 @@ import org.apache.flink.kubernetes.operator.api.spec.UpgradeMode; 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 org.apache.flink.kubernetes.operator.api.status.JobStatus; import org.apache.flink.kubernetes.operator.api.status.Savepoint; import org.apache.flink.kubernetes.operator.api.status.SnapshotTriggerType; @@ -112,13 +111,6 @@ public void testValidationWithoutDefaultConfig() { dep.getSpec().getJob().setParallelism(0); }); - testError( - dep -> { - dep.getSpec().setFlinkConfiguration(new HashMap<>()); - dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); - }, - "Job could not be upgraded with last-state while HA disabled"); - testError( dep -> { dep.getSpec() @@ -462,31 +454,6 @@ public void testValidationWithoutDefaultConfig() { }, "Cannot switch from standalone kubernetes to native kubernetes cluster"); - // Test upgrade mode change validation - testError( - dep -> { - dep.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE); - dep.getSpec() - .getFlinkConfiguration() - .remove(CheckpointingOptions.SAVEPOINT_DIRECTORY.key()); - dep.setStatus(new FlinkDeploymentStatus()); - dep.getStatus().setJobStatus(new JobStatus()); - - dep.getStatus() - .setReconciliationStatus(new FlinkDeploymentReconciliationStatus()); - FlinkDeploymentSpec spec = ReconciliationUtils.clone(dep.getSpec()); - spec.getJob().setUpgradeMode(UpgradeMode.STATELESS); - spec.getFlinkConfiguration().remove(HighAvailabilityOptions.HA_MODE.key()); - - dep.getStatus() - .getReconciliationStatus() - .serializeAndSetLastReconciledSpec(spec, dep); - dep.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.READY); - }, - String.format( - "Job could not be upgraded to last-state while config key[%s] is not set", - CheckpointingOptions.SAVEPOINT_DIRECTORY.key())); - testError(dep -> dep.getSpec().setFlinkVersion(null), "Flink Version must be defined."); testError( @@ -719,7 +686,7 @@ public void testSessionJobWithSession() { testSessionJobValidateWithModifier( sessionJob -> sessionJob.getSpec().getJob().setUpgradeMode(UpgradeMode.LAST_STATE), flinkDeployment -> {}, - "The LAST_STATE upgrade mode is not supported in session job now."); + null); testSessionJobValidateWithModifier( sessionJob ->