[ 
https://issues.apache.org/jira/browse/FLINK-25098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17451888#comment-17451888
 ] 

Neeraj Laad edited comment on FLINK-25098 at 12/1/21, 3:24 PM:
---------------------------------------------------------------

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 2 - quickstart-avro (acbd48dfe523a96ab8710396c3c9bf27)* {*}- this one 
fails to recover{*}{*}{{*}}
{quote}{{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-0000000000000002372. 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)}}
{quote}
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]}}


was (Author: neeraj.laad):
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 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-0000000000000002372. 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///<dir>{}}}.
>  * 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&page=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)

Reply via email to