You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by GitBox <gi...@apache.org> on 2022/03/08 05:02:49 UTC

[GitHub] [flink-kubernetes-operator] tweise opened a new pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

tweise opened a new pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46


   The PR is a sketch for handling a deployment that results in a permanent error state (invalid service account) that would require a CR update to resolve. I do not particularly like the exception based flow, but this scenario requires access to the k8s client.
   
   The event logged looks like this (kubectl describe):
   ```
   ReplicaFailure  FailedCreate  4h48m  JobManagerDeployment  pods "basic-example-d5c476cb6-" is forbidden: error looking up service account default/invalid-flink-operator: serviceaccount "invalid-flink-operator" not found
   ```


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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] SteNicholas commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
SteNicholas commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r821440105



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java
##########
@@ -113,29 +118,70 @@ public DeleteControl cleanup(FlinkDeployment flinkApp, Context context) {
         Configuration effectiveConfig =
                 FlinkUtils.getEffectiveConfig(flinkApp, defaultConfig.getFlinkConfig());
 
-        boolean readyToReconcile = observer.observe(flinkApp, context, effectiveConfig);
-        if (!readyToReconcile) {
-            return flinkApp.getStatus()
-                    .getJobManagerDeploymentStatus()
-                    .toUpdateControl(flinkApp, operatorConfiguration);
-        }
-
         try {
+            boolean readyToReconcile = observer.observe(flinkApp, context, effectiveConfig);
+            if (!readyToReconcile) {
+                return flinkApp.getStatus()
+                        .getJobManagerDeploymentStatus()
+                        .toUpdateControl(flinkApp, operatorConfiguration);
+            }
+
             UpdateControl<FlinkDeployment> updateControl =
                     reconcilerFactory
                             .getOrCreate(flinkApp)
                             .reconcile(operatorNamespace, flinkApp, context, effectiveConfig);
             updateForReconciliationSuccess(flinkApp);
             return updateControl;
-        } catch (InvalidDeploymentException ide) {
-            LOG.error("Reconciliation failed", ide);
-            updateForReconciliationError(flinkApp, ide.getMessage());
+        } catch (DeploymentFailedException dfe) {
+            updateForDeploymentFailed(flinkApp, dfe);
             return UpdateControl.updateStatus(flinkApp);
         } catch (Exception e) {
             throw new ReconciliationException(e);
         }
     }
 
+    private void updateForDeploymentFailed(
+            FlinkDeployment flinkApp, DeploymentFailedException dfe) {
+        LOG.error(
+                "Deployment {}/{} failed with {}",
+                flinkApp.getMetadata().getNamespace(),
+                flinkApp.getMetadata().getName(),
+                dfe.getMessage());
+        flinkApp.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.ERROR);
+        updateForReconciliationError(flinkApp, dfe.getMessage());
+
+        // TODO: avoid repeated event
+        EventBuilder evtb =
+                new EventBuilder()
+                        .withApiVersion("v1")

Review comment:
       Coud this use the ApiVersion of the FlinkDeployment?




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] tweise commented on pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
tweise commented on pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#issuecomment-1061411308


   @gyfora fyi we should probably rename `JobManagerDeploymentStatus` to something more general like `DeploymentStatus` as that should cover all relevant states, whether they are related to JobManager or not. If that is something agreeable, I'm going to add a ticket.


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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] gyfora commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
gyfora commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r821349489



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobManagerDeploymentStatus.java
##########
@@ -54,8 +58,9 @@
                                 operatorConfiguration.getPortCheckIntervalInSec(),
                                 TimeUnit.SECONDS);
             case MISSING:
+            case ERROR:
             default:
-                return null;
+                return UpdateControl.noUpdate();

