Hi,

We have faced a similar issue with flink kubernetes operator, having
multiple operators running as leader at the same time.
On our side the issue was on the java-operator-sdk which was not well
killing operator that lost leadership (
https://github.com/operator-framework/java-operator-sdk/issues/2056).
The issue has been solved in java-operator-sdk 4.4.4 and version has been
bumped in flink kubernetes operator (
https://issues.apache.org/jira/browse/FLINK-33125/https://github.com/apache/flink-kubernetes-operator/pull/680
).
But this patch will probably only provided on flink kubernetes operator
1.17 so I would recommend not to rely on multiple operators or patch locally

Nicolas

On Fri, Nov 3, 2023 at 9:57 AM Evgeniy Lyutikov <eblyuti...@avito.ru> wrote:

> Hello!
> I constantly get a similar error when operator (working in single
> instance) receiving deployment statuses
> Details described in this message
> https://lists.apache.org/thread/0odcc9pvlpz1x9y2nop9dlmcnp9v1696
> I tried changing versions and allocated resources, as well as the number
> of reconcile threads, but nothing helped
>
> ------------------------------
> *От:* Tony Chen <tony.ch...@robinhood.com>
> *Отправлено:* 3 ноября 2023 г. 9:13:51
> *Кому:* user@flink.apache.org
> *Копия:* Nathan Moderwell
> *Тема:* Re: flink-kubernetes-operator cannot handle SPECCHANGE for 100+
> FlinkDeployments concurrently
>
> One of the operator pods logged the following exception before the
> container restarted:
>
> �[m�[33m2023-11-01 14:24:21,260�[m �[36mo.a.f.s.n.i.n.c.AbstractChannel�[m
> �[33m[WARN ] Force-closing a channel whose registration task was not
> accepted by an event loop: [id: 0x1a7718c1]
> java.util.concurrent.RejectedExecutionException: event executor terminated
>
> I did notice that all of our 3 operator pods were reconciling
> FlinkDeployments, and this definitely is an issue. After I churned 2 of the
> pods, there was only 1 pod that was the leader, and this operator pod was
> able to reconcile SPECCHANGES of FlinkDeployments again.
>
> Are there any recommendations on how I can enforce only 1 pod to be the
> leader? For example, would increasing the lease-duration help?
>
>
> https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-main/docs/operations/configuration/
> <https://eur04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fnightlies.apache.org%2Fflink%2Fflink-kubernetes-operator-docs-main%2Fdocs%2Foperations%2Fconfiguration%2F&data=05%7C01%7Ceblyutikov%40avito.ru%7C9be0eed45d3643af4c3d08dbdc12b1cd%7Caf0e07b3b90b472392e63fab11dd5396%7C0%7C0%7C638345745082130170%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C3000%7C%7C%7C&sdata=Rrhw00SUQD3sOYWi17gjx8M9gGHe8JgrkPY9%2FswNVXk%3D&reserved=0>
>
> On Wed, Nov 1, 2023 at 11:16 PM Tony Chen <tony.ch...@robinhood.com>
> wrote:
>
>> Hi Flink Community,
>>
>> I am currently running flink-kubernetes-operator 1.6-patched (
>> https://github.com/apache/flink-kubernetes-operator/commit/3f0dc2ee5534084bc162e6deaded36e93bb5e384
>> <https://eur04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fgithub.com%2Fapache%2Fflink-kubernetes-operator%2Fcommit%2F3f0dc2ee5534084bc162e6deaded36e93bb5e384&data=05%7C01%7Ceblyutikov%40avito.ru%7C9be0eed45d3643af4c3d08dbdc12b1cd%7Caf0e07b3b90b472392e63fab11dd5396%7C0%7C0%7C638345745082130170%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C3000%7C%7C%7C&sdata=OEDCPJVv6wgz6U3PtBVjEVEDQfovjIPAaTxXKJ1GMBU%3D&reserved=0>),
>> and I have 3 flink-kubernetes-operator pods running. Recently, I deployed
>> around 110 new FlinkDeployments, and I had no issues with this initial
>> deployment. However, when I applied changes to all of these 110 new
>> FlinkDeployments concurrently to update their container image, the
>> flink-kubernetes-operator pods seemed to be in conflict with each other
>> constantly.
>>
>> For example, before the SPECCHANGE, FlinkDeployment rh-flinkdeployment-01
>> would be RUNNING (status.jobStatus.state) and STABLE
>> (status.lifecycleState). After the FlinkDeployment spec is updated,
>> rh-flinkdeployment-01 goes through FINISHED (status.jobStatus.state) and
>> UPGRADING (status.jobStatus.state), and then RECONCILING
>> (status.jobStatus.state) and DEPLOYED (status.jobStatus.state). It reaches
>> RUNNING and STABLE again, but then for some reason it goes back to FINISHED
>> and UPGRADING again, and I do notice that the newly created jobmanager pod
>> gets deleted and then recreated. rh-flinkdeployment-01 basically becomes
>> stuck in this loop where it becomes stable and then gets re-deployed by the
>> operator.
>>
>> This doesn't happen to all 110 FlinkDeployments, but it happens to around
>> 30 of them concurrently.
>>
>> I have pasted some logs from one of the operator pods on one of the
>> FlinkDeployments. I have also highlighted messages that seem suspicious to
>> me. I will try to gather more logs and send them tomorrow.
>>
>> For now, to mitigate this, I had to delete all of these FlinkDeployments
>> and run them with the deprecated GoogleCloudPlatform operator. I'm hoping
>> to resolve this soon so that I don't have to run anything on the
>> GoogleCloudPlatform operator anymore.
>>
>> Thanks!
>> Tony
>>
>>
>> �[m�[33m2023-11-02 05:26:02,132�[m
>> �[36mi.j.o.p.e.ReconciliationDispatcher�[m
>> �[1;31m[ERROR][<namespace>/<flinkdeployment>] Error during event processing
>> ExecutionScope{ resource id: ResourceID{name='<flinkdeployment',
>> namespace='<namespace>'}, version: 17772349729} failed.
>> org.apache.flink.kubernetes.operator.exception.ReconciliationException:
>> org.apache.flink.kubernetes.operator.exception.StatusConflictException:
>> Status have been modified externally in version 17772349851 Previous:
>> <REDACTED>
>> ...
>> 2023-11-02 05:27:25,945 o.a.f.k.o.o.d.ApplicationObserver [WARN
>> ][<namespace>/<flinkdeployment>] *Running deployment generation -1
>> doesn't match upgrade target generation 2.*
>> 2023-11-02 05:27:25,946 o.a.f.c.Configuration          [WARN
>> ][<namespace>/<flinkdeployment>] Config uses deprecated configuration key
>> 'high-availability' instead of proper key 'high-availability.type'
>> 2023-11-02 05:27:26,034 o.a.f.k.o.l.AuditUtils         [INFO
>> ][<namespace>/<flinkdeployment>] >>> Status | Info    | UPGRADING       |
>> The resource is being upgraded
>> 2023-11-02 05:27:26,057 o.a.f.k.o.l.AuditUtils         [INFO
>> ][<namespace>/<flinkdeployment>] >>> Event  | Info    | SUBMIT          |
>> Starting deployment
>> 2023-11-02 05:27:26,057 o.a.f.k.o.s.AbstractFlinkService [INFO
>> ][<namespace>/<flinkdeployment>] Deploying application cluster requiring
>> last-state from HA metadata
>> 2023-11-02 05:27:26,057 o.a.f.c.Configuration          [WARN
>> ][<namespace>/<flinkdeployment>] Config uses deprecated configuration key
>> 'high-availability' instead of proper key 'high-availability.type'
>> 2023-11-02 05:27:26,084 o.a.f.c.Configuration          [WARN
>> ][<namespace>/<flinkdeployment>] Config uses deprecated configuration key
>> 'high-availability' instead of proper key 'high-availability.type'
>> 2023-11-02 05:27:26,110 o.a.f.k.o.s.NativeFlinkService [INFO
>> ][<namespace>/<flinkdeployment>] Deploying application cluster
>> 2023-11-02 05:27:26,110 o.a.f.c.d.a.c.ApplicationClusterDeployer [INFO
>> ][<namespace>/<flinkdeployment>] Submitting application in 'Application
>> Mode'.
>> 2023-11-02 05:27:26,112 o.a.f.r.u.c.m.ProcessMemoryUtils [INFO
>> ][<namespace>/<flinkdeployment>] The derived from fraction jvm overhead
>> memory (1.000gb (1073741840 bytes)) is greater than its max value
>> 1024.000mb (1073741824 bytes), max value will be used instead
>> 2023-11-02 05:27:26,112 o.a.f.r.u.c.m.ProcessMemoryUtils [INFO
>> ][<namespace>/<flinkdeployment>] The derived from fraction jvm overhead
>> memory (1.000gb (1073741840 bytes)) is greater than its max value
>> 1024.000mb (1073741824 bytes), max value will be used instead
>> 2023-11-02 05:27:26,163 o.a.f.k.o.s.AbstractFlinkService [INFO
>> ][<namespace>/<flinkdeployment>] Waiting for cluster shutdown... (30s)
>> 2023-11-02 05:27:26,193 o.a.f.k.o.l.AuditUtils         [INFO
>> ][<namespace>/<flinkdeployment>] >>> Event  | Warning | 
>> *CLUSTERDEPLOYMENTEXCEPTION
>> | The Flink cluster <flinkdeployment> already exists.*
>> 2023-11-02 05:27:26,193 o.a.f.k.o.r.ReconciliationUtils [WARN
>> ][<namespace>/<flinkdeployment>] Attempt count: 0, last attempt: false
>> 2023-11-02 05:27:26,277 o.a.f.k.o.l.AuditUtils         [INFO
>> ][<namespace>/<flinkdeployment>] *>>> Status | Error   | UPGRADING
>> |
>> {"type":"org.apache.flink.kubernetes.operator.exception.ReconciliationException","message":"org.apache.flink.client.deployment.ClusterDeploymentException:
>> The Flink cluster <flinkdeployment> already
>> exists.","additionalMetadata":{},"throwableList":[{"type":"org.apache.flink.client.deployment.ClusterDeploymentException","message":"The
>> Flink cluster <flinkdeployment> already exists.","additionalMetadata":{}}]}*
>>
>>
>> --
>>
>>
>> <https://eur04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fwww.robinhood.com%2F&data=05%7C01%7Ceblyutikov%40avito.ru%7C9be0eed45d3643af4c3d08dbdc12b1cd%7Caf0e07b3b90b472392e63fab11dd5396%7C0%7C0%7C638345745082130170%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C3000%7C%7C%7C&sdata=DWEKglkxTzWg7huDPHhPWNgfkEUMfU5N3%2BM9CndI9QA%3D&reserved=0>
>>
>> Tony Chen
>>
>> Software Engineer
>>
>> Menlo Park, CA
>>
>> Don't copy, share, or use this email without permission. If you received
>> it by accident, please let us know and then delete it right away.
>>
>
>
> --
>
>
> <https://eur04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fwww.robinhood.com%2F&data=05%7C01%7Ceblyutikov%40avito.ru%7C9be0eed45d3643af4c3d08dbdc12b1cd%7Caf0e07b3b90b472392e63fab11dd5396%7C0%7C0%7C638345745082130170%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C3000%7C%7C%7C&sdata=DWEKglkxTzWg7huDPHhPWNgfkEUMfU5N3%2BM9CndI9QA%3D&reserved=0>
>
> Tony Chen
>
> Software Engineer
>
> Menlo Park, CA
>
> Don't copy, share, or use this email without permission. If you received
> it by accident, please let us know and then delete it right away.
>
> * ------------------------------ *“This message contains confidential
> information/commercial secret. If you are not the intended addressee of
> this message you may not copy, save, print or forward it to any third party
> and you are kindly requested to destroy this message and notify the sender
> thereof by email.
> Данное сообщение содержит конфиденциальную информацию/информацию,
> являющуюся коммерческой тайной. Если Вы не являетесь надлежащим адресатом
> данного сообщения, Вы не вправе копировать, сохранять, печатать или
> пересылать его каким либо иным лицам. Просьба уничтожить данное сообщение и
> уведомить об этом отправителя электронным письмом.”
>

Reply via email to