Hi Rui,

I suppose we could do some benchmarking on what works well for the resource
providers that Flink relies on e.g. Kubernetes. Based on conferences and
blogs, it seems most people are relying on Kubernetes to deploy Flink and
the restart strategy has a large dependency on how well Kubernetes can
scale to requests to redeploy the job.

Best,
Mason

On Fri, Nov 17, 2023 at 10:07 AM David Anderson <dander...@apache.org>
wrote:

> Rui,
>
> I don't have any direct experience with this topic, but given the
> motivation you shared, the proposal makes sense to me. Given that the new
> default feels more complex than the current behavior, if we decide to do
> this I think it will be important to include the rationale you've shared in
> the documentation.
>
> David
>
> On Wed, Nov 15, 2023 at 10:17 PM Rui Fan <1996fan...@gmail.com> wrote:
>
>> Hi dear flink users and devs:
>>
>> FLIP-364[1] intends to make some improvements to restart-strategy
>> and discuss updating some of the default values of exponential-delay,
>> and whether exponential-delay can be used as the default restart-strategy.
>> After discussing at dev mail list[2], we hope to collect more feedback
>> from Flink users.
>>
>> # Why does the default restart-strategy need to be updated?
>>
>> If checkpointing is enabled, the default value is fixed-delay with
>> Integer.MAX_VALUE restart attempts and '1 s' delay[3]. It means
>> the job will restart infinitely with high frequency when a job
>> continues to fail.
>>
>> When the Kafka cluster fails, a large number of flink jobs will be
>> restarted frequently. After the kafka cluster is recovered, a large
>> number of high-frequency restarts of flink jobs may cause the
>> kafka cluster to avalanche again.
>>
>> Considering the exponential-delay as the default strategy with
>> a couple of reasons:
>>
>> - The exponential-delay can reduce the restart frequency when
>>   a job continues to fail.
>> - It can restart a job quickly when a job fails occasionally.
>> - The restart-strategy.exponential-delay.jitter-factor can avoid r
>>   estarting multiple jobs at the same time. It’s useful to prevent
>>   avalanches.
>>
>> # What are the current default values[4] of exponential-delay?
>>
>> restart-strategy.exponential-delay.initial-backoff : 1s
>> restart-strategy.exponential-delay.backoff-multiplier : 2.0
>> restart-strategy.exponential-delay.jitter-factor : 0.1
>> restart-strategy.exponential-delay.max-backoff : 5 min
>> restart-strategy.exponential-delay.reset-backoff-threshold : 1h
>>
>> backoff-multiplier=2 means that the delay time of each restart
>> will be doubled. The delay times are:
>> 1s, 2s, 4s, 8s, 16s, 32s, 64s, 128s, 256s, 300s, 300s, etc.
>>
>> The delay time is increased rapidly, it will affect the recover
>> time for flink jobs.
>>
>> # Option improvements
>>
>> We think the backoff-multiplier between 1 and 2 is more sensible,
>> such as:
>>
>> restart-strategy.exponential-delay.backoff-multiplier : 1.2
>> restart-strategy.exponential-delay.max-backoff : 1 min
>>
>> After updating, the delay times are:
>>
>> 1s, 1.2s, 1.44s, 1.728s, 2.073s, 2.488s, 2.985s, 3.583s, 4.299s,
>> 5.159s, 6.191s, 7.430s, 8.916s, 10.699s, 12.839s, 15.407s, 18.488s,
>> 22.186s, 26.623s, 31.948s, 38.337s, etc
>>
>> They achieve the following goals:
>> - When restarts are infrequent in a short period of time, flink can
>>   quickly restart the job. (For example: the retry delay time when
>>   restarting 5 times is 2.073s)
>> - When restarting frequently in a short period of time, flink can
>>   slightly reduce the restart frequency to prevent avalanches.
>>   (For example: the retry delay time when retrying 10 times is 5.1 s,
>>   and the retry delay time when retrying 20 times is 38s, which is not
>> very
>> large.)
>>
>> As @Mingliang Liu <lium...@apache.org>  mentioned at dev mail list: the
>> one-size-fits-all
>> default values do not exist. So our goal is that the default values
>> can be suitable for most jobs.
>>
>> Looking forward to your thoughts and feedback, thanks~
>>
>> [1] https://cwiki.apache.org/confluence/x/uJqzDw
>> [2] https://lists.apache.org/thread/5cgrft73kgkzkgjozf9zfk0w2oj7rjym
>> [3]
>>
>> https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/deployment/config/#restart-strategy-type
>> [4]
>>
>> https://nightlies.apache.org/flink/flink-docs-master/docs/ops/state/task_failure_recovery/#exponential-delay-restart-strategy
>>
>> Best,
>> Rui
>>
>

Reply via email to