[ 
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)

Reply via email to