This documentation is for an out-of-date version of Apache Flink. We recommend you use the latest stable version.

Task Failure Recovery

When a task failure happens, Flink needs to restart the failed task and other affected tasks to recover the job to a normal state.

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.

Restart Strategies

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 flink-conf.yaml. 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.

Key Default Type Description
restart-strategy
(none) String Defines the restart strategy to use in case of job failures.
Accepted values are:
  • none, off, disable: No restart strategy.
  • fixeddelay, fixed-delay: Fixed delay restart strategy. More details can be found here.
  • failurerate, failure-rate: Failure rate restart strategy. More details can be found here.
If checkpointing is disabled, the default value is none. If checkpointing is enabled, the default value is fixed-delay with Integer.MAX_VALUE restart attempts and '1 s' delay.

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 ExecutionEnvironment. Note that this also works for 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.

ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(
  3, // number of restart attempts
  Time.of(10, TimeUnit.SECONDS) // delay
));
val env = ExecutionEnvironment.getExecutionEnvironment()
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(
  3, // number of restart attempts
  Time.of(10, TimeUnit.SECONDS) // delay
))

The following sections describe restart strategy specific configuration options.

Fixed Delay Restart Strategy

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.

restart-strategy: fixed-delay
Key Default Type Description
restart-strategy.fixed-delay.attempts
1 Integer The number of times that Flink retries the execution before the job is declared as failed if restart-strategy has been set to fixed-delay.
restart-strategy.fixed-delay.delay
1 s Duration Delay between two consecutive restart attempts if restart-strategy has been set to fixed-delay. Delaying the retries can be helpful when the program interacts with external systems where for example connections or pending transactions should reach a timeout before re-execution is attempted. It can be specified using notation: "1 min", "20 s"

For example:

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

The fixed delay restart strategy can also be set programmatically:

ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(
  3, // number of restart attempts
  Time.of(10, TimeUnit.SECONDS) // delay
));
val env = ExecutionEnvironment.getExecutionEnvironment()
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(
  3, // number of restart attempts
  Time.of(10, TimeUnit.SECONDS) // delay
))

Failure Rate Restart Strategy

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.

restart-strategy: failure-rate
Key Default Type Description
restart-strategy.failure-rate.delay
1 s Duration Delay between two consecutive restart attempts if restart-strategy has been set to failure-rate. It can be specified using notation: "1 min", "20 s"
restart-strategy.failure-rate.failure-rate-interval
1 min Duration Time interval for measuring failure rate if restart-strategy has been set to failure-rate. It can be specified using notation: "1 min", "20 s"
restart-strategy.failure-rate.max-failures-per-interval
1 Integer Maximum number of restarts in given time interval before failing a job if restart-strategy has been set to failure-rate.
restart-strategy.failure-rate.max-failures-per-interval: 3
restart-strategy.failure-rate.failure-rate-interval: 5 min
restart-strategy.failure-rate.delay: 10 s

The failure rate restart strategy can also be set programmatically:

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

No Restart Strategy

The job fails directly and no restart is attempted.

restart-strategy: none

The no restart strategy can also be set programmatically:

ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
env.setRestartStrategy(RestartStrategies.noRestart());
val env = ExecutionEnvironment.getExecutionEnvironment()
env.setRestartStrategy(RestartStrategies.noRestart())

Fallback Restart Strategy

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 flink-conf.yaml.

Failover Strategy Value for jobmanager.execution.failover-strategy
Restart all full
Restart pipelined region region

Restart All Failover Strategy

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

Restart Pipelined Region Failover Strategy

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.

The regions to restart are decided as below:

  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.
  3. If a region is to be restarted, all of its consumer regions will also be restarted. This is to guarantee data consistency because nondeterministic processing or partitioning can result in different partitions.

Back to top