[ 
https://issues.apache.org/jira/browse/FLINK-3187?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15115052#comment-15115052
 ] 

ASF GitHub Bot commented on FLINK-3187:
---------------------------------------

Github user uce commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1470#discussion_r50681234
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java
 ---
    @@ -0,0 +1,113 @@
    +/*
    + * 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 org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import scala.concurrent.duration.Duration;
    +
    +import java.util.concurrent.Callable;
    +
    +import static akka.dispatch.Futures.future;
    +
    +/**
    + * Restart strategy which tries to restart the given {@link 
ExecutionGraph} a fixed number of times
    + * with a fixed time delay in between.
    + */
    +public class FixedDelayRestartStrategy implements RestartStrategy {
    +   private static final Logger LOG = 
LoggerFactory.getLogger(FixedDelayRestartStrategy.class);
    +
    +
    +   private final int maxNumberRestartAttempts;
    +   private final long delayBetweenRestartAttempts;
    +   private int currentRestartAttempt;
    +
    +   public FixedDelayRestartStrategy(
    +           int maxNumberRestartAttempts,
    +           long delayBetweenRestartAttempts) {
    +
    +           Preconditions.checkArgument(maxNumberRestartAttempts >= 0, 
"Maximum number of restart attempts must be positive.");
    +           Preconditions.checkArgument(delayBetweenRestartAttempts >= 0, 
"Delay between restart attempts must be positive");
    +
    +           this.maxNumberRestartAttempts = maxNumberRestartAttempts;
    +           this.delayBetweenRestartAttempts = delayBetweenRestartAttempts;
    +           currentRestartAttempt = 0;
    +   }
    +
    +   @Override
    +   public boolean canRestart() {
    +           return currentRestartAttempt < maxNumberRestartAttempts;
    +   }
    +
    +   @Override
    +   public void restart(final ExecutionGraph executionGraph) {
    +           currentRestartAttempt++;
    +
    +           future(new Callable<Object>() {
    +                   @Override
    +                   public Object call() throws Exception {
    +                           try {
    +                                   LOG.info("Delaying retry of job 
execution for {} ms ...", delayBetweenRestartAttempts);
    +                                   // do the delay
    +                                   
Thread.sleep(delayBetweenRestartAttempts);
    +                           }
    +                           catch(InterruptedException e){
    --- End diff --
    
    whitespace after catch


> Decouple restart strategy from ExecutionGraph
> ---------------------------------------------
>
>                 Key: FLINK-3187
>                 URL: https://issues.apache.org/jira/browse/FLINK-3187
>             Project: Flink
>          Issue Type: Improvement
>    Affects Versions: 1.0.0
>            Reporter: Till Rohrmann
>            Assignee: Till Rohrmann
>            Priority: Minor
>
> Currently, the {{ExecutionGraph}} supports the following restart logic: 
> Whenever a failure occurs and the number of restart attempts aren't depleted, 
> wait for a fixed amount of time and then try to restart. This behaviour can 
> be controlled by the configuration parameters {{execution-retries.default}} 
> and {{execution-retries.delay}}.
> I propose to decouple the restart logic from the {{ExecutionGraph}} a bit by 
> introducing a strategy pattern. That way it would not only allow us to define 
> a job specific restart behaviour but also to implement different restart 
> strategies. Conceivable strategies could be: Fixed timeout restart, 
> exponential backoff restart, partial topology restarts, etc.
> This change is a preliminary step towards having a restart strategy which 
> will scale the parallelism of a job down in case that not enough slots are 
> available.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to