Task Failure Recovery

    Restart strategies and failover strategies are used to control the task restarting. Restart strategies decide whether and when the failed/affected tasks can be restarted. Failover strategies decide which tasks should be restarted to recover the job.

    The cluster can be started with a default restart strategy which is always used when no job specific restart strategy has been defined. In case that the job is submitted with a restart strategy, this strategy overrides the cluster’s default setting.

    The default restart strategy is set via Flink’s configuration file . The configuration parameter restart-strategy defines which strategy is taken. If checkpointing is not enabled, the “no restart” strategy is used. If checkpointing is activated and the restart strategy has not been configured, the fixed-delay strategy is used with Integer.MAX_VALUE restart attempts. See the following list of available restart strategies to learn what values are supported.

    Each restart strategy comes with its own set of parameters which control its behaviour. These values are also set in the configuration file. The description of each restart strategy contains more information about the respective configuration values.

    Apart from defining a default restart strategy, it is possible to define for each Flink job a specific restart strategy. This restart strategy is set programmatically by calling the setRestartStrategy method on the StreamExecutionEnvironment.

    The following example shows how we can set a fixed delay restart strategy for our job. In case of a failure the system tries to restart the job 3 times and waits 10 seconds in-between successive restart attempts.

    Java

    Scala

    1. val env = StreamExecutionEnvironment.getExecutionEnvironment()
    2. env.setRestartStrategy(RestartStrategies.fixedDelayRestart(
    3. 3, // number of restart attempts
    4. Time.of(10, TimeUnit.SECONDS) // delay
    5. ))

    The following sections describe restart strategy specific configuration options.

    The fixed delay restart strategy attempts a given number of times to restart the job. If the maximum number of attempts is exceeded, the job eventually fails. In-between two consecutive restart attempts, the restart strategy waits a fixed amount of time.

    This strategy is enabled as default by setting the following configuration parameter in flink-conf.yaml.

    1. restart-strategy: fixed-delay
    1. restart-strategy.fixed-delay.attempts: 3
    2. restart-strategy.fixed-delay.delay: 10 s

    The fixed delay restart strategy can also be set programmatically:

    Java

    1. StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    2. Time.of(10, TimeUnit.SECONDS) // delay
    3. ));

    Scala

    Exponential Delay Restart Strategy

    The exponential delay restart strategy attempts to restart the job infinitely, with increasing delay up to the maximum delay. The job never fails. In-between two consecutive restart attempts, the restart strategy keeps exponentially increasing until the maximum number is reached. Then, it keeps the delay at the maximum number.

    When the job executes correctly, the exponential delay value resets after some time; this threshold is configurable.

    1. restart-strategy: exponential-delay

    For example:

    1. restart-strategy.exponential-delay.initial-backoff: 10 s
    2. restart-strategy.exponential-delay.max-backoff: 2 min
    3. restart-strategy.exponential-delay.backoff-multiplier: 2.0
    4. restart-strategy.exponential-delay.reset-backoff-threshold: 10 min
    5. restart-strategy.exponential-delay.jitter-factor: 0.1

    The exponential delay restart strategy can also be set programmatically:

    Java

    1. StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    2. env.setRestartStrategy(RestartStrategies.exponentialDelayRestart(
    3. Time.milliseconds(1),
    4. Time.milliseconds(1000),
    5. 1.1, // exponential multiplier
    6. Time.milliseconds(2000), // threshold duration to reset delay to its initial value
    7. 0.1 // jitter
    8. ));

    Scala

    1. val env = StreamExecutionEnvironment.getExecutionEnvironment()
    2. env.setRestartStrategy(RestartStrategies.exponentialDelayRestart(
    3. Time.of(1, TimeUnit.MILLISECONDS), // initial delay between restarts
    4. 1.1, // exponential multiplier
    5. Time.of(2, TimeUnit.SECONDS), // threshold duration to reset delay to its initial value
    6. 0.1 // jitter

    The failure rate restart strategy restarts job after failure, but when failure rate (failures per time interval) is exceeded, the job eventually fails. In-between two consecutive restart attempts, the restart strategy waits a fixed amount of time.

    This strategy is enabled as default by setting the following configuration parameter in flink-conf.yaml.

    1. restart-strategy.failure-rate.max-failures-per-interval: 3
    2. restart-strategy.failure-rate.failure-rate-interval: 5 min
    3. restart-strategy.failure-rate.delay: 10 s

    Java

    1. StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    2. env.setRestartStrategy(RestartStrategies.failureRateRestart(
    3. 3, // max failures per interval
    4. Time.of(5, TimeUnit.MINUTES), //time interval for measuring failure rate
    5. Time.of(10, TimeUnit.SECONDS) // delay
    6. ));

    Scala

    1. val env = StreamExecutionEnvironment.getExecutionEnvironment()
    2. env.setRestartStrategy(RestartStrategies.failureRateRestart(
    3. 3, // max failures per unit
    4. Time.of(5, TimeUnit.MINUTES), //time interval for measuring failure rate
    5. Time.of(10, TimeUnit.SECONDS) // delay
    6. ))

    No Restart Strategy

    The job fails directly and no restart is attempted.

    1. restart-strategy: none

    The no restart strategy can also be set programmatically:

    Java

    Scala

    1. val env = StreamExecutionEnvironment.getExecutionEnvironment()
    2. env.setRestartStrategy(RestartStrategies.noRestart())

    The cluster defined restart strategy is used. This is helpful for streaming programs which enable checkpointing. By default, a fixed delay restart strategy is chosen if there is no other restart strategy defined.

    Failover Strategies

    Flink supports different failover strategies which can be configured via the configuration parameter jobmanager.execution.failover-strategy in Flink’s configuration file .

    Restart All Failover Strategy

    This strategy restarts all tasks in the job to recover from a task failure.

    This strategy groups tasks into disjoint regions. When a task failure is detected, this strategy computes the smallest set of regions that must be restarted to recover from the failure. For some jobs this can result in fewer tasks that will be restarted compared to the Restart All Failover Strategy.

    A region is a set of tasks that communicate via pipelined data exchanges. That is, batch data exchanges denote the boundaries of a region.

    • All data exchanges in a DataStream job or Streaming Table/SQL job are pipelined.
    • All data exchanges in a Batch Table/SQL job are batched by default.
    • The data exchange types in a DataSet job are determined by the ExecutionMode which can be set through ExecutionConfig.
    1. The region containing the failed task will be restarted.
    2. If a result partition is not available while it is required by a region that will be restarted, the region producing the result partition will be restarted as well.