[ 
https://issues.apache.org/jira/browse/FLINK-38290?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Urs Schoenenberger updated FLINK-38290:
---------------------------------------
    Description: 
Hi folks,

we are encountering the following issue, and I believe it's a bug or a missing 
feature.

Steps to reproduce:
 * Deploy the example FlinkDeployment ( 
[https://raw.githubusercontent.com/apache/flink-kubernetes-operator/release-1.12/examples/basic.yaml]
 ) with a bounded job (e.g. examples/streaming/WordCount.jar) and configure 
high-availability.type: "kubernetes" and a high-availability.storageDir.
 * Wait for the FlinkDeployment to reach FINISHED.
 * Kill the JobManager pod. (The way this happens in production use cases is 
e.g. if a node is tainted and scheduled for deletion due to being underused / a 
spot instance goes down / etc).

Observed behaviour:
 * A new JobManager is started. 
 * The new pod checks the HA dir and realizes that the job is already 
completed. Log from StandaloneDispatcher: "Ignoring JobGraph submission (...) 
because the job already reached a globally-terminal state (...).
 * The operator tries to reconcile the job. In JobStatusObserver, it queries 
the JobManager's REST API (/jobs/overview), but it receives a "not found".
 ** This is because the backend here does not check the HA store, but the 
JobStore instead. This is backed by RAM or a local file, so it is not recovered 
on JM restart.
 * This leads the k8s operator to believe something is wrong with the 
FlinkDeployment, and the FlinkDeployment goes back to state RECONCILING and 
gets stuck there.

 

This messes with monitoring and alerting among other things. Am I missing a 
particular piece of configuration to make this work?

  was:
Hi folks,

we are encountering the following issue, and I believe it's a bug or a missing 
feature.

Steps to reproduce:
 * Deploy the example FlinkDeployment ( 
[https://raw.githubusercontent.com/apache/flink-kubernetes-operator/release-1.12/examples/basic.yaml]
 ) with a bounded job (e.g. examples/streaming/WordCount.jar) and configure 
high-availability.type: "kubernetes" and a high-availability.storageDir.
 * Wait for the FlinkDeployment to reach FINISHED.
 * Kill the JobManager pod.

Observed behaviour:
 * A new JobManager is started. 
 * The new pod checks the HA dir and realizes that the job is already 
completed. Log from StandaloneDispatcher: "Ignoring JobGraph submission (...) 
because the job already reached a globally-terminal state (...).
 * The operator tries to reconcile the job. In JobStatusObserver, it queries 
the JobManager's REST API (/jobs/overview), but it receives a "not found".
 ** This is because the backend here does not check the HA store, but the 
JobStore instead. This is backed by RAM or a local file, so it is not recovered 
on JM restart.
 * This leads the k8s operator to believe something is wrong with the 
FlinkDeployment, and the FlinkDeployment goes back to state RECONCILING and 
gets stuck there.

 

This messes with monitoring and alerting among other things. Am I missing a 
particular piece of configuration to make this work?


> Application cluster: FINISHED FlinkDeployment falls back to RECONCILING if JM 
> restarts
> --------------------------------------------------------------------------------------
>
>                 Key: FLINK-38290
>                 URL: https://issues.apache.org/jira/browse/FLINK-38290
>             Project: Flink
>          Issue Type: Bug
>          Components: Client / Job Submission, Deployment / Kubernetes
>    Affects Versions: 1.20.1, kubernetes-operator-1.12.1
>            Reporter: Urs Schoenenberger
>            Priority: Major
>
> Hi folks,
> we are encountering the following issue, and I believe it's a bug or a 
> missing feature.
> Steps to reproduce:
>  * Deploy the example FlinkDeployment ( 
> [https://raw.githubusercontent.com/apache/flink-kubernetes-operator/release-1.12/examples/basic.yaml]
>  ) with a bounded job (e.g. examples/streaming/WordCount.jar) and configure 
> high-availability.type: "kubernetes" and a high-availability.storageDir.
>  * Wait for the FlinkDeployment to reach FINISHED.
>  * Kill the JobManager pod. (The way this happens in production use cases is 
> e.g. if a node is tainted and scheduled for deletion due to being underused / 
> a spot instance goes down / etc).
> Observed behaviour:
>  * A new JobManager is started. 
>  * The new pod checks the HA dir and realizes that the job is already 
> completed. Log from StandaloneDispatcher: "Ignoring JobGraph submission (...) 
> because the job already reached a globally-terminal state (...).
>  * The operator tries to reconcile the job. In JobStatusObserver, it queries 
> the JobManager's REST API (/jobs/overview), but it receives a "not found".
>  ** This is because the backend here does not check the HA store, but the 
> JobStore instead. This is backed by RAM or a local file, so it is not 
> recovered on JM restart.
>  * This leads the k8s operator to believe something is wrong with the 
> FlinkDeployment, and the FlinkDeployment goes back to state RECONCILING and 
> gets stuck there.
>  
> This messes with monitoring and alerting among other things. Am I missing a 
> particular piece of configuration to make this work?



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to