Hi all,

Zhu and I had an offline discussion today. We prefer this FLIP
focuses on improving exponential-delay and uses exponential-delay
as the default strategy. It means this FLIP doesn't include
improvements related to fixed-delay and failover-delay, and the
second part of FLIP(Improve restartAttempt's counting strategy)
just improves exponential-delay.

Following are reasons:

1. Judging from current discussion, many users want
  exponential-delay as the default restart strategy.

2. The semantics of naming and behavior are inconsistent

If we improve the restartAttempt counting mechanism for all
restart strategies, we need to unify the concept of restartAttempt
counting. We want to increase based on the number of restarts,
not the number of failures. The number of failures will increase
too fast, so we hope to aggregate multiple failures into one restart.

However, the failure-rate strategy's restart upper limit option is
named  restart-strategy.failure-rate.max-failures-per-interval,
it's  max-failures-per-interval instead of max-attempts-per-interval.
If we improve it directly, the name and behaviour aren't matched.

3. The restartAttempt counting mechanism and Exception History
  are not match

If we aggregate multiple failures into one restartAttempt, one failure
is an exception in Exception History. Users allowed 10 attempts,
but saw 100 failures on the Exception History, and the job has not
exited yet. Users may be confused. It's related to concurrentExceptions,
and it will be followed at FLINK-33565.

For these reasons, we prefer that current FLIP focus on exponential-delay.
After FLINK-33565 is done, we can discuss the rest of restart-strategies
again.

Looking forward to your feedback, thanks~

To Mingliang,

Sorry, I missed one of your questions this morning.

> One question is the max attempts. Is that the max attempt after which the
job will be deemed failed? I'm wondering if we just simplify the name from
`max-attempts-before-reset-backoff` to `max-attempts` or just `attempts` >
(like the static strategy `restart-strategy.fixed-delay.attempts`). The
wording `before-reset-backoff ` makes me think it's setting the backoff
interval to its initial value after this max attempt, instead of failing
the job.

The max-attempts-before-reset-backoff isn't the same with max-attempts or
attempts.
The exponential-delay has a reset mechanism, when no exception within
reset-backoff-threshold. Flink will reset the delay time to initial-backoff.
max-attempts-before-reset-backoff indicates the maximum number of restarts
we can attempt before resetting.
- When restartAttempt > max-attempts-before-reset-backoff, the job will
exit.
- When no exception within reset-backoff-threshold, the delay time will be
  reset to initial-backoff, and restartAttempt will be reset to 0 as well.

After your feedback, I think attempts-before-reset-backoff may be better,
the max can be removed, and it is like
`restart-strategy.fixed-delay.attempts`.
WDYT?

[1] https://issues.apache.org/jira/browse/FLINK-33565

Best,
Rui

On Thu, Nov 16, 2023 at 11:48 AM Rui Fan <1996fan...@gmail.com> wrote:

> Hi Zhu and Matthias:
>
> > 3. failure counting
> > Flink currently will try to recognize concurrent failures and group them
> > together, which can be seen in the web UI. So how about to align the
> > failure counting with the concurrent failures computing? This can make it
> > more consistent and easier for understanding. It will require changes to
> > the concurrent failures computing though, i.e. taking the backoff time
> > into consideration. So maybe we can open a seperate FLIP for this change.
>
> I recently analyzed concurrentExceptions in detail, and after
> double-checking
> with Matthias who is the contributor of exception history. We found
> the concurrentExceptions doesn't work, it's always empty in production.
> I created FLINK-33565[1] to follow it.
>
> To Zhu:
>
> Discussed with Matthias, we prefer it as a separate JIRA, and
> FLIP-364 doesn't include it due to it's a separate bug. WDYT?
>
> Thanks Zhu mentioned the concurrentExceptions, and thanks Matthias
> help double check.
>
> [1] https://issues.apache.org/jira/browse/FLINK-33565
>
> Best,
> Rui
>
> On Thu, Nov 16, 2023 at 11:39 AM Rui Fan <1996fan...@gmail.com> wrote:
>
>> Hi Zhu, Jing and Mingliang:
>>
>> Thanks for your feedback about consider exponential-delay
>> as the default restart-strategy, and updating the default
>> values of exponential-delay as well. I have started a
>> discussion on user, user-zh and dev mail list about it[1].
>>
>> [1] https://lists.apache.org/thread/6glz0d57r8gtpzq4f71vf9066c5x6nyw
>>
>> Best,
>> Rui
>>
>> On Thu, Nov 16, 2023 at 6:35 AM Mingliang Liu <lium...@apache.org> wrote:
>>
>>> Thanks for sharing your data points.
>>>
>>> Among a few thousand jobs (from the smallest 1 task manager and the
>>> largest 300+ task managers), I presume most of them use the default.
>>> However, the default values we have been using were not broadly discussed
>>> but instead based on a priori knowledge as we manage many jobs for our
>>> (internal) customers. So I believe it's a good idea to engage with user ML
>>> for more feedback. Currently we rely on the two explicit config:
>>>
>>>> restart-strategy.exponential-delay.initial-backoff: 5 s
>>>> restart-strategy.exponential-delay.max-backoff: 2 min
>>>
>>>
>>> I think the default values in the FLIP look good to me overall, though I
>>> completely understand that the one-size-fits-all default values do not
>>> exist. Specifically, a multiplier value between 1 and 2 is more sensible to
>>> me than the existing value 2, if we enable exponential backoff as the
>>> default. The proposed value 1.2 is in this range. Jitter-factor being 0.1
>>> and reset threshold being 1h are both the same as existing values.
>>>
>>> One question is the max attempts. Is that the max attempt after which
>>> the job will be deemed failed? I'm wondering if we just simplify the name
>>> from `max-attempts-before-reset-backoff` to `max-attempts` or just
>>> `attempts` (like the static strategy
>>> `restart-strategy.fixed-delay.attempts`). The wording `before-reset-backoff
>>> ` makes me think it's setting the backoff interval to its initial value
>>> after this max attempt, instead of failing the job.
>>>
>>> On Tue, Nov 14, 2023 at 8:16 PM Rui Fan <1996fan...@gmail.com> wrote:
>>>
>>>> Hi Mingliang:
>>>>
>>>> Thanks you for the feedback here!
>>>>
>>>> Glad to hear Netflix have made exponential-delay as the
>>>> default restart strategy. Our production(Shopee) also makes
>>>> exponential-delay as the default since May 2021, and the
>>>> current number of flink jobs far exceeds tens of thousands.
>>>> These jobs work well.
>>>>
>>>> Note: Our internal exponential-delay solves the problem
>>>> of a large number of tasks failing in a short period of time
>>>> causing restartAttempts to increase rapidly.
>>>>
>>>> Based on your production, do you have any suggestions
>>>> about default values of exponential-delay configuration?
>>>>
>>>> Zhu and Jing may also be interested in this question.
>>>>
>>>> Following are FLIP-364 proposed default values:
>>>>
>>>> restart-strategy.exponential-delay.max-attempts-before-reset-backoff :
>>>> Integer.MAX_VALUE
>>>> restart-strategy.exponential-delay.initial-backoff : 1s
>>>> restart-strategy.exponential-delay.backoff-multiplier : 1.2
>>>> restart-strategy.exponential-delay.jitter-factor : 0.1
>>>> restart-strategy.exponential-delay.max-backoff : 1 min
>>>> restart-strategy.exponential-delay.reset-backoff-threshold : 1h
>>>>
>>>> Looking forward to your feedback! And I will start a discussion
>>>> on user mail list to collect more feedback.
>>>>
>>>> In addition, I understand that the community needs to consider
>>>> a lot of compatibility and risks when modifying the default value.
>>>> If this is very difficult to reach consensus on, I can remove
>>>> this item from FLIP.
>>>>
>>>> Best,
>>>> Rui
>>>>
>>>> On Wed, Nov 15, 2023 at 6:40 AM Mingliang Liu <lium...@apache.org>
>>>> wrote:
>>>>
>>>>> Thanks Rui for driving this. I just call out that making
>>>>> exponential-delay
>>>>> the default is a good change. At Netflix, we have enabled this as the
>>>>> default restart strategy 2 quarters ago and it has been working well.
>>>>> Keeping it restarting indefinitely by default makes sense to me.
>>>>>
>>>>> On Mon, Oct 16, 2023 at 10:11 PM Rui Fan <1996fan...@gmail.com> wrote:
>>>>>
>>>>> > Hi all,
>>>>> >
>>>>> > I would like to start a discussion on FLIP-364: Improve the
>>>>> > restart-strategy[1]
>>>>> >
>>>>> > As we know, the restart-strategy is critical for flink jobs, it
>>>>> mainly
>>>>> > has two functions:
>>>>> > 1. When an exception occurs in the flink job, quickly restart the job
>>>>> > so that the job can return to the running state.
>>>>> > 2. When a job cannot be recovered after frequent restarts within
>>>>> > a certain period of time, Flink will not retry but will fail the job.
>>>>> >
>>>>> > The current restart-strategy support for function 2 has some issues:
>>>>> > 1. The exponential-delay doesn't have the max attempts mechanism,
>>>>> > it means that flink will restart indefinitely even if it fails
>>>>> frequently.
>>>>> > 2. For multi-region streaming jobs and all batch jobs, the failure of
>>>>> > each region will increase the total number of job failures by +1,
>>>>> > even if these failures occur at the same time. If the number of
>>>>> > failures increases too quickly, it will be difficult to set a
>>>>> reasonable
>>>>> > number of retries.
>>>>> > If the maximum number of failures is set too low, the job can easily
>>>>> > reach the retry limit, causing the job to fail. If set too high,
>>>>> some jobs
>>>>> > will never fail.
>>>>> >
>>>>> > In addition, when the above two problems are solved, we can also
>>>>> > discuss whether exponential-delay can replace fixed-delay as the
>>>>> > default restart-strategy. In theory, exponential-delay is smarter and
>>>>> > friendlier than fixed-delay.
>>>>> >
>>>>> > I also thank Zhu Zhu for his suggestions on the option name in
>>>>> > FLINK-32895[2] in advance.
>>>>> >
>>>>> > Looking forward to and welcome everyone's feedback and suggestions,
>>>>> thank
>>>>> > you.
>>>>> >
>>>>> > [1] https://cwiki.apache.org/confluence/x/uJqzDw
>>>>> > [2] https://issues.apache.org/jira/browse/FLINK-32895
>>>>> >
>>>>> > Best,
>>>>> > Rui
>>>>> >
>>>>>
>>>>

Reply via email to