Review comment:
       An error will trigger a status update, is `noUpdate` correct here? 

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java
##########
@@ -113,29 +118,70 @@ public DeleteControl cleanup(FlinkDeployment flinkApp, Context context) {
         Configuration effectiveConfig =
                 FlinkUtils.getEffectiveConfig(flinkApp, defaultConfig.getFlinkConfig());
 
-        boolean readyToReconcile = observer.observe(flinkApp, context, effectiveConfig);
-        if (!readyToReconcile) {
-            return flinkApp.getStatus()
-                    .getJobManagerDeploymentStatus()
-                    .toUpdateControl(flinkApp, operatorConfiguration);
-        }
-
         try {
+            boolean readyToReconcile = observer.observe(flinkApp, context, effectiveConfig);
+            if (!readyToReconcile) {
+                return flinkApp.getStatus()
+                        .getJobManagerDeploymentStatus()
+                        .toUpdateControl(flinkApp, operatorConfiguration);
+            }
+
             UpdateControl<FlinkDeployment> updateControl =
                     reconcilerFactory
                             .getOrCreate(flinkApp)
                             .reconcile(operatorNamespace, flinkApp, context, effectiveConfig);
             updateForReconciliationSuccess(flinkApp);
             return updateControl;
-        } catch (InvalidDeploymentException ide) {
-            LOG.error("Reconciliation failed", ide);
-            updateForReconciliationError(flinkApp, ide.getMessage());
+        } catch (DeploymentFailedException dfe) {
+            updateForDeploymentFailed(flinkApp, dfe);
             return UpdateControl.updateStatus(flinkApp);
         } catch (Exception e) {
             throw new ReconciliationException(e);
         }
     }
 
+    private void updateForDeploymentFailed(
+            FlinkDeployment flinkApp, DeploymentFailedException dfe) {
+        LOG.error(
+                "Deployment {}/{} failed with {}",
+                flinkApp.getMetadata().getNamespace(),
+                flinkApp.getMetadata().getName(),
+                dfe.getMessage());
+        flinkApp.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.ERROR);
+        updateForReconciliationError(flinkApp, dfe.getMessage());
+
+        // TODO: avoid repeated event
+        EventBuilder evtb =
+                new EventBuilder()
+                        .withApiVersion("v1")
+                        .withNewInvolvedObject()
+                        .withKind(flinkApp.getKind())
+                        .withName(flinkApp.getMetadata().getName())
+                        .withNamespace(flinkApp.getMetadata().getNamespace())
+                        .withUid(flinkApp.getMetadata().getUid())
+                        .endInvolvedObject()
+                        .withType(dfe.deployCondition.getType())
+                        .withReason(dfe.deployCondition.getReason())
+                        // TODO: timestamp
+                        // .withEventTime(new MicroTime("2006-01-02T15:04:05.000000Z"))
+                        // .withNewEventTime(timeFormat)
+                        .withFirstTimestamp(dfe.deployCondition.getLastTransitionTime())
+                        .withLastTimestamp(dfe.deployCondition.getLastUpdateTime())
+                        .withMessage(dfe.getMessage())
+                        .withNewMetadata()
+                        .withGenerateName(flinkApp.getMetadata().getName())
+                        .withNamespace(flinkApp.getMetadata().getNamespace())
+                        .endMetadata()
+                        .withNewSource()
+                        .withComponent(dfe.component)
+                        .endSource();

Review comment:
       This should be extracted to a utility method that returns `Event`

##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java
##########
@@ -113,29 +118,70 @@ public DeleteControl cleanup(FlinkDeployment flinkApp, Context context) {
         Configuration effectiveConfig =
                 FlinkUtils.getEffectiveConfig(flinkApp, defaultConfig.getFlinkConfig());
 
-        boolean readyToReconcile = observer.observe(flinkApp, context, effectiveConfig);
-        if (!readyToReconcile) {
-            return flinkApp.getStatus()
-                    .getJobManagerDeploymentStatus()
-                    .toUpdateControl(flinkApp, operatorConfiguration);
-        }
-
         try {
+            boolean readyToReconcile = observer.observe(flinkApp, context, effectiveConfig);
+            if (!readyToReconcile) {
+                return flinkApp.getStatus()
+                        .getJobManagerDeploymentStatus()
+                        .toUpdateControl(flinkApp, operatorConfiguration);
+            }
+
             UpdateControl<FlinkDeployment> updateControl =
                     reconcilerFactory
                             .getOrCreate(flinkApp)
                             .reconcile(operatorNamespace, flinkApp, context, effectiveConfig);
             updateForReconciliationSuccess(flinkApp);
             return updateControl;
-        } catch (InvalidDeploymentException ide) {
-            LOG.error("Reconciliation failed", ide);
-            updateForReconciliationError(flinkApp, ide.getMessage());
+        } catch (DeploymentFailedException dfe) {
+            updateForDeploymentFailed(flinkApp, dfe);
             return UpdateControl.updateStatus(flinkApp);
         } catch (Exception e) {
             throw new ReconciliationException(e);
         }
     }
 
