Hi there,

Quick dial back here, we have been running load testing and so far haven't
seen suspended state cause job restarts.

Some findings, instead of curator framework capture suspended state and
active notify leader lost, we have seen task manager propagate unhandled
errors from zk client, most likely due to
high-availability.zookeeper.client.max-retry-attempts
were set to 3 and with 5 seconds interval. It would be great if we handle
this exception gracefully with a meaningful exception message. Those error
messages happen when other task managers die due to user code exceptions,
we would like to know more insights on this as well.

For more context, Lu from our team also filed [2] stating issue with 1.9,
so far we haven't seen regression on ongoing load testing jobs.

Thanks,
Chen

Caused by:
> org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException$ConnectionLossException:
> KeeperErrorCode = ConnectionLoss
> at
> org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException.create(KeeperException.java:102)
> at
> org.apache.flink.shaded.curator4.org.apache.curator.framework.imps.CuratorFrameworkImpl.checkBackgroundRetry(CuratorFrameworkImpl.java:862)


[1] https://issues.apache.org/jira/browse/FLINK-10052
[2] https://issues.apache.org/jira/browse/FLINK-19985


On Thu, Apr 15, 2021 at 7:27 PM Yang Wang <danrtsey...@gmail.com> wrote:

> Thanks for trying the unfinished PR and sharing the testing results. Glad
> to here that it could work
> and really hope the result of more stringent load testing.
>
> After then I think we could revive this ticket.
>
>
> Best,
> Yang
>
> Chen Qin <qinnc...@gmail.com> 于2021年4月16日周五 上午2:01写道:
>
>> Hi there,
>>
>> Thanks for providing points to related changes and jira. Some updates
>> from our side, we applied a path by merging FLINK-10052
>> <https://issues.apache.org/jira/browse/FLINK-10052> with master as well
>> as only handling lost state leveraging SessionConnectionStateErrorPolicy
>>   FLINK-10052 <https://issues.apache.org/jira/browse/FLINK-10052>
>>  introduced.
>>
>> Preliminary results were good, the same workload (240 TM) on the same
>> environment runs stable without frequent restarts due to suspended state
>> (seems false positive). We are working on more stringent load testing as
>> well as chaos testing (blocking zk). Will keep folks posted.
>>
>> Thanks,
>> Chen
>>
>>
>> On Tue, Apr 13, 2021 at 1:34 AM Till Rohrmann <trohrm...@apache.org>
>> wrote:
>>
>>> Hi Chenqin,
>>>
>>> The current rationale behind assuming a leadership loss when seeing a
>>> SUSPENDED connection is to assume the worst and to be on the safe side.
>>>
>>> Yang Wang is correct. FLINK-10052 [1] has the goal to make the behaviour
>>> configurable. Unfortunately, the community did not have enough time to
>>> complete this feature.
>>>
>>> [1] https://issues.apache.org/jira/browse/FLINK-10052
>>>
>>> Cheers,
>>> Till
>>>
>>> On Tue, Apr 13, 2021 at 8:25 AM Yang Wang <danrtsey...@gmail.com> wrote:
>>>
>>> > This might be related with FLINK-10052[1].
>>> > Unfortunately, we do not have any progress on this ticket.
>>> >
>>> > cc @Till Rohrmann <trohrm...@apache.org>
>>> >
>>> > Best,
>>> > Yang
>>> >
>>> > chenqin <qinnc...@gmail.com> 于2021年4月13日周二 上午7:31写道:
>>> >
>>> >> Hi there,
>>> >>
>>> >> We observed several 1.11 job running in 1.11 restart due to job leader
>>> >> lost.
>>> >> Dig deeper, the issue seems related to SUSPENDED state handler in
>>> >> ZooKeeperLeaderRetrievalService.
>>> >>
>>> >> ASFAIK, suspended state is expected when zk is not certain if leader
>>> is
>>> >> still alive. It can follow up with RECONNECT or LOST. In current
>>> >> implementation [1] , we treat suspended state same as lost state and
>>> >> actively shutdown job. This pose stability issue on large HA setting.
>>> >>
>>> >> My question is can we get some insight behind this decision and could
>>> we
>>> >> add
>>> >> some tunable configuration for user to decide how long they can endure
>>> >> such
>>> >> uncertain suspended state in their jobs.
>>> >>
>>> >> Thanks,
>>> >> Chen
>>> >>
>>> >> [1]
>>> >>
>>> >>
>>> https://github.com/apache/flink/blob/release-1.11/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java#L201
>>> >>
>>> >>
>>> >>
>>> >>
>>> >> --
>>> >> Sent from:
>>> >> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/
>>> >>
>>> >
>>>
>>

Reply via email to