[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17562099#comment-17562099 ] Neelishaa Srivastava commented on FLINK-25098: -- Hi Till Rohrmann , can you please guide us to the similar issue jobmanager pod is stuck in CrashLoopBackOff state seen .The logs are already mentioned by MAU CHEE YEN on the above comment . > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: JM-FlinkException-checkpointHA.txt, > flink_checkpoint_issue.txt, > iaf-insights-engine--7fc4-eve-29ee-ep-jobmanager-1-jobmanager.log, > jm-flink-ha-jobmanager-log.txt, jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17560986#comment-17560986 ] MAU CHEE YEN commented on FLINK-25098: -- [^flink_checkpoint_issue.txt] > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: JM-FlinkException-checkpointHA.txt, > flink_checkpoint_issue.txt, > iaf-insights-engine--7fc4-eve-29ee-ep-jobmanager-1-jobmanager.log, > jm-flink-ha-jobmanager-log.txt, jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17479958#comment-17479958 ] Enrique Lacal commented on FLINK-25098: --- Hi [~trohrmann] , I didn't manage to replicate the above error and even tried with S3 back in December and left it running for a long period of time and it worked. Yes, we use StatefulSets for deploying the Flink JMs. We have found a similar issue to the above that is replicated constantly, here are the logs [^JM-FlinkException-checkpointHA.txt] . We have a manual workaround of deleting the affected HA ConfigMap which points to this checkpoint, but this is not feasible in a production environment. Would really appreciate any thoughts on this, and what sort of solution we could come to. Let me know if you need any more information, I'm trying to get the logs before this occurred. Thanks, Enrique > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: JM-FlinkException-checkpointHA.txt, > iaf-insights-engine--7fc4-eve-29ee-ep-jobmanager-1-jobmanager.log, > jm-flink-ha-jobmanager-log.txt, jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17452472#comment-17452472 ] Till Rohrmann commented on FLINK-25098: --- Hi [~EnriqueL8], Flink first persists the {{CompletedCheckpoint}} before writing a pointer into the ConfigMap. This is what happens in the [KubernetesStateHandleStore#addAndLock|https://github.com/apache/flink/blob/master/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java#L126]. The implementation should also fsync the created files so that the data is really persisted. The thing I am wondering is whether you could try it out w/o using the ReadWriteMany PV and instead use S3. I just want to make sure that this is not caused by some weird consistency guarantees coming from the used PV. How do you deploy the Flink JobManagers? I assume that you are using a StatefulSet for this, right? Let me know, once you have the logs of a failed run. I am really interested in understanding what's going wrong. > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: > iaf-insights-engine--7fc4-eve-29ee-ep-jobmanager-1-jobmanager.log, > jm-flink-ha-jobmanager-log.txt, jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17452441#comment-17452441 ] Enrique Lacal commented on FLINK-25098: --- Hi Till, I want to differentiate between a reinstall and this occurring whilst the Flink Cluster is running. I agree that to uninstall everything it is safer to manually delete the ConfigMaps and this is what Adrian has seen above with them pointing to an outdated job graph. Deleting the ConfigMaps solve this issue, but not the latter. The other problem we are seeing is that whilst a Flink Cluster is running the Configmap for one of the jobs becomes inconsistent for some reason and when the leader JM goes down and a new follower tries to restore that job using the ConfigMap it cannot find the checkpoint referenced. (This is what Neeraj has shared through the logs) I've done some investigation and it seems that Flink updates the ConfigMap before creating the `completedCheckpoint` folder in the dir set by `high-availability.storageDir`. I watched the ConfigMap and the Fs simultaneously. My assumption is that before the `completedCheckpoint` is written but after the ConfigMap is updated the leader JM goes down for some reason and the state becomes inconsistent. Then Flink cannot recover from this state without manual intervention, which is a significant problem. Another idea might be that the checkpoint fails but the CM is prematurely updated. I think this is less likely. I understand you want to see some logs on when the checkpoint fails to understand the root cause, so we have set up persistence for our logs and will share those as soon as we can reproduce this issue. I also couldn't find a way of reproducing this by crashing the job, killing the leader pod etc.. and since the interval between the CM being updated and then the file in FS being created is too short it's hard to crash the pod at a specific time. From my observation after the Flink Cluster is in this unrecoverable state, the actual checkpoint is stored in `state.checkpoints.dir` such as `chk-` but the `completeCheckpoint` doesn't exist which mean that the checkpoint has been taken correctly but the reference to it is not there. I believe this is the way it works, but not 100% sure. Just in case, these are the parameters used for checkpointing: |Checkpointing Mode|Exactly Once| |Checkpoint Storage|FileSystemCheckpointStorage| |State Backend|EmbeddedRocksDBStateBackend| |Interval|5s| |Timeout|10m 0s| |Minimum Pause Between Checkpoints|0ms| |Maximum Concurrent Checkpoints|1| |Unaligned Checkpoints|Disabled| |Persist Checkpoints Externally|Enabled (retain on cancellation)| |Tolerable Failed Checkpoints|0| Do you think there is a workaround for this issue? Maybe changing the above configuration to be less strict? Thanks for your time! > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: > iaf-insights-engine--7fc4-eve-29ee-ep-jobmanager-1-jobmanager.log, > jm-flink-ha-jobmanager-log.txt, jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17451929#comment-17451929 ] Till Rohrmann commented on FLINK-25098: --- Deleting a K8s deployment should effectively lead to killing the Flink process with a {{SIGTERM}} signal that runs in the pods that get terminated. The problem is that Flink cannot distinguish between the failure and the shutdown case because it looks the same. Therefore, we cannot easily tell Flink to clean things up when receiving a {{SIGTERM}}. There is a discussion about adding support for a shutdown command for a Flink cluster that will cancel all running jobs, clean the related HA information up and then shut down. This has not been implemented though. If you want to uninstall everything, then I would suggest to explicitly remove the config maps atm. They should have the label {{high-availability}} and the {{}}. [~neeraj.laad], the reason why there is no checkpoint 2373 might be that it has failed. The logs are from the run that failed. In order to fully debug the problem I would need the logs from the run that produced the checkpoint 2372. > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: > iaf-insights-engine--7fc4-eve-29ee-ep-jobmanager-1-jobmanager.log, > jm-flink-ha-jobmanager-log.txt, jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17451893#comment-17451893 ] Adrian Vasiliu commented on FLINK-25098: [~trohrmann] Again, we are not killing any process with our code. The use-case is: 1. Flink gets deployed in Kubernetes. 2. The user decides to uninstall (then, possibly, reinstall). For that, the K8S way is to delete the K8S custom resource which deployed Flink. => Flink configmaps remain (which, as you point out, is intentional). Thanks for the doc pointer. > The problem is that you are using storage that is not persistent as Flink > would need it to be. Now, [https://nightlies.apache.org/flink/flink-docs-release-1.13/docs/deployment/ha/kubernetes_ha/#high-availability-data-clean-up] says: "To keep HA data while restarting the Flink cluster, simply delete the deployment (via {{{}kubectl delete deployment {}}}). All the Flink cluster related resources will be deleted (e.g. JobManager Deployment, TaskManager pods, services, Flink conf ConfigMap). HA related ConfigMaps will be retained because they do not set the owner reference. When restarting the cluster, all previously running jobs will be recovered and restarted from the latest successful checkpoint." I would think there are two distinct use-cases for uninstallation: 1. The user wants to uninstall, then reinstall while preserving data from the previous install. In this case, per Flink constraint, if persistant storage is enabled, the PV holding it MUST not be removed, otherwise Flink will break at reinstall (as reported here). 2. The user wants a full uninstall, no data left behind, including the persistent volume. Then he may decide to reinstall from scratch. >From your description and from the doc, it looks to me that Flink HA supports >well the first use-case, and not so well the latter. Do I get it well? I would think there should be a way to configure Flink HA to tell whether we want it to do a full cleanup at uninstallation, or not. That's because a typical requirement for uninstalls in enterprise env. is to have nothing left behind... > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: > iaf-insights-engine--7fc4-eve-29ee-ep-jobmanager-1-jobmanager.log, > jm-flink-ha-jobmanager-log.txt, jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17451888#comment-17451888 ] Neeraj Laad commented on FLINK-25098: - There is another scenario where we are seeing this issue where we are not manually deleting/cleaning anything. We deploy Flink with HA (3 JM and 3 TM), persistent store (rook-ceph) and a couple of jobs. All seems to be working fine for a day or so and then we see this situation happening where one of the Flink jobs can not be recovered and this results in constant CrashLoopBackOff for all JMs. Here is some additional information that I gathered from the logs.. *Job 1 - (b070f612e5579029622ed22a3790987a) - this one recovers fine* application-defined checkpoint storage: org.apache.flink.runtime.state.storage.FileSystemCheckpointStorage@{*}a0a2ca3b{*} KubernetesStateHandleStore{configMapName='iaf-insights-engine--7fc4-eve-29ee-ep-{*}b070f612e5579029622ed22a3790987a{*}-jobmanager-leader'}. Found 3 checkpoints - 2627, 2628, 2629 counter in config map is at 3630 *Job 2 - quickstart-avro (acbd48dfe523a96ab8710396c3c9bf27)* {*}- this one fails to recover{*}{*}{*} application-defined checkpoint storage: org.apache.flink.runtime.state.storage.FileSystemCheckpointStorage@{*}c2eaf6cb{*} KubernetesStateHandleStore{configMapName='iaf-insights-engine--7fc4-eve-29ee-ep-{*}acbd48dfe523a96ab8710396c3c9bf27{*}-jobmanager-leader'}. Found 4 checkpoints - 2369, 2370, 2371, 2372 (fails) config maps has counter - 2374 *(where is 2373??)* ERROR: Could not retrieve checkpoint 2372 from state handle under checkpointID-0002372. This indicates that the retrieved state handle is broken. Try cleaning the state handle store. {_}caused by: java.io.FileNotFoundException{_}: /mnt/pv/flink-ha-storage/default/completedCheckpointee7e71900f6c (No such file or directory) One thing which I see is different this time is that the {{/mnt/pv/checkpoints/quickstart-avro/acbd48dfe523a96ab8710396c3c9bf27}} folder does not have {{chk-2372}} folder at all. Full JM logs - {{[^iaf-insights-engine--7fc4-eve-29ee-ep-jobmanager-1-jobmanager.log]}} > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: > iaf-insights-engine--7fc4-eve-29ee-ep-jobmanager-1-jobmanager.log, > jm-flink-ha-jobmanager-log.txt, jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17451817#comment-17451817 ] Till Rohrmann commented on FLINK-25098: --- If you are terminating the cluster before the jobs has properly terminated, then this explains the situation. If the job is not terminated but you are only killing the process, then the job won't be removed from Flink's HA state. Hence, when recovering, Flink assumes that the data is still there. However, the used PVs are cleaned up in the meantime and the data is gone. The problem is that you are using storage that is not persistent as Flink would need it to be. > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: jm-flink-ha-jobmanager-log.txt, > jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17451670#comment-17451670 ] Adrian Vasiliu commented on FLINK-25098: [~trohrmann] > How exactly are you tearing down the initial cluster? AFAIK we just rely on Flink's own tearing down when the removal of the K8S deployment is triggered by the removal of Custom Resource. > When tearing down the initial cluster, are you also deleting the PVC or the > PV? Not explicitly but Kubernetes does remove both PVC and PV (trying to list the PV previously bound to the deleted PVC, we see it doesn't exist anymore, so it couldn't be reused by the new PVC after redeployment. > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: jm-flink-ha-jobmanager-log.txt, > jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17451230#comment-17451230 ] Till Rohrmann commented on FLINK-25098: --- How exactly are you tearing down the initial cluster? That Flink does not delete the HA CMs is by design and documented [here|https://nightlies.apache.org/flink/flink-docs-release-1.13/docs/deployment/ha/kubernetes_ha/#high-availability-data-clean-up]. What should not happen is that killing the Flink cluster deletes the submitted {{JobGraph}} but does not remove the entry from the HA ConfigMap. When tearing down the initial cluster, are you also deleting the PVC or the PV? > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: jm-flink-ha-jobmanager-log.txt, > jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17451072#comment-17451072 ] Adrian Vasiliu commented on FLINK-25098: Yes, since then we also identified the K8S configmaps as being the cause. The scenario is: 1. Flink cluster deployed and receiving a Flink job. All good. 2. Uninstall - all K8S objects go away, except the Flink configmaps. 3. Reinstall => crashloopbackoff. I hear now from colleagues that the issue with Flink CMs being left behind at uninstall time has already been raised on the user list. Your take on it? > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: jm-flink-ha-jobmanager-log.txt, > jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17450976#comment-17450976 ] Chesnay Schepler commented on FLINK-25098: -- >From the logs we can see that the JM tries to recover a job (which it derived >from the K8 config map), but is unable to find the jobgraph on disk in the HA >store. Can you expand a bit on when exactly this occurred? Was the job running fine, then ran into a JM failover, and then this issue occurred? How regularly does it occur? Does it run fine for a few days and then suddenly fail? > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: jm-flink-ha-jobmanager-log.txt, > jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (FLINK-25098) Jobmanager CrashLoopBackOff in HA configuration
[ https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17450968#comment-17450968 ] Till Rohrmann commented on FLINK-25098: --- Thanks for reporting this issue [~adrianalexvasiliu]. From the attached logs I cannot see anything wrong. The JM process cannot read {{/mnt/pv/flink-ha-storage/default/submittedJobGrapha600b0596ee1}} because it does not exist. In order to better understand the problem, I would need the logs of the other JM processes and what happened before {{eventprocessor--7fc4-eve-29ee-ep-jobmanager-0}} took over the leadership. So ideally you could provide the logs for the whole lifetime of the job {{609559678972cbfee4830395f4c47e3f}}. Moreover, it would be great if you could turn on the {{DEBUG}} log level. What you could also try out is whether the same problem occurs when using S3, GCS or HDFS as the persistent storage. What I would like to rule out is that the problem is related to the ReadWriteMany PV using {{rocks-cephfs}}. > Jobmanager CrashLoopBackOff in HA configuration > --- > > Key: FLINK-25098 > URL: https://issues.apache.org/jira/browse/FLINK-25098 > Project: Flink > Issue Type: Bug > Components: Deployment / Kubernetes >Affects Versions: 1.13.2, 1.13.3 > Environment: Reproduced with: > * Persistent jobs storage provided by the rocks-cephfs storage class. > * OpenShift 4.9.5. >Reporter: Adrian Vasiliu >Priority: Critical > Attachments: jm-flink-ha-jobmanager-log.txt, > jm-flink-ha-tls-proxy-log.txt > > > In a Kubernetes deployment of Flink 1.13.2 (also reproduced with Flink > 1.13.3), turning to Flink HA by using 3 replicas of the jobmanager leads to > CrashLoopBackoff for all replicas. > Attaching the full logs of the {{jobmanager}} and {{tls-proxy}} containers of > jobmanager pod: > [^jm-flink-ha-jobmanager-log.txt] > [^jm-flink-ha-tls-proxy-log.txt] > Reproduced with: > * Persistent jobs storage provided by the {{rocks-cephfs}} storage class > (shared by all replicas - ReadWriteMany) and mount path set via > {{{}high-availability.storageDir: file///{}}}. > * OpenShift 4.9.5 and also 4.8.x - reproduced in several clusters, it's not > a "one-shot" trouble. > Remarks: > * This is a follow-up of > https://issues.apache.org/jira/browse/FLINK-22014?focusedCommentId=17450524=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17450524. > > * Picked Critical severity as HA is critical for our product. -- This message was sent by Atlassian Jira (v8.20.1#820001)