Hey Yang, Thank you for fast response.
I get your point but, assuming 3 Job managers are up, in case the leader fails, one of the other 2 should become the new leader, no? If the cluster fails, the new leader should handle that. Another scenario could be that the Job manager stops(get killed by k8s due to memory, CPU limitations, bugs etc...) while TMs are still operating, and the cluster is active. In some cases, due to resources limitation, k8s will not be able to get a new instance right away, until auto-scale takes place(The pod remains in pending state). It seems like we do achieve resilience by having HA enabled in Native k8s mode. What do you think? Given that you are running multiple JobManagers, it does not matter for the "already exists" exception during leader election. Should we ignore such error? if so , it should be a warning then What about the 1st error we encountered regarding the kube/config file exception? Thank you so much, Best, Tamir ________________________________ From: Yang Wang <danrtsey...@gmail.com> Sent: Thursday, September 8, 2022 7:08 AM To: Tamir Sagi <tamir.s...@niceactimize.com> Cc: user@flink.apache.org <user@flink.apache.org>; Lihi Peretz <lihi.per...@niceactimize.com> Subject: Re: [Flink 1.15.1 - Application mode native k8s Exception] - Exception occurred while acquiring lock 'ConfigMapLock EXTERNAL EMAIL Given that you are running multiple JobManagers, it does not matter for the "already exists" exception during leader election. BTW, I think running multiple JobManagers does not take enough advantages when deploying Flink on Kubernetes. Because a new JobManager will be started immediately once the old one crashed. And Flink JobManager always needs to recover the job from the latest checkpoint no matter how many JobManager are running. Best, Yang Tamir Sagi <tamir.s...@niceactimize.com<mailto:tamir.s...@niceactimize.com>> 于2022年9月5日周一 21:48写道: Hey Yang, The flink-conf.yaml submitted to the cluster does not contain "kubernetes.config.file" at all. In addition, I verified flink config maps under cluster's namespace do not contain "kubernetes.config.file". In addition, we also noticed the following exception (appears to happen sporadically) 2022-09-04T21:06:35,231][Error] {} [i.f.k.c.e.l.LeaderElector]: Exception occurred while acquiring lock 'ConfigMapLock: dev-0-flink-jobs - data-agg-events-insertion-cluster-config-map (fa3dbbc5-1753-46cd-afaf-0baf8ff0947f)' io.fabric8.kubernetes.client.extended.leaderelection.resourcelock.LockException: Unable to create ConfigMapLock Caused by: io.fabric8.kubernetes.client.KubernetesClientException: Failure executing: POST at: https://172.20.0.1/api/v1/namespaces/dev-0-flink-jobs/configmaps. Message: configmaps "data-agg-events-insertion-cluster-config-map" already exists. Log file is enclosed. Thanks, Tamir. ________________________________ From: Yang Wang <danrtsey...@gmail.com<mailto:danrtsey...@gmail.com>> Sent: Monday, September 5, 2022 3:03 PM To: Tamir Sagi <tamir.s...@niceactimize.com<mailto:tamir.s...@niceactimize.com>> Cc: user@flink.apache.org<mailto:user@flink.apache.org> <user@flink.apache.org<mailto:user@flink.apache.org>>; Lihi Peretz <lihi.per...@niceactimize.com<mailto:lihi.per...@niceactimize.com>> Subject: Re: [Flink 1.15.1 - Application mode native k8s Exception] - Exception occurred while acquiring lock 'ConfigMapLock EXTERNAL EMAIL Could you please check whether the "kubernetes.config.file" is configured to /opt/flink/.kube/config in the Flink configmap? It should be removed before creating the Flink configmap. Best, Yang Tamir Sagi <tamir.s...@niceactimize.com<mailto:tamir.s...@niceactimize.com>> 于2022年9月4日周日 18:08写道: Hey All, We recently updated to Flink 1.15.1. We deploy stream cluster in Application mode in Native K8S.(Deployed on Amazon EKS). The cluster is configured with Kubernetes HA Service, Minimum 3 replicas of Job manager and pod-template which is configured with topologySpreadConstraints to enable distribution across different availability zones. HA storage directory is on S3. The cluster is deployed and running properly, however, after a while we noticed the following exception in Job manager instance(the log file is enclosed) 2022-09-04T02:05:33,097][Error] {} [i.f.k.c.e.l.LeaderElector]: Exception occurred while acquiring lock 'ConfigMapLock: dev-0-flink-jobs - data-agg-events-insertion-cluster-config-map (b6da2ae2-ad2b-471c-801e-ea460a348fab)' io.fabric8.kubernetes.client.KubernetesClientException: Operation: [get] for kind: [ConfigMap] with name: [data-agg-events-insertion-cluster-config-map] in namespace: [dev-0-flink-jobs] failed. Caused by: java.io.FileNotFoundException: /opt/flink/.kube/config (No such file or directory) at java.io.FileInputStream.open0(Native Method) ~[?:?] at java.io.FileInputStream.open(Unknown Source) ~[?:?] at java.io.FileInputStream.<init>(Unknown Source) ~[?:?] at org.apache.flink.kubernetes.shaded.com.fasterxml.jackson.dataformat.yaml.YAMLFactory.createParser(YAMLFactory.java:354) ~[flink-dist-1.15.1.jar:1.15.1] at org.apache.flink.kubernetes.shaded.com.fasterxml.jackson.dataformat.yaml.YAMLFactory.createParser(YAMLFactory.java:15) ~[flink-dist-1.15.1.jar:1.15.1] at org.apache.flink.kubernetes.shaded.com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3494) ~[flink-dist-1.15.1.jar:1.15.1] at io.fabric8.kubernetes.client.internal.KubeConfigUtils.parseConfig(KubeConfigUtils.java:42) ~[flink-dist-1.15.1.jar:1.15.1] at io.fabric8.kubernetes.client.utils.TokenRefreshInterceptor.intercept(TokenRefreshInterceptor.java:44) ~[flink-dist-1.15.1.jar:1.15.1] at org.apache.flink.kubernetes.shaded.okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:142) ~[flink-dist-1.15.1.jar:1.15.1] at org.apache.flink.kubernetes.shaded.okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:117) ~[flink-dist-1.15.1.jar:1.15.1] at io.fabric8.kubernetes.client.utils.ImpersonatorInterceptor.intercept(ImpersonatorInterceptor.java:68) ~[flink-dist-1.15.1.jar:1.15.1] at org.apache.flink.kubernetes.shaded.okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:142) ~[flink-dist-1.15.1.jar:1.15.1] at org.apache.flink.kubernetes.shaded.okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:117) ~[flink-dist-1.15.1.jar:1.15.1] at io.fabric8.kubernetes.client.utils.HttpClientUtils.lambda$createApplicableInterceptors$6(HttpClientUtils.java:290) ~[flink-dist-1.15.1.jar:1.15.1] at org.apache.flink.kubernetes.shaded.okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:142) ~[flink-dist-1.15.1.jar:1.15.1] at org.apache.flink.kubernetes.shaded.okhttp3.internal.http.RealInterceptorChain.proceed(RealInterceptorChain.java:117) ~[flink-dist-1.15.1.jar:1.15.1] at org.apache.flink.kubernetes.shaded.okhttp3.RealCall.getResponseWithInterceptorChain(RealCall.java:229) ~[flink-dist-1.15.1.jar:1.15.1] at org.apache.flink.kubernetes.shaded.okhttp3.RealCall.execute(RealCall.java:81) ~[flink-dist-1.15.1.jar:1.15.1] at io.fabric8.kubernetes.client.dsl.base.OperationSupport.retryWithExponentialBackoff(OperationSupport.java:585) ~[flink-dist-1.15.1.jar:1.15.1] at io.fabric8.kubernetes.client.dsl.base.OperationSupport.handleResponse(OperationSupport.java:558) ~[flink-dist-1.15.1.jar:1.15.1] at io.fabric8.kubernetes.client.dsl.base.OperationSupport.handleResponse(OperationSupport.java:521) ~[flink-dist-1.15.1.jar:1.15.1] at io.fabric8.kubernetes.client.dsl.base.OperationSupport.handleGet(OperationSupport.java:488) ~[flink-dist-1.15.1.jar:1.15.1] at io.fabric8.kubernetes.client.dsl.base.OperationSupport.handleGet(OperationSupport.java:470) ~[flink-dist-1.15.1.jar:1.15.1] at io.fabric8.kubernetes.client.dsl.base.BaseOperation.handleGet(BaseOperation.java:830) ~[flink-dist-1.15.1.jar:1.15.1] at io.fabric8.kubernetes.client.dsl.base.BaseOperation.getMandatory(BaseOperation.java:200) ~[flink-dist-1.15.1.jar:1.15.1] ... 12 more Why is Kube/config needed in Native K8s, should not service account be checked instead? Are we missing something? Thanks, Tamir. Confidentiality: This communication and any attachments are intended for the above-named persons only and may be confidential and/or legally privileged. Any opinions expressed in this communication are not necessarily those of NICE Actimize. If this communication has come to you in error you must take no action based on it, nor must you copy or show it to anyone; please delete/destroy and inform the sender by e-mail immediately. Monitoring: NICE Actimize may monitor incoming and outgoing e-mails. Viruses: Although we have taken steps toward ensuring that this e-mail and attachments are free from any virus, we advise that in keeping with good computing practice the recipient should ensure they are actually virus free. Confidentiality: This communication and any attachments are intended for the above-named persons only and may be confidential and/or legally privileged. Any opinions expressed in this communication are not necessarily those of NICE Actimize. If this communication has come to you in error you must take no action based on it, nor must you copy or show it to anyone; please delete/destroy and inform the sender by e-mail immediately. Monitoring: NICE Actimize may monitor incoming and outgoing e-mails. Viruses: Although we have taken steps toward ensuring that this e-mail and attachments are free from any virus, we advise that in keeping with good computing practice the recipient should ensure they are actually virus free. Confidentiality: This communication and any attachments are intended for the above-named persons only and may be confidential and/or legally privileged. Any opinions expressed in this communication are not necessarily those of NICE Actimize. If this communication has come to you in error you must take no action based on it, nor must you copy or show it to anyone; please delete/destroy and inform the sender by e-mail immediately. Monitoring: NICE Actimize may monitor incoming and outgoing e-mails. Viruses: Although we have taken steps toward ensuring that this e-mail and attachments are free from any virus, we advise that in keeping with good computing practice the recipient should ensure they are actually virus free.