Hi Kenzyme,

at the moment Flink will stop the execution of jobs when it loses its
connection to ZooKeeper for whatever reason. If a ZK rolling update can
cause the connection loss to the quorum, then it's what you are seeing.
FLINK-10052 wants to add a feature which allows Flink to tolerate a
SUSPENDED ZK connection for a short amount of time. I haven't tried it out
with a rolling ZK update but it might solve the problem you are observing.

Cheers,
Till

On Tue, Oct 20, 2020 at 5:41 AM Kenzyme <[email protected]> wrote:

> Hi Roman,
>
> Thank you for your reply.
>
> I'm not 100% sure if those features discussed in the threads will fix the
> issue, but they seemed related in some way.
>
> Basically, the expected behaviour I had for Flink was similar to how Kafka
> works i.e.  Kafka services continues w/o disruption whenever ZK quorum is
> maintained during rolling updates.
>
> Best,
>
> Kenzyme Le
>
>
> ‐‐‐‐‐‐‐ Original Message ‐‐‐‐‐‐‐
> On Monday, October 19th, 2020 at 4:38 PM, Khachatryan Roman <
> [email protected]> wrote:
>
> Hi,
>
> AFAIK, the features discussed in the threads you mentioned are not yet
> implemented. So there is no way to avoid Job restarts in case of ZK rolling
> restarts.
> I'm pulling in Till as he might know better.
>
> Regards,
> Roman
>
>
> On Fri, Oct 16, 2020 at 7:45 PM Kenzyme <[email protected]> wrote:
>
>> Hi,
>>
>> Related to
>> https://mail-archives.apache.org/mod_mbox/flink-dev/201709.mbox/%3CCA+faj9yvPyzmmLoEWAMPgXDP6kx+0oed1Z5k4s3K9sgiCFyb=w...@mail.gmail.com%3E
>> and https://issues.apache.org/jira/browse/FLINK-10052, I was wondering
>> if there's a way to prevent Flink instances from failing while doing a
>> rolling restart on ZK followers while still keeping the quorum?
>>
>> This is what was shown in Flink logs while restarting ZK :
>> ZooKeeper connection SUSPENDING. Changes to the submitted job graphs are
>> not monitored (temporarily).
>>
>> I was able to reproduce this twice with a quorum of 5 ZK nodes while
>> doing some ZK maintenance.
>>
>> Thanks!
>>
>> Kenzyme Le
>>
>>
>>
>

Reply via email to