gyfora commented on code in PR #165: URL: https://github.com/apache/flink-kubernetes-operator/pull/165#discussion_r850305005
########## flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/reconciler/deployment/ApplicationReconciler.java: ########## @@ -102,36 +111,84 @@ public void reconcile(FlinkDeployment flinkApp, Context context, Configuration e } if (currentJobState == JobState.SUSPENDED && desiredJobState == JobState.RUNNING) { if (upgradeMode == UpgradeMode.STATELESS) { - deployFlinkJob(flinkApp, effectiveConfig, Optional.empty()); - } else if (upgradeMode == UpgradeMode.LAST_STATE - || upgradeMode == UpgradeMode.SAVEPOINT) { - restoreFromLastSavepoint(flinkApp, effectiveConfig); + deployFlinkJob(currentJobSpec, status, effectiveConfig, Optional.empty()); + } else { + restoreFromLastSavepoint(currentJobSpec, status, effectiveConfig); } stateAfterReconcile = JobState.RUNNING; } - IngressUtils.updateIngressRules(flinkApp, effectiveConfig, kubernetesClient); + IngressUtils.updateIngressRules( + deployMeta, currentDeploySpec, effectiveConfig, kubernetesClient); ReconciliationUtils.updateForSpecReconciliationSuccess(flinkApp, stateAfterReconcile); - } else if (SavepointUtils.shouldTriggerSavepoint(flinkApp) && isJobRunning(flinkApp)) { + } else if (ReconciliationUtils.shouldRollBack(reconciliationStatus, effectiveConfig)) { + rollbackApplication(flinkApp); + } else if (SavepointUtils.shouldTriggerSavepoint(currentJobSpec, status) + && isJobRunning(status)) { triggerSavepoint(flinkApp, effectiveConfig); ReconciliationUtils.updateSavepointReconciliationSuccess(flinkApp); + } else { + LOG.info("Deployment is fully reconciled, nothing to do."); } } + private void rollbackApplication(FlinkDeployment flinkApp) throws Exception { + ReconciliationStatus reconciliationStatus = flinkApp.getStatus().getReconciliationStatus(); + + if (reconciliationStatus.getState() != ReconciliationStatus.State.ROLLING_BACK) { + LOG.warn("Preparing to roll back to last stable spec."); + if (flinkApp.getStatus().getError() == null) { + flinkApp.getStatus() + .setError( + "Deployment is not ready within the configured timeout, rolling-back."); + } + reconciliationStatus.setState(ReconciliationStatus.State.ROLLING_BACK); + return; + } + + LOG.warn("Executing roll-back operation"); + + FlinkDeploymentSpec rollbackSpec = reconciliationStatus.deserializeLastStableSpec(); + Configuration rollbackConfig = + FlinkUtils.getEffectiveConfig(flinkApp.getMetadata(), rollbackSpec, defaultConfig); + + UpgradeMode upgradeMode = flinkApp.getSpec().getJob().getUpgradeMode(); + + suspendJob( Review Comment: I am afraid it is not that simple and it is by design like this. Let me try to explain. I think we earlier decided against making changes to the spec itself as it breaks the idempotency of the operator logic. If we simply change the spec then an external process might simply reapply what was previous rolled back causing a loop. This is also the reason why we do not upgrade the `lastReconciledSpec`. That field shows the last spec that the user have sent in that was reconciled by the controller. Even if it was rolled back later that does not change. The ReconciliationStatus.State shows what actually happened to the lastReconciledSpec: DEPLOYED means it is running on the cluster, ROLLING_BACK/ROLLED_BACK actually signal that it was reverted after it was DEPLOYED. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org