[ https://issues.apache.org/jira/browse/FLINK-3190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15335985#comment-15335985 ]
ASF GitHub Bot commented on FLINK-3190: --------------------------------------- Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/1954#discussion_r67505611 --- Diff: docs/apis/streaming/fault_tolerance.md --- @@ -338,6 +342,77 @@ The default value is the value of *akka.ask.timeout*. {% top %} +### Failure Rate Restart Strategy + +The failure rate restart strategy restarts job after failure, but when `failure rate` (failures per time unit) 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 +~~~ + +<table class="table table-bordered"> + <thead> + <tr> + <th class="text-left" style="width: 40%">Configuration Parameter</th> + <th class="text-left" style="width: 40%">Description</th> + <th class="text-left">Default Value</th> + </tr> + </thead> + <tbody> + <tr> + <td><it>restart-strategy.failure-rate.max-failures-per-unit</it></td> + <td>Maximum number of restarts in given time unit before failing a job</td> + <td>1</td> + </tr> + <tr> + <td><it>restart-strategy.failure-rate.failure-rate-unit</it></td> + <td>Time unit for measuring failure rate. One of java.util.concurrent.TimeUnit values</td> + <td>MINUTES</td> + </tr> + <tr> + <td><it>restart-strategy.failure-rate.delay</it></td> + <td>Delay between two consecutive restart attempts</td> + <td><it>akka.ask.timeout</it></td> + </tr> + </tbody> +</table> + +~~~ +restart-strategy.failure-rate.max-failures-per-unit: 3 +restart-strategy.failure-rate.failure-rate-unit: MINUTES +restart-strategy.failure-rate.delay: 10 s +~~~ + +The failure rate restart strategy can also be set programmatically: + +<div class="codetabs" markdown="1"> +<div data-lang="java" markdown="1"> +{% highlight java %} +ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); +env.setRestartStrategy(RestartStrategies.failureRateRestart( + 3, // max failures per unit + java.util.concurrent.TimeUnit.MINUTES, //time unit for measuring failure rate + 10000 // delay in milliseconds --- End diff -- Maybe we should support a more flexible delay specification. Something like `"10 seconds"` or `TimeUnit.seconds(10)`. > Retry rate limits for DataStream API > ------------------------------------ > > Key: FLINK-3190 > URL: https://issues.apache.org/jira/browse/FLINK-3190 > Project: Flink > Issue Type: Improvement > Reporter: Sebastian Klemke > Assignee: Michał Fijołek > Priority: Minor > > For a long running stream processing job, absolute numbers of retries don't > make much sense: The job will accumulate transient errors over time and will > die eventually when thresholds are exceeded. Rate limits are better suited in > this scenario: A job should only die, if it fails too often in a given time > frame. To better overcome transient errors, retry delays could be used, as > suggested in other issues. > Absolute numbers of retries can still make sense, if failing operators don't > make any progress at all. We can measure progress by OperatorState changes > and by observing output, as long as the operator in question is not a sink. > If operator state changes and/or operator produces output, we can assume it > makes progress. > As an example, let's say we configured a retry rate limit of 10 retries per > hour and a non-sink operator A. If the operator fails once every 10 minutes > and produces output between failures, it should not lead to job termination. > But if the operator fails 11 times in an hour or does not produce output > between 11 consecutive failures, job should be terminated. -- This message was sent by Atlassian JIRA (v6.3.4#6332)