+    private void updateForDeploymentFailed(

Review comment:
       I think the method name should say something about the event being fired. This is not really just a status update as the others. 




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] tweise commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r822107754



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java
##########
@@ -113,29 +118,70 @@ public DeleteControl cleanup(FlinkDeployment flinkApp, Context context) {
         Configuration effectiveConfig =
                 FlinkUtils.getEffectiveConfig(flinkApp, defaultConfig.getFlinkConfig());
 
-        boolean readyToReconcile = observer.observe(flinkApp, context, effectiveConfig);
-        if (!readyToReconcile) {
-            return flinkApp.getStatus()
-                    .getJobManagerDeploymentStatus()
-                    .toUpdateControl(flinkApp, operatorConfiguration);
-        }
-
         try {
+            boolean readyToReconcile = observer.observe(flinkApp, context, effectiveConfig);
+            if (!readyToReconcile) {
+                return flinkApp.getStatus()
+                        .getJobManagerDeploymentStatus()
+                        .toUpdateControl(flinkApp, operatorConfiguration);
+            }
+
             UpdateControl<FlinkDeployment> updateControl =
                     reconcilerFactory
                             .getOrCreate(flinkApp)
                             .reconcile(operatorNamespace, flinkApp, context, effectiveConfig);
             updateForReconciliationSuccess(flinkApp);
             return updateControl;
-        } catch (InvalidDeploymentException ide) {
-            LOG.error("Reconciliation failed", ide);
-            updateForReconciliationError(flinkApp, ide.getMessage());
+        } catch (DeploymentFailedException dfe) {
+            updateForDeploymentFailed(flinkApp, dfe);
             return UpdateControl.updateStatus(flinkApp);
         } catch (Exception e) {
             throw new ReconciliationException(e);
         }
     }
 
+    private void updateForDeploymentFailed(
+            FlinkDeployment flinkApp, DeploymentFailedException dfe) {
+        LOG.error(
+                "Deployment {}/{} failed with {}",
+                flinkApp.getMetadata().getNamespace(),
+                flinkApp.getMetadata().getName(),
+                dfe.getMessage());
+        flinkApp.getStatus().setJobManagerDeploymentStatus(JobManagerDeploymentStatus.ERROR);
+        updateForReconciliationError(flinkApp, dfe.getMessage());
+
+        // TODO: avoid repeated event
+        EventBuilder evtb =
+                new EventBuilder()
+                        .withApiVersion("v1")

Review comment:
       No, that version is `flink.apache.org/v1alpha1`




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] tweise commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r822108437



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java
##########
@@ -92,8 +94,11 @@ public DeleteControl cleanup(FlinkDeployment flinkApp, Context context) {
         LOG.info("Stopping cluster {}", flinkApp.getMetadata().getName());
         Configuration effectiveConfig =
                 FlinkUtils.getEffectiveConfig(flinkApp, defaultConfig.getFlinkConfig());
-
-        observer.observe(flinkApp, context, effectiveConfig);
+        try {
+            observer.observe(flinkApp, context, effectiveConfig);
+        } catch (DeploymentFailedException dfe) {

Review comment:
       Relevant information is already logged, for example:
   ```
   [ERROR] [default.basic-example] Deployment default/basic-example failed with pods "basic-example-d5c476cb6-" is forbidden: error looking up service account default/invalid-flink-operator: serviceaccount "invalid-flink-operator" not found
   ```




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] SteNicholas commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
SteNicholas commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r821441066



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobManagerDeploymentStatus.java
##########
@@ -37,7 +37,11 @@
     DEPLOYING,
 
     /** JobManager deployment not found, probably not started or killed by user. */
-    MISSING;
+    // TODO: currently a mix of SUSPENDED and ERROR, needs cleanup

Review comment:
       Does this need to add the `SUSPENDED` status?




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] tweise commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r822312168



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/Observer.java
##########
@@ -94,6 +96,14 @@ private void observeJmDeployment(
                     flinkApp.getMetadata().getNamespace(),
                     status);
 
