[
https://issues.apache.org/jira/browse/FLINK-3190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15336030#comment-15336030
]
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_r67509184
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FailureRateRestartStrategy.java
---
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.executiongraph.restart;
+
+import com.google.common.base.Preconditions;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import scala.concurrent.duration.Duration;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static akka.dispatch.Futures.future;
+
+/**
+ * Restart strategy which tries to restart the given {@link
ExecutionGraph} when failure rate exceeded
+ * with a fixed time delay in between.
+ */
+public class FailureRateRestartStrategy implements RestartStrategy {
+ private final int maxFailuresPerUnit;
+ private final TimeUnit failureRateUnit;
+ private final long delayBetweenRestartAttempts;
+ private List<Long> restartTimestamps = new ArrayList<>();
+ private boolean disabled = false;
+
+ public FailureRateRestartStrategy(int maxFailuresPerUnit, TimeUnit
failureRateUnit, long delayBetweenRestartAttempts) {
+ Preconditions.checkArgument(maxFailuresPerUnit > 0, "Maximum
number of restart attempts per time unit must be greater than 0.");
+ Preconditions.checkArgument(delayBetweenRestartAttempts >= 0,
"Delay between restart attempts must be positive");
+
+ this.maxFailuresPerUnit = maxFailuresPerUnit;
+ this.failureRateUnit = failureRateUnit;
+ this.delayBetweenRestartAttempts = delayBetweenRestartAttempts;
+ }
+
+ @Override
+ public boolean canRestart() {
+ return !disabled && canRestartJob();
+ }
+
+ private boolean canRestartJob() {
+ int restartsInWindowSoFar = restartTimestamps.size();
+ if (restartsInWindowSoFar >= maxFailuresPerUnit) {
+ List<Long> lastFailures =
restartTimestamps.subList(restartsInWindowSoFar - maxFailuresPerUnit,
restartsInWindowSoFar);
+ restartTimestamps = lastFailures; //deallocating not
needed timestamps
--- End diff --
This does not deallocate not needed timestamps, because `subList` only
returns a view on the `restartTimestamps` list. So internally, the original
list will always be referenced. Furthermore, each add might cause that the
whole list is copied to a new list which is one element larger.
I think we should use a fixed size queue (size = maxFailuresPerUnit).
Whenever we call `canRestartJob` we first pop all timestamps which fall out of
the current failure interval. If the resulting size of the queue is `<
maxFailuresPerUnit`, then we can restart. Otherwise we've seen more than
`maxFailurePerUnit` in the specified time interval.
> 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)