[
https://issues.apache.org/jira/browse/FLINK-3190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15367819#comment-15367819
]
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_r70094058
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FailureRateRestartStrategy.java
---
@@ -36,19 +36,19 @@
public class FailureRateRestartStrategy implements RestartStrategy {
private final Duration failuresInterval;
private final Duration delayInterval;
- private EvictingQueue<Long> restartTimestampsQueue;
+ private FixedSizeFifoQueue<Long> restartTimestampsQueue;
private boolean disabled = false;
public FailureRateRestartStrategy(int maxFailuresPerInterval, Duration
failuresInterval, Duration delayInterval) {
- Preconditions.checkArgument(maxFailuresPerInterval > 0,
"Maximum number of restart attempts per time unit must be greater than 0.");
Preconditions.checkNotNull(failuresInterval, "Failures interval
cannot be null.");
- Preconditions.checkNotNull(failuresInterval.length() > 0,
"Failures interval must be greater than 0 ms.");
Preconditions.checkNotNull(delayInterval, "Delay interval
cannot be null.");
- Preconditions.checkNotNull(delayInterval.length() >= 0, "Delay
interval must be at least 0 ms.");
+ Preconditions.checkArgument(maxFailuresPerInterval > 0,
"Maximum number of restart attempts per time unit must be greater than 0.");
+ Preconditions.checkArgument(failuresInterval.length() > 0,
"Failures interval must be greater than 0 ms.");
+ Preconditions.checkArgument(delayInterval.length() >= 0, "Delay
interval must be at least 0 ms.");
this.failuresInterval = failuresInterval;
this.delayInterval = delayInterval;
- this.restartTimestampsQueue =
EvictingQueue.create(maxFailuresPerInterval);
+ this.restartTimestampsQueue = new
FixedSizeFifoQueue<>(maxFailuresPerInterval);
--- End diff --
Can't we simply use `new ArrayDeque(maxFailuresPerInterval)`? Of course, we
would then allocate 2^(ceil(log(maxFailuresPerInterval)/log(2)) elements, but
this should be ok. We could then check in the `restart` method via the `size`
method whether the queue is full or not.
> 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)