+            List<DeploymentCondition> conditions = status.getConditions();
+            for (DeploymentCondition dc : conditions) {
+                if ("FailedCreate".equals(dc.getReason())
+                        && "ReplicaFailure".equals(dc.getType())) {
+                    throw new DeploymentFailedException(
+                            DeploymentFailedException.COMPONENT_JOBMANAGER, dc);
+                }
+            }

Review comment:
       Done, exception is now only raised when not already in ERROR




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] tweise commented on pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
tweise commented on pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#issuecomment-1062588999


   @gyfora PTAL


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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] tweise commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r822312279



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobManagerDeploymentStatus.java
##########
@@ -37,7 +37,11 @@
     DEPLOYING,
 
     /** JobManager deployment not found, probably not started or killed by user. */
-    MISSING;
+    // TODO: currently a mix of SUSPENDED and ERROR, needs cleanup
+    MISSING,
+
+    /** Deployment in terminal error, requires spec change for reconciliation to continue. */
+    ERROR;

Review comment:
       Good catch, done!




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] gyfora commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
gyfora commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r821511530



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/Observer.java
##########
@@ -94,6 +96,14 @@ private void observeJmDeployment(
                     flinkApp.getMetadata().getNamespace(),
                     status);
 
+            List<DeploymentCondition> conditions = status.getConditions();
+            for (DeploymentCondition dc : conditions) {
+                if ("FailedCreate".equals(dc.getReason())
+                        && "ReplicaFailure".equals(dc.getType())) {
+                    throw new DeploymentFailedException(
+                            DeploymentFailedException.COMPONENT_JOBMANAGER, dc);
+                }
+            }

Review comment:
       We should not check for new Errors if we are already in ERROR deploymant status, otherwise it wont be possible to submit a new spec to fix the job.




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] SteNicholas commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
SteNicholas commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r821437948



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java
##########
@@ -113,29 +118,70 @@ public DeleteControl cleanup(FlinkDeployment flinkApp, Context context) {
         Configuration effectiveConfig =
                 FlinkUtils.getEffectiveConfig(flinkApp, defaultConfig.getFlinkConfig());
 
-        boolean readyToReconcile = observer.observe(flinkApp, context, effectiveConfig);
-        if (!readyToReconcile) {
-            return flinkApp.getStatus()
-                    .getJobManagerDeploymentStatus()
-                    .toUpdateControl(flinkApp, operatorConfiguration);
-        }
-
         try {
+            boolean readyToReconcile = observer.observe(flinkApp, context, effectiveConfig);
+            if (!readyToReconcile) {
+                return flinkApp.getStatus()
+                        .getJobManagerDeploymentStatus()
+                        .toUpdateControl(flinkApp, operatorConfiguration);
+            }
+
             UpdateControl<FlinkDeployment> updateControl =
                     reconcilerFactory
                             .getOrCreate(flinkApp)
                             .reconcile(operatorNamespace, flinkApp, context, effectiveConfig);
             updateForReconciliationSuccess(flinkApp);
             return updateControl;
-        } catch (InvalidDeploymentException ide) {
-            LOG.error("Reconciliation failed", ide);
-            updateForReconciliationError(flinkApp, ide.getMessage());
+        } catch (DeploymentFailedException dfe) {
+            updateForDeploymentFailed(flinkApp, dfe);

Review comment:
       Why does this remove the error log?




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] SteNicholas commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
SteNicholas commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r821437181



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java
##########
@@ -92,8 +94,11 @@ public DeleteControl cleanup(FlinkDeployment flinkApp, Context context) {
         LOG.info("Stopping cluster {}", flinkApp.getMetadata().getName());
         Configuration effectiveConfig =
                 FlinkUtils.getEffectiveConfig(flinkApp, defaultConfig.getFlinkConfig());
-
-        observer.observe(flinkApp, context, effectiveConfig);
+        try {
+            observer.observe(flinkApp, context, effectiveConfig);
+        } catch (DeploymentFailedException dfe) {

Review comment:
       Does this need to log the exception for the failure of deployment?




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] tweise commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r822105691



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobManagerDeploymentStatus.java
##########
@@ -37,7 +37,11 @@
     DEPLOYING,
 
     /** JobManager deployment not found, probably not started or killed by user. */
-    MISSING;
+    // TODO: currently a mix of SUSPENDED and ERROR, needs cleanup

Review comment:
       I think yes, but outside of this PR. We should get rid of MISSING also.




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] asfgit closed pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46


   


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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] tweise commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r822320677



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/controller/FlinkDeploymentController.java
##########
@@ -113,29 +118,70 @@ public DeleteControl cleanup(FlinkDeployment flinkApp, Context context) {
         Configuration effectiveConfig =
                 FlinkUtils.getEffectiveConfig(flinkApp, defaultConfig.getFlinkConfig());
 
-        boolean readyToReconcile = observer.observe(flinkApp, context, effectiveConfig);
-        if (!readyToReconcile) {
-            return flinkApp.getStatus()
-                    .getJobManagerDeploymentStatus()
-                    .toUpdateControl(flinkApp, operatorConfiguration);
-        }
-
         try {
+            boolean readyToReconcile = observer.observe(flinkApp, context, effectiveConfig);
+            if (!readyToReconcile) {
+                return flinkApp.getStatus()
+                        .getJobManagerDeploymentStatus()
+                        .toUpdateControl(flinkApp, operatorConfiguration);
+            }
+
             UpdateControl<FlinkDeployment> updateControl =
                     reconcilerFactory
                             .getOrCreate(flinkApp)
                             .reconcile(operatorNamespace, flinkApp, context, effectiveConfig);
             updateForReconciliationSuccess(flinkApp);
             return updateControl;
-        } catch (InvalidDeploymentException ide) {
-            LOG.error("Reconciliation failed", ide);
-            updateForReconciliationError(flinkApp, ide.getMessage());
+        } catch (DeploymentFailedException dfe) {
+            updateForDeploymentFailed(flinkApp, dfe);
             return UpdateControl.updateStatus(flinkApp);
         } catch (Exception e) {
             throw new ReconciliationException(e);
         }
     }
 
+    private void updateForDeploymentFailed(

Review comment:
       renamed




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] tweise commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
tweise commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r822318702



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobManagerDeploymentStatus.java
##########
@@ -54,8 +58,9 @@
                                 operatorConfiguration.getPortCheckIntervalInSec(),
                                 TimeUnit.SECONDS);
             case MISSING:
+            case ERROR:
             default:
-                return null;
+                return UpdateControl.noUpdate();

Review comment:
       seems to work as expected




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-kubernetes-operator] SteNicholas commented on a change in pull request #46: [FLINK-26261] Introduce status and create event for terminal deployment error

Posted by GitBox <gi...@apache.org>.
SteNicholas commented on a change in pull request #46:
URL: https://github.com/apache/flink-kubernetes-operator/pull/46#discussion_r821442452



##########
File path: flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/observer/JobManagerDeploymentStatus.java
##########
@@ -37,7 +37,11 @@
     DEPLOYING,
 
     /** JobManager deployment not found, probably not started or killed by user. */
-    MISSING;
+    // TODO: currently a mix of SUSPENDED and ERROR, needs cleanup
+    MISSING,
+
+    /** Deployment in terminal error, requires spec change for reconciliation to continue. */
+    ERROR;

Review comment:
       Should the `Observer#isReadyToReconcile` add the `ERROR` case branch?




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

To unsubscribe, e-mail: commits-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org