From 4a9f1950132f8ef5b08170de716d3ce99da606fd Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 17 Dec 2015 13:49:10 +0100 Subject: [PATCH] [FLINK-3187] [restart] Introduce RestartStrategy to ExecutionGraph A RestartStrategy defines how the ExecutionGraph reacts in case of a restart. Different strategies are conceivable. For example, no restart, fixed delay restart, exponential backoff restart, scaling in/out restart, etc. Expose RestartStrategy to user API This removes the setNumberExecutionRetries and the setDelayBetweenRetries on the ExecutionEnvironment and the ExecutionConfig. Instead the more general RestartStrategy can be set. In order to maintain the separation between the runtime and api module, one sets a RestartStrategyConfiguration which is transformed into a RestartStrategy on the JobManager. Replace old execution-retries configuration parameters by restart-strategy. Add FixedDelayRestartStrategy test case Reintroduce old configuration values and API calls for the deprecated restart mechanism The old configuration values and API calls will be respected if no explicit RestartStrategy has been set. The values, if correct, are used to instantiate a FixedDelayRestartStrategy. Add deprecation comments to the JavaDocs Add logging statement for job recovery Fix JobManagerProcessFailureBatchRecoveryITCase by introducing a job recovery timeout Add proper annotations to RestartStrategies Let ExecutionGraphRestartTest extend TestLogger Fix --- docs/apis/streaming/fault_tolerance.md | 168 ++++++++++++++++++ docs/internals/monitoring_rest_api.md | 2 +- docs/setup/config.md | 16 +- .../flink/api/common/ExecutionConfig.java | 85 ++++++++- .../org/apache/flink/api/common/Plan.java | 24 +-- .../restartstrategy/RestartStrategies.java | 120 +++++++++++++ .../flink/configuration/ConfigConstants.java | 42 ++++- .../examples/kafka/ReadFromKafka.java | 3 +- .../examples/kafka/WriteIntoKafka.java | 3 +- .../flink/api/java/ExecutionEnvironment.java | 32 ++++ .../flink/python/api/PythonPlanBinder.java | 4 +- .../plantranslate/JobGraphGenerator.java | 4 +- .../webmonitor/handlers/JobConfigHandler.java | 4 +- .../app/partials/jobs/job.config.jade | 2 +- .../executiongraph/ExecutionGraph.java | 82 ++------- .../restart/FixedDelayRestartStrategy.java | 116 ++++++++++++ .../restart/NoRestartStrategy.java | 48 +++++ .../restart/RestartStrategy.java | 41 +++++ .../restart/RestartStrategyFactory.java | 127 +++++++++++++ .../flink/runtime/jobgraph/JobGraph.java | 57 ++---- .../flink/runtime/jobmanager/JobManager.scala | 112 ++++++------ ...ecutionGraphCheckpointCoordinatorTest.java | 18 +- .../ExecutionGraphConstructionTest.java | 25 ++- .../ExecutionGraphDeploymentTest.java | 9 +- .../ExecutionGraphRestartTest.java | 56 +++--- .../ExecutionGraphTestUtils.java | 4 +- .../ExecutionStateProgressTest.java | 6 +- .../executiongraph/LocalInputSplitsTest.java | 7 +- .../executiongraph/PointwisePatternTest.java | 22 ++- .../TerminalStateDeadlockTest.java | 13 +- .../VertexLocationConstraintTest.java | 19 +- .../executiongraph/VertexSlotSharingTest.java | 4 +- .../FixedDelayRestartStrategyTest.java | 51 ++++++ .../partition/PipelinedSubpartitionTest.java | 4 - .../JobManagerLeaderElectionTest.java | 7 +- .../runtime/testutils/ZooKeeperTestUtils.java | 2 +- .../TaskManagerLossFailsTasksTest.scala | 6 +- .../runtime/jobmanager/RecoveryITCase.scala | 9 +- .../runtime/testingUtils/TestingCluster.scala | 12 +- .../testingUtils/TestingJobManager.scala | 13 +- .../runtime/testingUtils/TestingUtils.scala | 12 +- .../api/scala/ExecutionEnvironment.scala | 46 ++++- .../connectors/kafka/Kafka08ITCase.java | 19 +- .../kafka/KafkaConsumerTestBase.java | 27 +-- .../kafka/KafkaProducerTestBase.java | 3 +- .../connectors/kafka/KafkaTestBase.java | 2 +- .../kafka/testutils/DataGenerators.java | 5 +- .../StreamExecutionEnvironment.java | 32 ++++ .../api/graph/StreamingJobGraphGenerator.java | 30 ++-- .../partitioner/RescalePartitionerTest.java | 7 +- .../runtime/state/StateBackendITCase.java | 3 +- .../streaming/timestamp/TimestampITCase.java | 1 - .../scala/StreamExecutionEnvironment.scala | 49 +++-- ...EventTimeAllWindowCheckpointingITCase.java | 10 +- .../EventTimeWindowCheckpointingITCase.java | 14 +- .../test/checkpointing/SavepointITCase.java | 18 +- .../StreamCheckpointNotifierITCase.java | 2 +- .../StreamFaultToleranceTestBase.java | 1 - .../WindowCheckpointingITCase.java | 10 +- .../test/classloading/ClassLoaderITCase.java | 1 - .../jar/CheckpointedStreamingProgram.java | 3 +- .../failingPrograms/TaskFailureITCase.java | 4 +- ...TaskManagerProcessFailureRecoveryTest.java | 2 +- .../test/recovery/FastFailuresITCase.java | 6 +- ...agerProcessFailureBatchRecoveryITCase.java | 3 +- .../ProcessFailureCancelingITCase.java | 3 +- .../test/recovery/SimpleRecoveryITCase.java | 13 +- .../TaskManagerFailureRecoveryITCase.java | 3 +- ...agerProcessFailureBatchRecoveryITCase.java | 3 +- ...ProcessFailureStreamingRecoveryITCase.java | 3 +- .../ZooKeeperLeaderElectionITCase.java | 2 +- .../flink/yarn/TestingYarnJobManager.scala | 16 +- .../apache/flink/yarn/YarnJobManager.scala | 16 +- 73 files changed, 1314 insertions(+), 434 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/restartstrategy/RestartStrategies.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/NoRestartStrategy.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/RestartStrategy.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/RestartStrategyFactory.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategyTest.java diff --git a/docs/apis/streaming/fault_tolerance.md b/docs/apis/streaming/fault_tolerance.md index 4b91c61a92e58..f1a6803f92468 100644 --- a/docs/apis/streaming/fault_tolerance.md +++ b/docs/apis/streaming/fault_tolerance.md @@ -194,3 +194,171 @@ state updates) of Flink coupled with bundled sinks: {% top %} + +Restart Strategies +------------------ + +Flink supports different restart strategies which control how the jobs are restarted in case of a failure. +The cluster can be started with a default restart strategy which is always used when no job specific restart strategy has been defined. +In case that the job is submitted with a restart strategy, this strategy overrides the cluster's default setting. + +The default restart strategy is set via Flink's configuration file `flink-conf.yaml`. +The configuration parameter *restart-strategy* defines which strategy is taken. +Per default, the no-restart strategy is used. +See the following list of available restart strategies to learn what values are supported. + +Each restart strategy comes with its own set of parameters which control its behaviour. +These values are also set in the configuration file. +The description of each restart strategy contains more information about the respective configuration values. + + + + + + + + + + + + + + + + + + +
Restart StrategyValue for restart-strategy
Fixed delayfixed-delay
No restartnone
+ +Apart from defining a default restart strategy, it is possible to define for each Flink job a specific restart strategy. +This restart strategy is set programmatically by calling the `setRestartStrategy` method on the `ExecutionEnvironment`. +Note that this also works for the `StreamExecutionEnvironment`. + +The following example shows how we can set a fixed delay restart strategy for our job. +In case of a failure the system tries to restart the job 3 times and waits 10 seconds in-between successive restart attempts. + +
+
+{% highlight java %} +ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); +env.setRestartStrategy(RestartStrategies.fixedDelay( + 3, // number of restart attempts + 10000 // delay in milliseconds +)); +{% endhighlight %} +
+
+{% highlight scala %} +val env = ExecutionEnvironment.getExecutionEnvironment() +env.setRestartStrategy(RestartStrategies.fixedDelay( + 3, // number of restart attempts + 10000 // delay in milliseconds +)) +{% endhighlight %} +
+
+ +## Fixed Delay Restart Strategy + +The fixed delay restart strategy attempts a given number of times to restart the job. +If the maximum number of attempts 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: fixed-delay +~~~ + + + + + + + + + + + + + + + + + + + + + +
Configuration ParameterDescriptionDefault Value
restart-strategy.fixed-delay.attemptsNumber of restart attempts1
restart-strategy.fixed-delay.delayDelay between two consecutive restart attemptsakka.ask.timeout
+ +~~~ +restart-strategy.fixed-delay.attempts: 3 +restart-strategy.fixed-delay.delay: 10 s +~~~ + +The fixed delay restart strategy can also be set programmatically: + +
+
+{% highlight java %} +ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); +env.setRestartStrategy(RestartStrategies.fixedDelay( + 3, // number of restart attempts + 10000 // delay in milliseconds +)); +{% endhighlight %} +
+
+{% highlight scala %} +val env = ExecutionEnvironment.getExecutionEnvironment() +env.setRestartStrategy(RestartStrategies.fixedDelay( + 3, // number of restart attempts + 10000 // delay in milliseconds +)) +{% endhighlight %} +
+
+ +### Restart Attempts + +The number of times that Flink retries the execution before the job is declared as failed is configurable via the *restart-strategy.fixed-delay.attempts* parameter. + +The default value is **1**. + +### Retry Delays + +Execution retries can be configured to be delayed. Delaying the retry means that after a failed execution, the re-execution does not start immediately, but only after a certain delay. + +Delaying the retries can be helpful when the program interacts with external systems where for example connections or pending transactions should reach a timeout before re-execution is attempted. + +The default value is the value of *akka.ask.timeout*. + +## No Restart Strategy + +The job fails directly and no restart is attempted. + +~~~ +restart-strategy: none +~~~ + +The no restart strategy can also be set programmatically: + +
+
+{% highlight java %} +ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); +env.setRestartStrategy(RestartStrategies.noRestart()); +{% endhighlight %} +
+
+{% highlight scala %} +val env = ExecutionEnvironment.getExecutionEnvironment() +env.setRestartStrategy(RestartStrategies.noRestart()) +{% endhighlight %} +
+
+ +[Back to top](#top) + + diff --git a/docs/internals/monitoring_rest_api.md b/docs/internals/monitoring_rest_api.md index 70952f5272a62..c4a625eaeb3ce 100644 --- a/docs/internals/monitoring_rest_api.md +++ b/docs/internals/monitoring_rest_api.md @@ -244,7 +244,7 @@ Sample Result: "name": "WordCount Example", "execution-config": { "execution-mode": "PIPELINED", - "max-execution-retries": -1, + "restart-strategy": "Restart deactivated", "job-parallelism": -1, "object-reuse-mode": false } diff --git a/docs/setup/config.md b/docs/setup/config.md index 343a85684ebcc..59cccc313a9a1 100644 --- a/docs/setup/config.md +++ b/docs/setup/config.md @@ -118,9 +118,17 @@ If you are on YARN, then it is sufficient to authenticate the client with Kerber - `blob.server.port`: Port definition for the blob server (serving user jar's) on the Taskmanagers. By default the port is set to 0, which means that the operating system is picking an ephemeral port. Flink also accepts a list of ports ("50100,50101"), ranges ("50100-50200") or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple JobManagers are running on the same machine. -- `execution-retries.delay`: Delay between execution retries. Default value "5 s". Note that values have to be specified as strings with a unit. - -- `execution-retries.default`: Default number of execution retries, used by jobs that do not explicitly specify that value on the execution environment. Default value is zero. +- `restart-strategy`: Default restart strategy to use in case that no restart strategy has been specified for the submitted job. +Currently, it can be chosen between using a fixed delay restart strategy and to turn it off. +To use the fixed delay strategy you have to specify "fixed-delay". +To turn the restart behaviour off you have to specify "none". +Default value "none". + +- `restart-strategy.fixed-delay.attempts`: Number of restart attempts, used if the default restart strategy is set to "fixed-delay". +Default value is 1. + +- `restart-strategy.fixed-delay.delay`: Delay between restart attempts, used if the default restart strategy is set to "fixed-delay". +Default value is the `akka.ask.timeout`. ## Full Reference @@ -247,6 +255,8 @@ For example when running Flink on YARN on an environment with a restrictive fire - `recovery.zookeeper.client.max-retry-attempts`: (Default '3') Defines the number of connection retries before the client gives up. +- `recovery.job.delay`: (Default 'akka.ask.timeout') Defines the delay before persisted jobs are recovered in case of a recovery situation. + ## Background ### Configuring the Network Buffers diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java index 8d5211bba349f..b31106f4a1908 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java @@ -21,6 +21,7 @@ import com.esotericsoftware.kryo.Serializer; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.Public; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import java.io.Serializable; import java.util.LinkedHashMap; @@ -76,7 +77,11 @@ public class ExecutionConfig implements Serializable { private int parallelism = -1; - private int numberOfExecutionRetries = -1; + /** + * @deprecated Should no longer be used because it is subsumed by RestartStrategyConfiguration + */ + @Deprecated + private int numberOfExecutionRetries = 0; private boolean forceKryo = false; @@ -96,9 +101,15 @@ public class ExecutionConfig implements Serializable { private long autoWatermarkInterval = 0; private boolean timestampsEnabled = false; - - private long executionRetryDelay = -1; + /** + * @deprecated Should no longer be used because it is subsumed by RestartStrategyConfiguration + */ + @Deprecated + private long executionRetryDelay = 0; + + private RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration; + // Serializers and types registered with Kryo and the PojoSerializer // we store them in linked maps/sets to ensure they are registered in order in all kryo instances. @@ -244,20 +255,65 @@ public ExecutionConfig setParallelism(int parallelism) { return this; } + /** + * Sets the restart strategy to be used for recovery. + * + *
{@code
+	 * ExecutionConfig config = env.getConfig();
+	 *
+	 * config.setRestartStrategy(RestartStrategies.fixedDelayRestart(
+	 * 	10,  // number of retries
+	 * 	1000 // delay between retries));
+	 * }
+ * + * @param restartStrategyConfiguration Configuration defining the restart strategy to use + */ + @PublicEvolving + public void setRestartStrategy(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration) { + this.restartStrategyConfiguration = restartStrategyConfiguration; + } + + /** + * Returns the restart strategy which has been set for the current job. + * + * @return The specified restart configuration + */ + @PublicEvolving + public RestartStrategies.RestartStrategyConfiguration getRestartStrategy() { + if (restartStrategyConfiguration == null) { + // support the old API calls by creating a restart strategy from them + if (getNumberOfExecutionRetries() > 0 && getExecutionRetryDelay() >= 0) { + return RestartStrategies.fixedDelayRestart(getNumberOfExecutionRetries(), getExecutionRetryDelay()); + } else { + return null; + } + } else { + return restartStrategyConfiguration; + } + } + /** * Gets the number of times the system will try to re-execute failed tasks. A value * of {@code -1} indicates that the system default value (as defined in the configuration) * should be used. * * @return The number of times the system will try to re-execute failed tasks. + * + * @deprecated Should no longer be used because it is subsumed by RestartStrategyConfiguration */ + @Deprecated public int getNumberOfExecutionRetries() { return numberOfExecutionRetries; } /** * Returns the delay between execution retries. + * + * @return The delay between successive execution retries in milliseconds. + * + * @deprecated Should no longer be used because it is subsumed by RestartStrategyConfiguration */ + @Deprecated public long getExecutionRetryDelay() { return executionRetryDelay; } @@ -268,11 +324,18 @@ public long getExecutionRetryDelay() { * default value (as defined in the configuration) should be used. * * @param numberOfExecutionRetries The number of times the system will try to re-execute failed tasks. + * + * @return The current execution configuration + * + * @deprecated This method will be replaced by {@link #setRestartStrategy}. The + * {@link RestartStrategies.FixedDelayRestartStrategyConfiguration} contains the number of + * execution retries. */ + @Deprecated public ExecutionConfig setNumberOfExecutionRetries(int numberOfExecutionRetries) { if (numberOfExecutionRetries < -1) { throw new IllegalArgumentException( - "The number of execution retries must be non-negative, or -1 (use system default)"); + "The number of execution retries must be non-negative, or -1 (use system default)"); } this.numberOfExecutionRetries = numberOfExecutionRetries; return this; @@ -282,15 +345,23 @@ public ExecutionConfig setNumberOfExecutionRetries(int numberOfExecutionRetries) * Sets the delay between executions. A value of {@code -1} indicates that the default value * should be used. * @param executionRetryDelay The number of milliseconds the system will wait to retry. + * + * @return The current execution configuration + * + * @deprecated This method will be replaced by {@link #setRestartStrategy}. The + * {@link RestartStrategies.FixedDelayRestartStrategyConfiguration} contains the delay between + * successive execution attempts. */ + @Deprecated public ExecutionConfig setExecutionRetryDelay(long executionRetryDelay) { if (executionRetryDelay < -1 ) { throw new IllegalArgumentException( - "The delay between reties must be non-negative, or -1 (use system default)"); + "The delay between reties must be non-negative, or -1 (use system default)"); } this.executionRetryDelay = executionRetryDelay; return this; } + /** * Sets the execution mode to execute the program. The execution mode defines whether * data exchanges are performed in a batch or on a pipelined manner. @@ -614,7 +685,7 @@ public boolean equals(Object obj) { Objects.equals(executionMode, other.executionMode) && useClosureCleaner == other.useClosureCleaner && parallelism == other.parallelism && - numberOfExecutionRetries == other.numberOfExecutionRetries && + restartStrategyConfiguration.equals(other.restartStrategyConfiguration) && forceKryo == other.forceKryo && objectReuse == other.objectReuse && autoTypeRegistrationEnabled == other.autoTypeRegistrationEnabled && @@ -640,7 +711,7 @@ public int hashCode() { executionMode, useClosureCleaner, parallelism, - numberOfExecutionRetries, + restartStrategyConfiguration, forceKryo, objectReuse, autoTypeRegistrationEnabled, diff --git a/flink-core/src/main/java/org/apache/flink/api/common/Plan.java b/flink-core/src/main/java/org/apache/flink/api/common/Plan.java index 899b9d6c8962f..d81fcd15ab24f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/Plan.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/Plan.java @@ -38,6 +38,7 @@ import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry; import org.apache.flink.api.common.operators.GenericDataSinkBase; import org.apache.flink.api.common.operators.Operator; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.util.Visitable; @@ -292,24 +293,15 @@ public void setDefaultParallelism(int defaultParallelism) { this.defaultParallelism = defaultParallelism; } - - /** - * Gets the number of times the system will try to re-execute failed tasks. A value - * of {@code -1} indicates that the system default value (as defined in the configuration) - * should be used. - * - * @return The number of times the system will try to re-execute failed tasks. - */ - public int getNumberOfExecutionRetries() { - return getExecutionConfig().getNumberOfExecutionRetries(); - } - + /** - * Gets the delay between retry failed task. - * @return The delay the system will wait to retry. + * Returns the specified restart strategy configuration. This configuration defines the used + * restart strategy to be used at runtime. + * + * @return The specified restart strategy configuration */ - public long getExecutionRetryDelay() { - return getExecutionConfig().getExecutionRetryDelay(); + public RestartStrategies.RestartStrategyConfiguration getRestartStrategyConfiguration() { + return getExecutionConfig().getRestartStrategy(); } /** diff --git a/flink-core/src/main/java/org/apache/flink/api/common/restartstrategy/RestartStrategies.java b/flink-core/src/main/java/org/apache/flink/api/common/restartstrategy/RestartStrategies.java new file mode 100644 index 0000000000000..12f9d08092ff1 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/restartstrategy/RestartStrategies.java @@ -0,0 +1,120 @@ +/* + * 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.api.common.restartstrategy; + +import org.apache.flink.annotation.PublicEvolving; + +import java.io.Serializable; + +/** + * This class defines methods to generate RestartStrategyConfigurations. These configurations are + * used to create RestartStrategies at runtime. + * + * The RestartStrategyConfigurations are used to decouple the core module from the runtime module. + */ +@PublicEvolving +public class RestartStrategies { + + /** + * Generates NoRestartStrategyConfiguration + * + * @return NoRestartStrategyConfiguration + */ + public static RestartStrategyConfiguration noRestart() { + return new NoRestartStrategyConfiguration(); + } + + /** + * Generates a FixedDelayRestartStrategyConfiguration. + * + * @param restartAttempts Number of restart attempts for the FixedDelayRestartStrategy + * @param delayBetweenAttempts Delay in-between restart attempts for the FixedDelayRestartStrategy + * @return FixedDelayRestartStrategy + */ + public static RestartStrategyConfiguration fixedDelayRestart( + int restartAttempts, + long delayBetweenAttempts) { + + return new FixedDelayRestartStrategyConfiguration(restartAttempts, delayBetweenAttempts); + } + + public abstract static class RestartStrategyConfiguration implements Serializable { + private static final long serialVersionUID = 6285853591578313960L; + + private RestartStrategyConfiguration() {} + + /** + * Returns a description which is shown in the web interface + * + * @return Description of the restart strategy + */ + public abstract String getDescription(); + } + + final public static class NoRestartStrategyConfiguration extends RestartStrategyConfiguration { + private static final long serialVersionUID = -5894362702943349962L; + + @Override + public String getDescription() { + return "Restart deactivated."; + } + } + + final public static class FixedDelayRestartStrategyConfiguration extends RestartStrategyConfiguration { + private static final long serialVersionUID = 4149870149673363190L; + + private final int restartAttempts; + private final long delayBetweenAttempts; + + FixedDelayRestartStrategyConfiguration(int restartAttempts, long delayBetweenAttempts) { + this.restartAttempts = restartAttempts; + this.delayBetweenAttempts = delayBetweenAttempts; + } + + public int getRestartAttempts() { + return restartAttempts; + } + + public long getDelayBetweenAttempts() { + return delayBetweenAttempts; + } + + @Override + public int hashCode() { + return 31 * restartAttempts + (int)(delayBetweenAttempts ^ (delayBetweenAttempts >>> 32)); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof FixedDelayRestartStrategyConfiguration) { + FixedDelayRestartStrategyConfiguration other = (FixedDelayRestartStrategyConfiguration) obj; + + return restartAttempts == other.restartAttempts && delayBetweenAttempts == other.delayBetweenAttempts; + } else { + return false; + } + } + + @Override + public String getDescription() { + return "Restart with fixed delay (" + delayBetweenAttempts + " ms). #" + + restartAttempts + " restart attempts."; + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index 2b756449d53d4..b2bbda12c16dc 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -19,6 +19,7 @@ package org.apache.flink.configuration; import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.PublicEvolving; /** * This class contains all constants for the configuration. That includes the configuration keys and @@ -38,16 +39,50 @@ public final class ConfigConstants { */ public static final String DEFAULT_PARALLELISM_KEY = "parallelism.default"; + // ---------------------------- Restart strategies ------------------------ + + /** + * Defines the restart strategy to be used. It can be "off", "none", "disable" to be disabled or + * it can be "fixeddelay", "fixed-delay" to use the FixedDelayRestartStrategy. You can also + * specify a class name which implements the RestartStrategy interface and has a static + * create method which takes a Configuration object. + */ + @PublicEvolving + public static final String RESTART_STRATEGY = "restart-strategy"; + + /** + * Maximum number of attempts the fixed delay restart strategy will try before failing a job. + */ + @PublicEvolving + public static final String RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS = "restart-strategy.fixed-delay.attempts"; + + /** + * Delay between two consecutive restart attempts. It can be specified using Scala's + * FiniteDuration notation: "1 min", "20 s" + */ + @PublicEvolving + public static final String RESTART_STRATEGY_FIXED_DELAY_DELAY = "restart-strategy.fixed-delay.delay"; + /** * Config parameter for the number of re-tries for failed tasks. Setting this * value to 0 effectively disables fault tolerance. + * + * @deprecated The configuration value will be replaced by {@link #RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS} + * and the corresponding FixedDelayRestartStrategy. */ + @Deprecated + @PublicEvolving public static final String EXECUTION_RETRIES_KEY = "execution-retries.default"; /** * Config parameter for the delay between execution retries. The value must be specified in the * notation "10 s" or "1 min" (style of Scala Finite Durations) + * + * @deprecated The configuration value will be replaced by {@link #RESTART_STRATEGY_FIXED_DELAY_DELAY} + * and the corresponding FixedDelayRestartStrategy. */ + @Deprecated + @PublicEvolving public static final String EXECUTION_RETRY_DELAY_KEY = "execution-retries.delay"; // -------------------------------- Runtime ------------------------------- @@ -268,8 +303,6 @@ public final class ConfigConstants { */ public static final String YARN_TASK_MANAGER_ENV_PREFIX = "yarn.taskmanager.env."; - - /** * The config parameter defining the Akka actor system port for the ApplicationMaster and * JobManager @@ -471,6 +504,9 @@ public final class ConfigConstants { /** Ports used by the job manager if not in standalone recovery mode */ public static final String RECOVERY_JOB_MANAGER_PORT = "recovery.jobmanager.port"; + /** The time before the JobManager recovers persisted jobs */ + public static final String RECOVERY_JOB_DELAY = "recovery.job.delay"; + // --------------------------- ZooKeeper ---------------------------------- /** ZooKeeper servers. */ @@ -521,7 +557,7 @@ public final class ConfigConstants { * The default number of execution retries. */ public static final int DEFAULT_EXECUTION_RETRIES = 0; - + // ------------------------------ Runtime --------------------------------- /** diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/ReadFromKafka.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/ReadFromKafka.java index 2179eca3026c1..0b6e7f733f676 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/ReadFromKafka.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/ReadFromKafka.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.examples.kafka; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -45,7 +46,7 @@ public static void main(String[] args) throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.getConfig().disableSysoutLogging(); - env.setNumberOfExecutionRetries(3); // retry if job fails + env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(4, 10000)); env.enableCheckpointing(5000); // create a checkpoint every 5 secodns env.getConfig().setGlobalJobParameters(parameterTool); // make parameters available in the web interface diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/WriteIntoKafka.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/WriteIntoKafka.java index 0a332652df453..697b8e075865a 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/WriteIntoKafka.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/WriteIntoKafka.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.examples.kafka; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -43,7 +44,7 @@ public static void main(String[] args) throws Exception { StreamExecutionEnvironment env =StreamExecutionEnvironment.getExecutionEnvironment(); env.getConfig().disableSysoutLogging(); - env.setNumberOfExecutionRetries(3); + env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(4, 10000)); // very simple data generator DataStream messageStream = env.addSource(new SourceFunction() { diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java index 512fe4254a0d7..f0006a46dc8e3 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java @@ -35,6 +35,7 @@ import org.apache.flink.api.common.io.FileInputFormat; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.operators.OperatorInformation; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.hadoop.mapred.HadoopInputFormat; @@ -181,13 +182,39 @@ public void setParallelism(int parallelism) { config.setParallelism(parallelism); } + /** + * Sets the restart strategy configuration. The configuration specifies which restart strategy + * will be used for the execution graph in case of a restart. + * + * @param restartStrategyConfiguration Restart strategy configuration to be set + */ + @PublicEvolving + public void setRestartStrategy(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration) { + config.setRestartStrategy(restartStrategyConfiguration); + } + + /** + * Returns the specified restart strategy configuration. + * + * @return The restart strategy configuration to be used + */ + @PublicEvolving + public RestartStrategies.RestartStrategyConfiguration getRestartStrategy() { + return config.getRestartStrategy(); + } + /** * Sets the number of times that failed tasks are re-executed. A value of zero * effectively disables fault tolerance. A value of {@code -1} indicates that the system * default value (as defined in the configuration) should be used. * * @param numberOfExecutionRetries The number of times the system will try to re-execute failed tasks. + * + * @deprecated This method will be replaced by {@link #setRestartStrategy}. The + * {@link RestartStrategies.FixedDelayRestartStrategyConfiguration} contains the number of + * execution retries. */ + @Deprecated @PublicEvolving public void setNumberOfExecutionRetries(int numberOfExecutionRetries) { config.setNumberOfExecutionRetries(numberOfExecutionRetries); @@ -199,7 +226,12 @@ public void setNumberOfExecutionRetries(int numberOfExecutionRetries) { * should be used. * * @return The number of times the system will try to re-execute failed tasks. + * + * @deprecated This method will be replaced by {@link #getRestartStrategy}. The + * {@link RestartStrategies.FixedDelayRestartStrategyConfiguration} contains the number of + * execution retries. */ + @Deprecated @PublicEvolving public int getNumberOfExecutionRetries() { return config.getNumberOfExecutionRetries(); diff --git a/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java b/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java index 0479c0be76097..91e2369058c6d 100644 --- a/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java +++ b/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java @@ -19,7 +19,9 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Random; + import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.LocalEnvironment; @@ -262,7 +264,7 @@ private void receiveParameters() throws IOException { break; case RETRY: int retry = (Integer) value.getField(1); - env.setNumberOfExecutionRetries(retry); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(retry, 10000L)); break; case DEBUG: DEBUG = (Boolean) value.getField(1); diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java index 159a94ae4ecb3..f59c347667510 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java @@ -218,8 +218,8 @@ public JobGraph compileJobGraph(OptimizedPlan program, JobID jobId) { // create the job graph object JobGraph graph = new JobGraph(jobId, program.getJobName()); - graph.setNumberOfExecutionRetries(program.getOriginalPlan().getNumberOfExecutionRetries()); - graph.setExecutionRetryDelay(program.getOriginalPlan().getExecutionRetryDelay()); + + graph.setRestartStrategyConfiguration(program.getOriginalPlan().getRestartStrategyConfiguration()); graph.setAllowQueuedScheduling(false); graph.setSessionTimeout(program.getOriginalPlan().getSessionTimeout()); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java index f39bb1a706f53..0f2f51443607f 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java @@ -50,7 +50,9 @@ public String handleRequest(ExecutionGraph graph, Map params) th gen.writeObjectFieldStart("execution-config"); gen.writeStringField("execution-mode", ec.getExecutionMode().name()); - gen.writeNumberField("max-execution-retries", ec.getNumberOfExecutionRetries()); + + final String restartStrategyDescription = ec.getRestartStrategy() != null ? ec.getRestartStrategy().getDescription() : "default"; + gen.writeStringField("restart-strategy", restartStrategyDescription); gen.writeNumberField("job-parallelism", ec.getParallelism()); gen.writeBooleanField("object-reuse-mode", ec.isObjectReuseEnabled()); diff --git a/flink-runtime-web/web-dashboard/app/partials/jobs/job.config.jade b/flink-runtime-web/web-dashboard/app/partials/jobs/job.config.jade index 83b78807685e8..96d03699ea0e3 100644 --- a/flink-runtime-web/web-dashboard/app/partials/jobs/job.config.jade +++ b/flink-runtime-web/web-dashboard/app/partials/jobs/job.config.jade @@ -28,7 +28,7 @@ table.table.table-properties(ng-if="job['execution-config']") tr td Max. number of execution retries - td {{ job['execution-config']['max-execution-retries'] === -1 ? 'deactivated' : job['execution-config']['max-execution-retries'] }} + td {{ job['execution-config']['restart-strategy'] }} tr td Job parallelism diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index c3bfbb045e1af..20288fb8fa1d8 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -42,6 +42,7 @@ import org.apache.flink.runtime.checkpoint.stats.SimpleCheckpointStatsTracker; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.UnrecoverableException; +import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -81,13 +82,10 @@ import java.util.HashSet; import java.util.NoSuchElementException; import java.util.UUID; -import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; -import static akka.dispatch.Futures.future; - /** * The execution graph is the central data structure that coordinates the distributed * execution of a data flow. It keeps representations of each parallel task, each @@ -183,12 +181,6 @@ public class ExecutionGraph implements Serializable { // ------ Configuration of the Execution ------- - /** The number of times failed executions should be retried. */ - private int numberOfRetriesLeft; - - /** The delay that the system should wait before restarting failed executions. */ - private long delayBeforeRetrying; - /** Flag to indicate whether the scheduler may queue tasks for execution, or needs to be able * to deploy them immediately. */ private boolean allowQueuedScheduling = false; @@ -219,6 +211,10 @@ public class ExecutionGraph implements Serializable { @SuppressWarnings("NonSerializableFieldInSerializableClass") private Scheduler scheduler; + /** Strategy to use for restarts */ + @SuppressWarnings("NonSerializableFieldInSerializableClass") + private RestartStrategy restartStrategy; + /** The classloader for the user code. Needed for calls into user code classes */ @SuppressWarnings("NonSerializableFieldInSerializableClass") private ClassLoader userClassLoader; @@ -255,13 +251,15 @@ public class ExecutionGraph implements Serializable { JobID jobId, String jobName, Configuration jobConfig, - FiniteDuration timeout) { + FiniteDuration timeout, + RestartStrategy restartStrategy) { this( executionContext, jobId, jobName, jobConfig, timeout, + restartStrategy, new ArrayList(), new ArrayList(), ExecutionGraph.class.getClassLoader() @@ -274,6 +272,7 @@ public ExecutionGraph( String jobName, Configuration jobConfig, FiniteDuration timeout, + RestartStrategy restartStrategy, List requiredJarFiles, List requiredClasspaths, ClassLoader userClassLoader) { @@ -304,6 +303,8 @@ public ExecutionGraph( this.requiredClasspaths = requiredClasspaths; this.timeout = timeout; + + this.restartStrategy = restartStrategy; } // -------------------------------------------------------------------------------------------- @@ -317,28 +318,6 @@ public ExecutionGraph( public int getNumberOfExecutionJobVertices() { return this.verticesInCreationOrder.size(); } - - public void setNumberOfRetriesLeft(int numberOfRetriesLeft) { - if (numberOfRetriesLeft < -1) { - throw new IllegalArgumentException(); - } - this.numberOfRetriesLeft = numberOfRetriesLeft; - } - - public int getNumberOfRetriesLeft() { - return numberOfRetriesLeft; - } - - public void setDelayBeforeRetrying(long delayBeforeRetrying) { - if (delayBeforeRetrying < 0) { - throw new IllegalArgumentException("Delay before retry must be non-negative."); - } - this.delayBeforeRetrying = delayBeforeRetrying; - } - - public long getDelayBeforeRetrying() { - return delayBeforeRetrying; - } public boolean isQueuedSchedulingAllowed() { return this.allowQueuedScheduling; @@ -477,6 +456,10 @@ public SavepointCoordinator getSavepointCoordinator() { return savepointCoordinator; } + public RestartStrategy getRestartStrategy() { + return restartStrategy; + } + public CheckpointStatsTracker getCheckpointStatsTracker() { return checkpointStatsTracker; } @@ -1029,40 +1012,13 @@ else if (current == JobStatus.CANCELLING) { else if (current == JobStatus.FAILING) { boolean isRecoverable = !(failureCause instanceof UnrecoverableException); - if (isRecoverable && numberOfRetriesLeft > 0 && + if (isRecoverable && restartStrategy.canRestart() && transitionState(current, JobStatus.RESTARTING)) { - - numberOfRetriesLeft--; - - if (delayBeforeRetrying > 0) { - future(new Callable() { - @Override - public Object call() throws Exception { - try { - LOG.info("Delaying retry of job execution for {} ms ...", delayBeforeRetrying); - Thread.sleep(delayBeforeRetrying); - } - catch(InterruptedException e){ - // should only happen on shutdown - } - restart(); - return null; - } - }, executionContext); - } else { - future(new Callable() { - @Override - public Object call() throws Exception { - restart(); - return null; - } - }, executionContext); - } + restartStrategy.restart(this); break; - } - else if ((!isRecoverable || numberOfRetriesLeft <= 0) && - transitionState(current, JobStatus.FAILED, failureCause)) { + } else if ((!isRecoverable || !restartStrategy.canRestart()) && + transitionState(current, JobStatus.FAILED, failureCause)) { postRunCleanup(); break; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java new file mode 100644 index 0000000000000..b5b00e4ea5289 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java @@ -0,0 +1,116 @@ +/* + * 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; + } + + public int getCurrentRestartAttempt() { + return currentRestartAttempt; + } + + @Override + public boolean canRestart() { + return currentRestartAttempt < maxNumberRestartAttempts; + } + + @Override + public void restart(final ExecutionGraph executionGraph) { + currentRestartAttempt++; + + future(new Callable() { + @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) { + // should only happen on shutdown + } + executionGraph.restart(); + return null; + } + }, executionGraph.getExecutionContext()); + } + + /** + * Creates a FixedDelayRestartStrategy from the given Configuration. + * + * @param configuration Configuration containing the parameter values for the restart strategy + * @return Initialized instance of FixedDelayRestartStrategy + * @throws Exception + */ + public static FixedDelayRestartStrategy create(Configuration configuration) throws Exception { + int maxAttempts = configuration.getInteger(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 1); + + String timeoutString = configuration.getString( + ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, + ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT); + + String delayString = configuration.getString( + ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, + timeoutString + ); + + long delay; + + try { + delay = Duration.apply(delayString).toMillis(); + } catch (NumberFormatException nfe) { + throw new Exception("Invalid config value for " + ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY + + ": " + delayString + ". Value must be a valid duration (such as 100 s or 1 min)."); + } + + return new FixedDelayRestartStrategy(maxAttempts, delay); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/NoRestartStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/NoRestartStrategy.java new file mode 100644 index 0000000000000..6be56eab7c5df --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/NoRestartStrategy.java @@ -0,0 +1,48 @@ +/* + * 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 org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; + +/** + * Restart strategy which does not restart an {@link ExecutionGraph}. + */ +public class NoRestartStrategy implements RestartStrategy { + + @Override + public boolean canRestart() { + return false; + } + + @Override + public void restart(ExecutionGraph executionGraph) { + throw new RuntimeException("NoRestartStrategy does not support restart."); + } + + /** + * Creates a NoRestartStrategy instance. + * + * @param configuration Configuration object which is ignored + * @return NoRestartStrategy instance + */ + public static NoRestartStrategy create(Configuration configuration) { + return new NoRestartStrategy(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/RestartStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/RestartStrategy.java new file mode 100644 index 0000000000000..2880c0128f246 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/RestartStrategy.java @@ -0,0 +1,41 @@ +/* + * 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 org.apache.flink.runtime.executiongraph.ExecutionGraph; + +/** + * Strategy for {@link ExecutionGraph} restarts. + */ +public interface RestartStrategy { + + /** + * True if the restart strategy can be applied to restart the {@link ExecutionGraph}. + * + * @return true if restart is possible, otherwise false + */ + boolean canRestart(); + + /** + * Restarts the given {@link ExecutionGraph}. + * + * @param executionGraph The ExecutionGraph to be restarted + */ + void restart(ExecutionGraph executionGraph); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/RestartStrategyFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/RestartStrategyFactory.java new file mode 100644 index 0000000000000..b9da63d5613fb --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/RestartStrategyFactory.java @@ -0,0 +1,127 @@ +/* + * 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 org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.concurrent.duration.Duration; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; + +public class RestartStrategyFactory { + private static final Logger LOG = LoggerFactory.getLogger(RestartStrategyFactory.class); + private static final String CREATE_METHOD = "create"; + + /** + * Creates a {@link RestartStrategy} instance from the given {@link org.apache.flink.api.common.restartstrategy.RestartStrategies.RestartStrategyConfiguration}. + * + * @param restartStrategyConfiguration Restart strategy configuration which specifies which + * restart strategy to instantiate + * @return RestartStrategy instance + */ + public static RestartStrategy createRestartStrategy(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration) { + if (restartStrategyConfiguration instanceof RestartStrategies.NoRestartStrategyConfiguration) { + return new NoRestartStrategy(); + } else if (restartStrategyConfiguration instanceof RestartStrategies.FixedDelayRestartStrategyConfiguration) { + RestartStrategies.FixedDelayRestartStrategyConfiguration fixedDelayConfig = + (RestartStrategies.FixedDelayRestartStrategyConfiguration) restartStrategyConfiguration; + + return new FixedDelayRestartStrategy( + fixedDelayConfig.getRestartAttempts(), + fixedDelayConfig.getDelayBetweenAttempts()); + } else { + throw new IllegalArgumentException("Unknown restart strategy configuration " + + restartStrategyConfiguration + "."); + } + } + + /** + * Creates a {@link RestartStrategy} instance from the given {@link Configuration}. + * + * @param configuration Configuration object containing the configuration values. + * @return RestartStrategy instance + * @throws Exception which indicates that the RestartStrategy could not be instantiated. + */ + public static RestartStrategy createFromConfig(Configuration configuration) throws Exception { + String restartStrategyName = configuration.getString(ConfigConstants.RESTART_STRATEGY, "none").toLowerCase(); + + switch (restartStrategyName) { + case "none": + // support deprecated ConfigConstants values + final int numberExecutionRetries = configuration.getInteger(ConfigConstants.EXECUTION_RETRIES_KEY, + ConfigConstants.DEFAULT_EXECUTION_RETRIES); + String pauseString = configuration.getString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, + ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT); + String delayString = configuration.getString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, + pauseString); + + long delay; + + try { + delay = Duration.apply(delayString).toMillis(); + } catch (NumberFormatException nfe) { + throw new Exception("Invalid config value for " + ConfigConstants.EXECUTION_RETRY_DELAY_KEY + + ": " + delayString + ". Value must be a valid duration (such as 100 s or 1 min)."); + } + + if (numberExecutionRetries > 0 && delay >= 0) { + return new FixedDelayRestartStrategy(numberExecutionRetries, delay); + } else { + return NoRestartStrategy.create(configuration); + } + case "off": + case "disable": + return NoRestartStrategy.create(configuration); + case "fixeddelay": + case "fixed-delay": + return FixedDelayRestartStrategy.create(configuration); + default: + try { + Class clazz = Class.forName(restartStrategyName); + + if (clazz != null) { + Method method = clazz.getMethod(CREATE_METHOD, Configuration.class); + + if (method != null) { + Object result = method.invoke(null, configuration); + + if (result != null) { + return (RestartStrategy) result; + } + } + } + } catch (ClassNotFoundException cnfe) { + LOG.warn("Could not find restart strategy class {}.", restartStrategyName); + } catch (NoSuchMethodException nsme) { + LOG.warn("Class {} does not has static method {}.", restartStrategyName, CREATE_METHOD); + } catch (InvocationTargetException ite) { + LOG.warn("Cannot call static method {} from class {}.", CREATE_METHOD, restartStrategyName); + } catch (IllegalAccessException iae) { + LOG.warn("Illegal access while calling method {} from class {}.", CREATE_METHOD, restartStrategyName); + } + + // fallback in case of an error + return NoRestartStrategy.create(configuration); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java index 403ad67c0c7ec..e20f73719e499 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java @@ -41,6 +41,7 @@ import java.util.Map; import java.util.Set; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; /** * The JobGraph represents a Flink dataflow program, at the low level that the JobManager accepts. * All programs from higher level APIs are transformed into JobGraphs. @@ -78,10 +79,8 @@ public class JobGraph implements Serializable { /** Name of this job. */ private final String jobName; - /** The number of times that failed tasks should be re-executed */ - private int numExecutionRetries; - - private long executionRetryDelay; + /** Configuration which defines which restart strategy to use for the job recovery */ + private RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration; /** The number of seconds after which the corresponding ExecutionGraph is removed at the * job manager after it has been executed. */ @@ -193,54 +192,22 @@ public Configuration getJobConfiguration() { } /** - * Sets the number of times that failed tasks are re-executed. A value of zero - * effectively disables fault tolerance. A value of {@code -1} indicates that the system - * default value (as defined in the configuration) should be used. - * - * @param numberOfExecutionRetries The number of times the system will try to re-execute failed tasks. - */ - public void setNumberOfExecutionRetries(int numberOfExecutionRetries) { - if (numberOfExecutionRetries < -1) { - throw new IllegalArgumentException( - "The number of execution retries must be non-negative, or -1 (use system default)"); - } - this.numExecutionRetries = numberOfExecutionRetries; - } - - /** - * Gets the number of times the system will try to re-execute failed tasks. A value - * of {@code -1} indicates that the system default value (as defined in the configuration) - * should be used. + * Sets the restart strategy configuration. This configuration specifies the restart strategy + * to be used by the ExecutionGraph in case of a restart. * - * @return The number of times the system will try to re-execute failed tasks. - */ - public int getNumberOfExecutionRetries() { - return numExecutionRetries; - } - - /** - * Gets the delay of time the system will try to re-execute failed tasks. A value of - * {@code -1} indicates the system default value (as defined in the configuration) - * should be used. - * @return The delay of time in milliseconds the system will try to re-execute failed tasks. + * @param restartStrategyConfiguration Restart strategy configuration to be set */ - public long getExecutionRetryDelay() { - return executionRetryDelay; + public void setRestartStrategyConfiguration(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration) { + this.restartStrategyConfiguration = restartStrategyConfiguration; } /** - * Sets the delay that failed tasks are re-executed. A value of zero - * effectively disables fault tolerance. A value of {@code -1} indicates that the system - * default value (as defined in the configuration) should be used. + * Gets the restart strategy configuration * - * @param executionRetryDelay The delay of time the system will wait to re-execute failed tasks. + * @return Restart strategy configuration to be used */ - public void setExecutionRetryDelay(long executionRetryDelay){ - if (executionRetryDelay < -1) { - throw new IllegalArgumentException( - "The delay between reties must be non-negative, or -1 (use system default)"); - } - this.executionRetryDelay = executionRetryDelay; + public RestartStrategies.RestartStrategyConfiguration getRestartStrategyConfiguration() { + return restartStrategyConfiguration; } /** diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 78612c0910f17..bd1816040600c 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -21,7 +21,7 @@ package org.apache.flink.runtime.jobmanager import java.io.{File, IOException} import java.net.{BindException, ServerSocket, UnknownHostException, InetAddress, InetSocketAddress} import java.util.UUID -import java.util.concurrent.ExecutorService +import java.util.concurrent.{TimeUnit, ExecutorService} import akka.actor.Status.Failure import akka.actor._ @@ -39,6 +39,7 @@ import org.apache.flink.runtime.checkpoint._ import org.apache.flink.runtime.client._ import org.apache.flink.runtime.execution.UnrecoverableException import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager +import org.apache.flink.runtime.executiongraph.restart.{RestartStrategy, RestartStrategyFactory} import org.apache.flink.runtime.executiongraph.{ExecutionGraph, ExecutionJobVertex} import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceManager} import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator @@ -108,13 +109,13 @@ class JobManager( protected val scheduler: FlinkScheduler, protected val libraryCacheManager: BlobLibraryCacheManager, protected val archive: ActorRef, - protected val defaultExecutionRetries: Int, - protected val delayBetweenRetries: Long, + protected val defaultRestartStrategy: RestartStrategy, protected val timeout: FiniteDuration, protected val leaderElectionService: LeaderElectionService, protected val submittedJobGraphs : SubmittedJobGraphStore, protected val checkpointRecoveryFactory : CheckpointRecoveryFactory, - protected val savepointStore: SavepointStore) + protected val savepointStore: SavepointStore, + protected val jobRecoveryTimeout: FiniteDuration) extends FlinkActor with LeaderSessionMessageFilter // mixin oder is important, we want filtering after logging with LogMessages // mixin order is important, we want first logging @@ -131,7 +132,7 @@ class JobManager( log.error("Executor could not execute task", t) } }) - + /** Either running or not yet archived jobs (session hasn't been ended). */ protected val currentJobs = scala.collection.mutable.HashMap[JobID, (ExecutionGraph, JobInfo)]() @@ -256,7 +257,7 @@ class JobManager( // shut down the extra thread pool for futures executorService.shutdown() - + log.debug(s"Job manager ${self.path} is completely stopped.") } @@ -280,10 +281,13 @@ class JobManager( // TODO (critical next step) This needs to be more flexible and robust (e.g. wait for task // managers etc.) if (recoveryMode != RecoveryMode.STANDALONE) { - log.info(s"Delaying recovery of all jobs for $delayBetweenRetries ms.") + log.info(s"Delaying recovery of all jobs by $jobRecoveryTimeout.") - context.system.scheduler.scheduleOnce(new FiniteDuration(delayBetweenRetries, - MILLISECONDS), self, decorateMessage(RecoverAllJobs))(context.dispatcher) + context.system.scheduler.scheduleOnce( + jobRecoveryTimeout, + self, + decorateMessage(RecoverAllJobs))( + context.dispatcher) } }(context.dispatcher) @@ -903,6 +907,12 @@ class JobManager( throw new JobSubmissionException(jobId, "The given job is empty") } + val restartStrategy = Option(jobGraph.getRestartStrategyConfiguration()) + .map(RestartStrategyFactory.createRestartStrategy(_)) match { + case Some(strategy) => strategy + case None => defaultRestartStrategy + } + // see if there already exists an ExecutionGraph for the corresponding job ID executionGraph = currentJobs.get(jobGraph.getJobID) match { case Some((graph, currentJobInfo)) => @@ -915,6 +925,7 @@ class JobManager( jobGraph.getName, jobGraph.getJobConfiguration, timeout, + restartStrategy, jobGraph.getUserJarBlobKeys, jobGraph.getClasspaths, userCodeLoader) @@ -923,22 +934,6 @@ class JobManager( graph } - // configure the execution graph - val jobNumberRetries = if (jobGraph.getNumberOfExecutionRetries() >= 0) { - jobGraph.getNumberOfExecutionRetries() - } else { - defaultExecutionRetries - } - - val executionRetryDelay = if (jobGraph.getExecutionRetryDelay() >= 0) { - jobGraph.getExecutionRetryDelay() - } - else { - delayBetweenRetries - } - - executionGraph.setNumberOfRetriesLeft(jobNumberRetries) - executionGraph.setDelayBeforeRetrying(executionRetryDelay) executionGraph.setScheduleMode(jobGraph.getScheduleMode()) executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling()) @@ -2032,14 +2027,15 @@ object JobManager { InstanceManager, FlinkScheduler, BlobLibraryCacheManager, - Int, // execution retries - Long, // delay between retries + RestartStrategy, FiniteDuration, // timeout Int, // number of archived jobs LeaderElectionService, SubmittedJobGraphStore, CheckpointRecoveryFactory, - SavepointStore) = { + SavepointStore, + FiniteDuration // timeout for job recovery + ) = { val timeout: FiniteDuration = AkkaUtils.getTimeout(configuration) @@ -2047,35 +2043,12 @@ object JobManager { ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL, ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000 - val executionRetries = configuration.getInteger( - ConfigConstants.EXECUTION_RETRIES_KEY, - ConfigConstants.DEFAULT_EXECUTION_RETRIES) + val restartStrategy = RestartStrategyFactory + .createFromConfig(configuration) val archiveCount = configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_ARCHIVE_COUNT, ConfigConstants.DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT) - // configure the delay between execution retries. - // unless explicitly specifies, this is dependent on the heartbeat timeout - val pauseString = configuration.getString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, - ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT) - val delayString = configuration.getString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, - pauseString) - - val delayBetweenRetries: Long = try { - Duration(delayString).toMillis - } - catch { - case n: NumberFormatException => - if (delayString.equals(pauseString)) { - throw new Exception( - s"Invalid config value for ${ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE}: " + - s"$pauseString. Value must be a valid duration (such as '10 s' or '1 min')") - } else { - throw new Exception( - s"Invalid config value for ${ConfigConstants.EXECUTION_RETRY_DELAY_KEY}: " + - s"$delayString. Value must be a valid duration (such as '100 milli' or '10 s')") - } - } var blobServer: BlobServer = null var instanceManager: InstanceManager = null @@ -2139,18 +2112,33 @@ object JobManager { val savepointStore = SavepointStoreFactory.createFromConfig(configuration) + val jobRecoveryTimeoutStr = configuration.getString(ConfigConstants.RECOVERY_JOB_DELAY, ""); + + val jobRecoveryTimeout = if (jobRecoveryTimeoutStr == null || jobRecoveryTimeoutStr.isEmpty) { + timeout + } else { + try { + FiniteDuration(Duration(jobRecoveryTimeoutStr).toMillis, TimeUnit.MILLISECONDS) + } catch { + case n: NumberFormatException => + throw new Exception( + s"Invalid config value for ${ConfigConstants.RECOVERY_JOB_DELAY}: " + + s"$jobRecoveryTimeoutStr. Value must be a valid duration (such as '10 s' or '1 min')") + } + } + (executorService, instanceManager, scheduler, libraryCacheManager, - executionRetries, - delayBetweenRetries, + restartStrategy, timeout, archiveCount, leaderElectionService, submittedJobGraphs, checkpointRecoveryFactory, - savepointStore) + savepointStore, + jobRecoveryTimeout) } /** @@ -2206,14 +2194,14 @@ object JobManager { instanceManager, scheduler, libraryCacheManager, - executionRetries, - delayBetweenRetries, + restartStrategy, timeout, archiveCount, leaderElectionService, submittedJobGraphs, checkpointRecoveryFactory, - savepointStore) = createJobManagerComponents( + savepointStore, + jobRecoveryTimeout) = createJobManagerComponents( configuration, None) @@ -2233,13 +2221,13 @@ object JobManager { scheduler, libraryCacheManager, archive, - executionRetries, - delayBetweenRetries, + restartStrategy, timeout, leaderElectionService, submittedJobGraphs, checkpointRecoveryFactory, - savepointStore) + savepointStore, + jobRecoveryTimeout) val jobManager: ActorRef = jobMangerActorName match { case Some(actorName) => actorSystem.actorOf(jobManagerProps, actorName) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java index e921e92f53485..26a626e256c78 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.jobmanager.RecoveryMode; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.junit.Test; @@ -43,14 +44,15 @@ public class ExecutionGraphCheckpointCoordinatorTest { @Test public void testCheckpointAndSavepointCoordinatorShareCheckpointIDCounter() throws Exception { ExecutionGraph executionGraph = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - new JobID(), - "test", - new Configuration(), - new FiniteDuration(1, TimeUnit.DAYS), - Collections.emptyList(), - Collections.emptyList(), - ClassLoader.getSystemClassLoader()); + TestingUtils.defaultExecutionContext(), + new JobID(), + "test", + new Configuration(), + new FiniteDuration(1, TimeUnit.DAYS), + new NoRestartStrategy(), + Collections.emptyList(), + Collections.emptyList(), + ClassLoader.getSystemClassLoader()); ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java index a47ea7769b880..2ca51db17c02b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java @@ -27,6 +27,7 @@ import static org.mockito.Mockito.when; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.junit.Test; @@ -105,7 +106,8 @@ public void testCreateSimpleGraphBipartite() { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); } @@ -148,7 +150,8 @@ public void testAttachViaDataSets() { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); } @@ -214,7 +217,8 @@ public void testAttachViaIds() { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); } @@ -467,7 +471,8 @@ public void testCannotConnectMissingId() { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); } @@ -522,7 +527,8 @@ public void testCannotConnectWrongOrder() { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); fail("Attached wrong jobgraph"); @@ -582,7 +588,8 @@ public void testSetupInputSplits() { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); } @@ -626,7 +633,8 @@ public void testMoreThanOneConsumerForIntermediateResult() { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); @@ -696,7 +704,8 @@ public void testCoLocationConstraintCreation() { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); eg.attachJobGraph(jg.getVerticesSortedTopologicallyFromSources()); // check the v1 / v2 co location hints ( assumes parallelism(v1) >= parallelism(v2) ) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java index bea7c22ae236b..9221bda3e1b4c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java @@ -36,6 +36,7 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -83,7 +84,8 @@ public void testBuildDeploymentDescriptor() { jobId, "some job", new Configuration(), - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); List ordered = Arrays.asList(v1, v2, v3, v4); @@ -285,7 +287,8 @@ private Map setupExecution(JobVertex v1, int dop1 jobId, "some job", new Configuration(), - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); eg.setQueuedSchedulingAllowed(false); List ordered = Arrays.asList(v1, v2); @@ -325,4 +328,4 @@ public void finalizeOnMaster(ClassLoader cl) throws Exception { throw new Exception(); } } -} \ No newline at end of file +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java index 47a48a299dca9..7cef66a5bb82b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java @@ -23,6 +23,9 @@ import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.UnrecoverableException; +import org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; +import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; @@ -32,6 +35,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.TestLogger; import org.junit.Test; import scala.concurrent.duration.Deadline; import scala.concurrent.duration.FiniteDuration; @@ -50,7 +54,7 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; -public class ExecutionGraphRestartTest { +public class ExecutionGraphRestartTest extends TestLogger { private final static int NUM_TASKS = 31; @@ -74,8 +78,8 @@ public void testNoManualRestart() throws Exception { new JobID(), "test job", new Configuration(), - AkkaUtils.getDefaultTimeout()); - eg.setNumberOfRetriesLeft(0); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); assertEquals(JobStatus.CREATED, eg.getState()); @@ -128,8 +132,8 @@ public void testConstraintsAfterRestart() throws Exception { new JobID(), "test job", new Configuration(), - AkkaUtils.getDefaultTimeout()); - eg.setNumberOfRetriesLeft(1); + AkkaUtils.getDefaultTimeout(), + new FixedDelayRestartStrategy(1, 0L)); eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); assertEquals(JobStatus.CREATED, eg.getState()); @@ -183,8 +187,8 @@ public void testRestartAutomatically() throws Exception { new JobID(), "Test job", new Configuration(), - AkkaUtils.getDefaultTimeout()); - eg.setNumberOfRetriesLeft(1); + AkkaUtils.getDefaultTimeout(), + new FixedDelayRestartStrategy(1, 1000)); eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); assertEquals(JobStatus.CREATED, eg.getState()); @@ -211,7 +215,9 @@ public void testCancelWhileRestarting() throws Exception { new JobID(), "TestJob", new Configuration(), - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + // We want to manually control the restart and delay + new FixedDelayRestartStrategy(Integer.MAX_VALUE, Long.MAX_VALUE)); JobVertex jobVertex = new JobVertex("NoOpInvokable"); jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); @@ -219,9 +225,6 @@ public void testCancelWhileRestarting() throws Exception { JobGraph jobGraph = new JobGraph("TestJob", jobVertex); - // We want to manually control the restart and delay - executionGraph.setNumberOfRetriesLeft(Integer.MAX_VALUE); - executionGraph.setDelayBeforeRetrying(Integer.MAX_VALUE); executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); assertEquals(JobStatus.CREATED, executionGraph.getState()); @@ -270,7 +273,9 @@ public void testCancelWhileFailing() throws Exception { new JobID(), "TestJob", new Configuration(), - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + // We want to manually control the restart and delay + new FixedDelayRestartStrategy(Integer.MAX_VALUE, Long.MAX_VALUE)); // Spy on the graph executionGraph = spy(executionGraph); @@ -285,9 +290,6 @@ public void testCancelWhileFailing() throws Exception { JobGraph jobGraph = new JobGraph("TestJob", jobVertex); - // We want to manually control the restart and delay - executionGraph.setNumberOfRetriesLeft(Integer.MAX_VALUE); - executionGraph.setDelayBeforeRetrying(Integer.MAX_VALUE); executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); assertEquals(JobStatus.CREATED, executionGraph.getState()); @@ -348,13 +350,13 @@ public void testNoRestartOnUnrecoverableException() throws Exception { JobGraph jobGraph = new JobGraph("Pointwise job", sender); ExecutionGraph eg = spy(new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - new JobID(), - "Test job", - new Configuration(), - AkkaUtils.getDefaultTimeout())); + TestingUtils.defaultExecutionContext(), + new JobID(), + "Test job", + new Configuration(), + AkkaUtils.getDefaultTimeout(), + new FixedDelayRestartStrategy(1, 1000))); - eg.setNumberOfRetriesLeft(1); eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); assertEquals(JobStatus.CREATED, eg.getState()); @@ -385,7 +387,12 @@ public void testNoRestartOnUnrecoverableException() throws Exception { // No restart verify(eg, never()).restart(); - assertEquals(1, eg.getNumberOfRetriesLeft()); + + RestartStrategy restartStrategy = eg.getRestartStrategy(); + + assertTrue(restartStrategy instanceof FixedDelayRestartStrategy); + + assertEquals(0, ((FixedDelayRestartStrategy) restartStrategy).getCurrentRestartAttempt()); } /** @@ -417,8 +424,9 @@ public void testFailingExecutionAfterRestart() throws Exception { new JobID(), "test job", new Configuration(), - AkkaUtils.getDefaultTimeout()); - eg.setNumberOfRetriesLeft(1); + AkkaUtils.getDefaultTimeout(), + new FixedDelayRestartStrategy(1, 1000)); + eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); assertEquals(JobStatus.CREATED, eg.getState()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index ad30b9e3f7b23..35ab8ac33fdc1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.instance.BaseTestingActorGateway; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.Instance; @@ -173,7 +174,8 @@ public static ExecutionJobVertex getExecutionVertex(JobVertexID id, ExecutionCon new JobID(), "test job", new Configuration(), - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); ExecutionJobVertex ejv = spy(new ExecutionJobVertex(graph, ajv, 1, AkkaUtils.getDefaultTimeout())); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java index 89b82f3990d4b..9c520b65166ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java @@ -26,6 +26,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.api.common.JobID; @@ -52,7 +53,8 @@ public void testAccumulatedStateFinished() { jid, "test job", new Configuration(), - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); graph.attachJobGraph(Arrays.asList(ajv)); @@ -82,4 +84,4 @@ public void testAccumulatedStateFinished() { fail(e.getMessage()); } } -} \ No newline at end of file +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java index 2530a53d73ef7..e0da2c9f799a4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java @@ -30,6 +30,7 @@ import org.apache.flink.core.io.InputSplitSource; import org.apache.flink.core.io.LocatableInputSplit; import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.InstanceConnectionInfo; @@ -270,7 +271,8 @@ public void testMultipleInstancesPerHost() { jobGraph.getJobID(), jobGraph.getName(), jobGraph.getJobConfiguration(), - TIMEOUT); + TIMEOUT, + new NoRestartStrategy()); eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); eg.setQueuedSchedulingAllowed(false); @@ -333,7 +335,8 @@ private static String[] runTests(int numHosts, int slotsPerHost, int parallelism jobGraph.getJobID(), jobGraph.getName(), jobGraph.getJobConfiguration(), - TIMEOUT); + TIMEOUT, + new NoRestartStrategy()); eg.setQueuedSchedulingAllowed(false); eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java index 3cedb6313124c..eda911595d7a6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java @@ -23,6 +23,7 @@ import static org.junit.Assert.fail; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.junit.Test; @@ -62,7 +63,8 @@ public void testNToN() { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); } @@ -102,7 +104,8 @@ public void test2NToN() { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); } @@ -143,7 +146,8 @@ public void test3NToN() { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); } @@ -185,7 +189,8 @@ public void testNTo2N() { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); } @@ -225,7 +230,8 @@ public void testNTo7N() { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); } @@ -285,7 +291,8 @@ private void testLowToHigh(int lowDop, int highDop) { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); } @@ -336,7 +343,8 @@ private void testHighToLow(int highDop, int lowDop) { jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); try { eg.attachJobGraph(ordered); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java index f42543f311d04..2f0d5e707758e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy; import org.apache.flink.runtime.instance.DummyActorGateway; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.Instance; @@ -124,9 +125,7 @@ public void testProvokeDeadlock() { for (int i = 0; i < 20000; i++) { final TestExecGraph eg = new TestExecGraph(jobId); eg.attachJobGraph(vertices); - eg.setDelayBeforeRetrying(0); - eg.setNumberOfRetriesLeft(1); - + final Execution e1 = eg.getJobVertex(vid1).getTaskVertices()[0].getCurrentExecutionAttempt(); final Execution e2 = eg.getJobVertex(vid2).getTaskVertices()[0].getCurrentExecutionAttempt(); @@ -181,7 +180,13 @@ static class TestExecGraph extends ExecutionGraph { private volatile boolean done; TestExecGraph(JobID jobId) { - super(TestingUtils.defaultExecutionContext(), jobId, "test graph", EMPTY_CONFIG, TIMEOUT); + super( + TestingUtils.defaultExecutionContext(), + jobId, + "test graph", + EMPTY_CONFIG, + TIMEOUT, + new FixedDelayRestartStrategy(1, 0)); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java index 8604b63a111b7..36703585a6009 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.concurrent.TimeUnit; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.instance.DummyActorGateway; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.Instance; @@ -80,7 +81,8 @@ public void testScheduleWithConstraint1() { jg.getJobID(), jg.getName(), jg.getJobConfiguration(), - timeout); + timeout, + new NoRestartStrategy()); eg.attachJobGraph(Collections.singletonList(jobVertex)); ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); @@ -151,7 +153,8 @@ public void testScheduleWithConstraint2() { jg.getJobID(), jg.getName(), jg.getJobConfiguration(), - timeout); + timeout, + new NoRestartStrategy()); eg.attachJobGraph(Collections.singletonList(jobVertex)); ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); @@ -226,7 +229,8 @@ public void testScheduleWithConstraintAndSlotSharing() { jg.getJobID(), jg.getName(), jg.getJobConfiguration(), - timeout); + timeout, + new NoRestartStrategy()); eg.attachJobGraph(Arrays.asList(jobVertex1, jobVertex2)); ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex1.getID()); @@ -292,7 +296,8 @@ public void testScheduleWithUnfulfillableConstraint() { jg.getJobID(), jg.getName(), jg.getJobConfiguration(), - timeout); + timeout, + new NoRestartStrategy()); eg.attachJobGraph(Collections.singletonList(jobVertex)); ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); @@ -360,7 +365,8 @@ public void testScheduleWithUnfulfillableConstraintInSharingGroup() { jg.getJobID(), jg.getName(), jg.getJobConfiguration(), - timeout); + timeout, + new NoRestartStrategy()); eg.attachJobGraph(Arrays.asList(jobVertex1, jobVertex2)); ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex1.getID()); @@ -398,7 +404,8 @@ public void testArchivingClearsFields() { jg.getJobID(), jg.getName(), jg.getJobConfiguration(), - timeout); + timeout, + new NoRestartStrategy()); eg.attachJobGraph(Collections.singletonList(vertex)); ExecutionVertex ev = eg.getAllVertices().get(vertex.getID()).getTaskVertices()[0]; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java index d9e422c021949..5c7297e5b3136 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java @@ -26,6 +26,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.api.common.JobID; @@ -74,7 +75,8 @@ public void testAssignSlotSharingGroup() { new JobID(), "test job", new Configuration(), - AkkaUtils.getDefaultTimeout()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); eg.attachJobGraph(vertices); // verify that the vertices are all in the same slot sharing group diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategyTest.java new file mode 100644 index 0000000000000..d19060ada7d7a --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategyTest.java @@ -0,0 +1,51 @@ +/* + * 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 static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.common.util.concurrent.MoreExecutors; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.junit.Test; +import org.mockito.Mockito; +import scala.concurrent.ExecutionContext$; + +public class FixedDelayRestartStrategyTest { + + @Test + public void testFixedDelayRestartStrategy() { + int numberRestarts = 10; + long restartDelay = 10; + + FixedDelayRestartStrategy fixedDelayRestartStrategy = new FixedDelayRestartStrategy( + numberRestarts, + restartDelay); + + ExecutionGraph executionGraph = mock(ExecutionGraph.class); + when(executionGraph.getExecutionContext()) + .thenReturn(ExecutionContext$.MODULE$.fromExecutor(MoreExecutors.directExecutor())); + + while(fixedDelayRestartStrategy.canRestart()) { + fixedDelayRestartStrategy.restart(executionGraph); + } + + Mockito.verify(executionGraph, Mockito.times(numberRestarts)).restart(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index 652006629118a..8750a1a1e1c2b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -18,14 +18,11 @@ package org.apache.flink.runtime.io.network.partition; -import org.apache.flink.api.common.JobID; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.event.AbstractEvent; -import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; -import org.apache.flink.runtime.io.network.util.TestBufferFactory; import org.apache.flink.runtime.io.network.util.TestConsumerCallback; import org.apache.flink.runtime.io.network.util.TestNotificationListener; import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider; @@ -39,7 +36,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import static org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode.ASYNC; import static org.apache.flink.runtime.io.network.util.TestBufferFactory.createBuffer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java index 73c764613e038..fe35c0ded347c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.checkpoint.SavepointStoreFactory; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.instance.InstanceManager; import org.apache.flink.runtime.jobmanager.RecoveryMode; import org.apache.flink.runtime.jobmanager.StandaloneSubmittedJobGraphStore; @@ -191,13 +192,13 @@ private Props createJobManagerProps(Configuration configuration) throws Exceptio new Scheduler(TestingUtils.defaultExecutionContext()), new BlobLibraryCacheManager(new BlobServer(configuration), 10L), ActorRef.noSender(), - 1, - 1L, + new NoRestartStrategy(), AkkaUtils.getDefaultTimeout(), leaderElectionService, submittedJobGraphStore, checkpointRecoveryFactory, - savepointStore + savepointStore, + AkkaUtils.getDefaultTimeout() ); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java index 75569ecc11488..da3c02d995644 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java @@ -88,7 +88,7 @@ public static Configuration setZooKeeperRecoveryMode( config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, "6 s"); config.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 9); config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, "100 s"); - config.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "10 s"); + config.setString(ConfigConstants.RECOVERY_JOB_DELAY, "10 s"); return config; } diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala index 177dc851ec19d..dbd87d0262421 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala @@ -22,6 +22,7 @@ import org.apache.flink.api.common.JobID import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.akka.AkkaUtils import org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.SimpleActorGateway +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy import org.apache.flink.runtime.jobgraph.{JobStatus, JobGraph, JobVertex} import org.apache.flink.runtime.jobmanager.Tasks import org.apache.flink.runtime.jobmanager.scheduler.Scheduler @@ -56,8 +57,9 @@ class TaskManagerLossFailsTasksTest extends WordSpecLike with Matchers { new JobID(), "test job", new Configuration(), - AkkaUtils.getDefaultTimeout) - eg.setNumberOfRetriesLeft(0) + AkkaUtils.getDefaultTimeout, + new NoRestartStrategy()) + eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources) eg.getState should equal(JobStatus.CREATED) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala index 68923fe11c4b6..41b6702d71437 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala @@ -33,6 +33,8 @@ import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike} import org.scalatest.junit.JUnitRunner import scala.concurrent.duration._ +import language.postfixOps + @RunWith(classOf[JUnitRunner]) class RecoveryITCase(_system: ActorSystem) extends TestKit(_system) @@ -57,7 +59,9 @@ class RecoveryITCase(_system: ActorSystem) config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots) config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskManagers) config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, heartbeatTimeout) - config.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, heartbeatTimeout) + config.setString(ConfigConstants.RESTART_STRATEGY, "fixeddelay") + config.setInteger(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 1) + config.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, heartbeatTimeout) new TestingCluster(config) } @@ -79,7 +83,6 @@ class RecoveryITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) val jobGraph = new JobGraph("Pointwise job", sender, receiver) - jobGraph.setNumberOfExecutionRetries(1) val cluster = createTestClusterWithHeartbeatTimeout(2 * NUM_TASKS, 1, "2 s") cluster.start() @@ -124,7 +127,6 @@ class RecoveryITCase(_system: ActorSystem) receiver.setSlotSharingGroup(sharingGroup) val jobGraph = new JobGraph("Pointwise job", sender, receiver) - jobGraph.setNumberOfExecutionRetries(1) val cluster = createTestClusterWithHeartbeatTimeout(NUM_TASKS, 1, "2 s") cluster.start() @@ -169,7 +171,6 @@ class RecoveryITCase(_system: ActorSystem) receiver.setSlotSharingGroup(sharingGroup) val jobGraph = new JobGraph("Pointwise job", sender, receiver) - jobGraph.setNumberOfExecutionRetries(1) val cluster = createTestClusterWithHeartbeatTimeout(NUM_TASKS, 2, "2 s") cluster.start() diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala index cfb1192770ded..7b7750a4b117f 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala @@ -96,14 +96,14 @@ class TestingCluster( instanceManager, scheduler, libraryCacheManager, - executionRetries, - delayBetweenRetries, + restartStrategy, timeout, archiveCount, leaderElectionService, submittedJobsGraphs, checkpointRecoveryFactory, - savepointStore) = JobManager.createJobManagerComponents( + savepointStore, + jobRecoveryTimeout) = JobManager.createJobManagerComponents( config, createLeaderElectionService()) @@ -118,13 +118,13 @@ class TestingCluster( scheduler, libraryCacheManager, archive, - executionRetries, - delayBetweenRetries, + restartStrategy, timeout, leaderElectionService, submittedJobsGraphs, checkpointRecoveryFactory, - savepointStore)) + savepointStore, + jobRecoveryTimeout)) val dispatcherJobManagerProps = if (synchronousDispatcher) { // disable asynchronous futures (e.g. accumulator update in Heartbeat) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala index 98d886375d48a..53867e0afeb0c 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala @@ -23,6 +23,7 @@ import akka.actor.ActorRef import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.checkpoint.{SavepointStore, CheckpointRecoveryFactory} import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager +import org.apache.flink.runtime.executiongraph.restart.RestartStrategy import org.apache.flink.runtime.instance.InstanceManager import org.apache.flink.runtime.jobmanager.scheduler.Scheduler import org.apache.flink.runtime.jobmanager.{JobManager, SubmittedJobGraphStore} @@ -43,13 +44,13 @@ class TestingJobManager( scheduler: Scheduler, libraryCacheManager: BlobLibraryCacheManager, archive: ActorRef, - defaultExecutionRetries: Int, - delayBetweenRetries: Long, + restartStrategy: RestartStrategy, timeout: FiniteDuration, leaderElectionService: LeaderElectionService, submittedJobGraphs : SubmittedJobGraphStore, checkpointRecoveryFactory : CheckpointRecoveryFactory, - savepointStore : SavepointStore) + savepointStore : SavepointStore, + jobRecoveryTimeout: FiniteDuration) extends JobManager( flinkConfiguration, executorService, @@ -57,11 +58,11 @@ class TestingJobManager( scheduler, libraryCacheManager, archive, - defaultExecutionRetries, - delayBetweenRetries, + restartStrategy, timeout, leaderElectionService, submittedJobGraphs, checkpointRecoveryFactory, - savepointStore) + savepointStore, + jobRecoveryTimeout) with TestingJobManagerLike {} diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala index 6057f65c1a2f0..d68211235dcde 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala @@ -323,14 +323,14 @@ object TestingUtils { instanceManager, scheduler, libraryCacheManager, - executionRetries, - delayBetweenRetries, + restartStrategy, timeout, archiveCount, leaderElectionService, submittedJobGraphs, checkpointRecoveryFactory, - savepointStore) = JobManager.createJobManagerComponents( + savepointStore, + jobRecoveryTimeout) = JobManager.createJobManagerComponents( configuration, None ) @@ -347,12 +347,12 @@ object TestingUtils { scheduler, libraryCacheManager, archive, - executionRetries, - delayBetweenRetries, + restartStrategy, timeout, leaderElectionService, submittedJobGraphs, - checkpointRecoveryFactory) + checkpointRecoveryFactory, + jobRecoveryTimeout) val jobManager: ActorRef = actorSystem.actorOf(jobManagerProps, JobManager.JOB_MANAGER_NAME) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala index b3d2430939ecd..5c923cbd2562f 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala @@ -21,6 +21,7 @@ import com.esotericsoftware.kryo.Serializer import com.google.common.base.Preconditions import org.apache.flink.annotation.{PublicEvolving, Public} import org.apache.flink.api.common.io.{FileInputFormat, InputFormat} +import org.apache.flink.api.common.restartstrategy.RestartStrategies.RestartStrategyConfiguration import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.common.typeutils.CompositeType import org.apache.flink.api.common.{ExecutionConfig, JobExecutionResult, JobID} @@ -93,20 +94,49 @@ class ExecutionEnvironment(javaEnv: JavaEnv) { def getParallelism = javaEnv.getParallelism /** - * Sets the number of times that failed tasks are re-executed. A value of zero - * effectively disables fault tolerance. A value of "-1" indicates that the system - * default value (as defined in the configuration) should be used. - */ + * Sets the restart strategy configuration. The configuration specifies which restart strategy + * will be used for the execution graph in case of a restart. + * + * @param restartStrategyConfiguration Restart strategy configuration to be set + */ + @PublicEvolving + def setRestartStrategy(restartStrategyConfiguration: RestartStrategyConfiguration): Unit = { + javaEnv.setRestartStrategy(restartStrategyConfiguration) + } + + /** + * Returns the specified restart strategy configuration. + * + * @return The restart strategy configuration to be used + */ + @PublicEvolving + def getRestartStrategy: RestartStrategyConfiguration = { + javaEnv.getRestartStrategy() + } + + /** + * Sets the number of times that failed tasks are re-executed. A value of zero + * effectively disables fault tolerance. A value of "-1" indicates that the system + * default value (as defined in the configuration) should be used. + * + * @deprecated This method will be replaced by [[setRestartStrategy()]]. The + * FixedDelayRestartStrategyConfiguration contains the number of execution retries. + */ + @Deprecated @PublicEvolving def setNumberOfExecutionRetries(numRetries: Int): Unit = { javaEnv.setNumberOfExecutionRetries(numRetries) } /** - * Gets the number of times the system will try to re-execute failed tasks. A value - * of "-1" indicates that the system default value (as defined in the configuration) - * should be used. - */ + * Gets the number of times the system will try to re-execute failed tasks. A value + * of "-1" indicates that the system default value (as defined in the configuration) + * should be used. + * + * @deprecated This method will be replaced by [[getRestartStrategy]]. The + * FixedDelayRestartStrategyConfiguration contains the number of execution retries. + */ + @Deprecated @PublicEvolving def getNumberOfExecutionRetries = javaEnv.getNumberOfExecutionRetries diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java index a3e815ec05615..a0caadd4bfc12 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java @@ -18,22 +18,15 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.curator.framework.CuratorFramework; -import org.apache.flink.api.java.functions.FlatMapIterator; -import org.apache.flink.api.java.operators.DataSource; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; -import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler; import org.junit.Test; -import java.util.Iterator; import java.util.Properties; import static org.junit.Assert.assertTrue; - public class Kafka08ITCase extends KafkaConsumerTestBase { // ------------------------------------------------------------------------ @@ -166,19 +159,19 @@ public void testOffsetInZookeeper() throws Exception { StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env1.getConfig().disableSysoutLogging(); env1.enableCheckpointing(50); - env1.setNumberOfExecutionRetries(0); + env1.getConfig().setRestartStrategy(RestartStrategies.noRestart()); env1.setParallelism(parallelism); StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env2.getConfig().disableSysoutLogging(); env2.enableCheckpointing(50); - env2.setNumberOfExecutionRetries(0); + env2.getConfig().setRestartStrategy(RestartStrategies.noRestart()); env2.setParallelism(parallelism); StreamExecutionEnvironment env3 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env3.getConfig().disableSysoutLogging(); env3.enableCheckpointing(50); - env3.setNumberOfExecutionRetries(0); + env3.getConfig().setRestartStrategy(RestartStrategies.noRestart()); env3.setParallelism(parallelism); // write a sequence from 0 to 99 to each of the 3 partitions. @@ -222,13 +215,13 @@ public void testOffsetAutocommitTest() throws Exception { StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env1.getConfig().disableSysoutLogging(); - env1.setNumberOfExecutionRetries(0); + env1.getConfig().setRestartStrategy(RestartStrategies.noRestart()); env1.setParallelism(parallelism); StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); // NOTE: We are not enabling the checkpointing! env2.getConfig().disableSysoutLogging(); - env2.setNumberOfExecutionRetries(0); + env2.getConfig().setRestartStrategy(RestartStrategies.noRestart()); env2.setParallelism(parallelism); diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index 680e4ec4e68c2..7c8cdb2a21ed1 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -33,6 +33,7 @@ import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -136,7 +137,7 @@ public void runFailOnNoBrokerTest() throws Exception { StreamExecutionEnvironment see = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); see.getConfig().disableSysoutLogging(); - see.setNumberOfExecutionRetries(0); + see.setRestartStrategy(RestartStrategies.noRestart()); see.setParallelism(1); // use wrong ports for the consumers @@ -255,7 +256,7 @@ public void runSimpleConcurrentProducerConsumerTopology() throws Exception { StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.setParallelism(parallelism); env.enableCheckpointing(500); - env.setNumberOfExecutionRetries(0); // fail immediately + env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately env.getConfig().disableSysoutLogging(); TypeInformation> longStringType = TypeInfoParser.parse("Tuple2"); @@ -385,7 +386,7 @@ public void runOneToOneExactlyOnceTest() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.enableCheckpointing(500); env.setParallelism(parallelism); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 1000)); env.getConfig().disableSysoutLogging(); FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps); @@ -430,7 +431,7 @@ public void runOneSourceMultiplePartitionsExactlyOnceTest() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.enableCheckpointing(500); env.setParallelism(parallelism); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 1000)); env.getConfig().disableSysoutLogging(); FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps); @@ -475,7 +476,7 @@ public void runMultipleSourcesOnePartitionExactlyOnceTest() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.enableCheckpointing(500); env.setParallelism(parallelism); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 1000)); env.getConfig().disableSysoutLogging(); env.setBufferTimeout(0); @@ -781,7 +782,7 @@ public void runBigRecordTestTopology() throws Exception { new TypeInformationSerializationSchema<>(longBytesInfo, new ExecutionConfig()); final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); - env.setNumberOfExecutionRetries(0); + env.setRestartStrategy(RestartStrategies.noRestart()); env.getConfig().disableSysoutLogging(); env.enableCheckpointing(100); env.setParallelism(parallelism); @@ -895,7 +896,7 @@ public void runBrokerFailureTest() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.setParallelism(parallelism); env.enableCheckpointing(500); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 1000)); env.getConfig().disableSysoutLogging(); @@ -923,7 +924,7 @@ public void runKeyValueTest() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.setParallelism(1); - env.setNumberOfExecutionRetries(0); + env.setRestartStrategy(RestartStrategies.noRestart()); env.getConfig().disableSysoutLogging(); DataStream> kvStream = env.addSource(new SourceFunction>() { @@ -953,7 +954,7 @@ public void cancel() { env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.setParallelism(1); - env.setNumberOfExecutionRetries(0); + env.setRestartStrategy(RestartStrategies.noRestart()); env.getConfig().disableSysoutLogging(); @@ -1005,7 +1006,7 @@ public void runAllDeletesTest() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.setParallelism(1); - env.setNumberOfExecutionRetries(0); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); env.getConfig().disableSysoutLogging(); DataStream> kvStream = env.addSource(new SourceFunction>() { @@ -1038,7 +1039,7 @@ public void cancel() { env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.setParallelism(1); - env.setNumberOfExecutionRetries(0); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); env.getConfig().disableSysoutLogging(); DataStream> fromKafka = env.addSource(kafkaServer.getConsumer(topic, schema, standardProps)); @@ -1076,7 +1077,7 @@ public void runMetricsAndEndOfStreamTest() throws Exception { // write some data StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.setParallelism(1); - env.setNumberOfExecutionRetries(0); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); env.getConfig().disableSysoutLogging(); writeSequence(env, topic, ELEMENT_COUNT, 1); @@ -1084,7 +1085,7 @@ public void runMetricsAndEndOfStreamTest() throws Exception { // read using custom schema final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env1.setParallelism(1); - env1.setNumberOfExecutionRetries(0); + env1.getConfig().setRestartStrategy(RestartStrategies.noRestart()); env1.getConfig().disableSysoutLogging(); DataStream> fromKafka = env.addSource(kafkaServer.getConsumer(topic, new FixedNumberDeserializationSchema(ELEMENT_COUNT), standardProps)); diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java index 228f3ac01788e..e251174c5140d 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeInfoParser; @@ -70,7 +71,7 @@ public void runCustomPartitioningTest() { TypeInformation> longStringInfo = TypeInfoParser.parse("Tuple2"); StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); - env.setNumberOfExecutionRetries(0); + env.setRestartStrategy(RestartStrategies.noRestart()); env.getConfig().disableSysoutLogging(); TypeInformationSerializationSchema> serSchema = diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index ab1d5b68c2cc3..1108e2f71e346 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -104,7 +104,7 @@ public static void prepare() throws IOException, ClassNotFoundException { flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8); flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16); - flinkConfig.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "0 s"); + flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s"); flink = new ForkableFlinkMiniCluster(flinkConfig, false); flink.start(); diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java index cc237b6ebe7af..c489a279aef9d 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.kafka.testutils; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; @@ -48,7 +49,7 @@ public static void generateLongStringTupleSequence(StreamExecutionEnvironment en env.setParallelism(numPartitions); env.getConfig().disableSysoutLogging(); - env.setNumberOfExecutionRetries(0); + env.setRestartStrategy(RestartStrategies.noRestart()); DataStream> stream =env.addSource( new RichParallelSourceFunction>() { @@ -90,7 +91,7 @@ public static void generateRandomizedIntegerSequence(StreamExecutionEnvironment final boolean randomizeOrder) throws Exception { env.setParallelism(numPartitions); env.getConfig().disableSysoutLogging(); - env.setNumberOfExecutionRetries(0); + env.setRestartStrategy(RestartStrategies.noRestart()); DataStream stream = env.addSource( new RichParallelSourceFunction() { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index 8d829c6ec0ac1..82e536c51add4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -29,6 +29,7 @@ import org.apache.flink.api.common.functions.InvalidTypesException; import org.apache.flink.api.common.io.FileInputFormat; import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -423,6 +424,27 @@ public AbstractStateBackend getStateBackend() { return defaultStateBackend; } + /** + * Sets the restart strategy configuration. The configuration specifies which restart strategy + * will be used for the execution graph in case of a restart. + * + * @param restartStrategyConfiguration Restart strategy configuration to be set + */ + @PublicEvolving + public void setRestartStrategy(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration) { + config.setRestartStrategy(restartStrategyConfiguration); + } + + /** + * Returns the specified restart strategy configuration. + * + * @return The restart strategy configuration to be used + */ + @PublicEvolving + public RestartStrategies.RestartStrategyConfiguration getRestartStrategy() { + return config.getRestartStrategy(); + } + /** * Sets the number of times that failed tasks are re-executed. A value of * zero effectively disables fault tolerance. A value of {@code -1} @@ -431,7 +453,12 @@ public AbstractStateBackend getStateBackend() { * * @param numberOfExecutionRetries * The number of times the system will try to re-execute failed tasks. + * + * @deprecated This method will be replaced by {@link #setRestartStrategy}. The + * {@link RestartStrategies.FixedDelayRestartStrategyConfiguration} contains the number of + * execution retries. */ + @Deprecated @PublicEvolving public void setNumberOfExecutionRetries(int numberOfExecutionRetries) { config.setNumberOfExecutionRetries(numberOfExecutionRetries); @@ -443,7 +470,12 @@ public void setNumberOfExecutionRetries(int numberOfExecutionRetries) { * in the configuration) should be used. * * @return The number of times the system will try to re-execute failed tasks. + * + * @deprecated This method will be replaced by {@link #getRestartStrategy}. The + * {@link RestartStrategies.FixedDelayRestartStrategyConfiguration} contains the number of + * execution retries. */ + @Deprecated @PublicEvolving public int getNumberOfExecutionRetries() { return config.getNumberOfExecutionRetries(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index 04b1917437417..a21e3b74462e0 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -24,6 +24,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; @@ -75,6 +76,12 @@ public class StreamingJobGraphGenerator { private static final Logger LOG = LoggerFactory.getLogger(StreamingJobGraphGenerator.class); + /** + * Restart delay used for the FixedDelayRestartStrategy in case checkpointing was enabled but + * no restart strategy has been specified. + */ + private static final long DEFAULT_RESTART_DELAY = 10000L; + private StreamGraph streamGraph; private Map jobVertices; @@ -121,9 +128,7 @@ public JobGraph createJobGraph(String jobName) { configureCheckpointing(); - configureExecutionRetries(); - - configureExecutionRetryDelay(); + configureRestartStrategy(); try { InstantiationUtil.writeObjectToConfig(this.streamGraph.getExecutionConfig(), this.jobGraph.getJobConfiguration(), ExecutionConfig.CONFIG_KEY); @@ -477,22 +482,17 @@ private void configureCheckpointing() { cfg.getMaxConcurrentCheckpoints()); jobGraph.setSnapshotSettings(settings); - // if the user enabled checkpointing, the default number of exec retries is infinitive. - int executionRetries = streamGraph.getExecutionConfig().getNumberOfExecutionRetries(); - if(executionRetries == -1) { - streamGraph.getExecutionConfig().setNumberOfExecutionRetries(Integer.MAX_VALUE); + // check if a restart strategy has been set, if not then set the FixedDelayRestartStrategy + if (streamGraph.getExecutionConfig().getRestartStrategy() == null) { + // if the user enabled checkpointing, the default number of exec retries is infinitive. + streamGraph.getExecutionConfig().setRestartStrategy( + RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, DEFAULT_RESTART_DELAY)); } } } - private void configureExecutionRetries() { - int executionRetries = streamGraph.getExecutionConfig().getNumberOfExecutionRetries(); - jobGraph.setNumberOfExecutionRetries(executionRetries); - } - - private void configureExecutionRetryDelay() { - long executionRetryDelay = streamGraph.getExecutionConfig().getExecutionRetryDelay(); - jobGraph.setExecutionRetryDelay(executionRetryDelay); + private void configureRestartStrategy() { + jobGraph.setRestartStrategyConfiguration(streamGraph.getExecutionConfig().getRestartStrategy()); } // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java index bac7fa5edfb8b..184f87e6ef39c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java @@ -29,6 +29,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.plugable.SerializationDelegate; @@ -133,7 +134,11 @@ public void flatMap(String value, jobId, jobName, cfg, - AkkaUtils.getDefaultTimeout(),new ArrayList(), new ArrayList(), ExecutionGraph.class.getClassLoader()); + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy(), + new ArrayList(), + new ArrayList(), + ExecutionGraph.class.getClassLoader()); try { eg.attachJobGraph(jobVertices); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/state/StateBackendITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/state/StateBackendITCase.java index 3aa60dc2059ca..222924c05e19c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/state/StateBackendITCase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/state/StateBackendITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.FoldingState; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.FoldingStateDescriptor; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -56,7 +57,7 @@ public void testStateBackendWithoutCheckpointing() throws Exception { StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); see.setParallelism(1); - see.setNumberOfExecutionRetries(0); + see.getConfig().setRestartStrategy(RestartStrategies.noRestart()); see.setStateBackend(new FailingStateBackend()); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java index 0e03b71bd19e6..eed455c463b51 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java @@ -79,7 +79,6 @@ public static void startCluster() { Configuration config = new Configuration(); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); - config.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "0 ms"); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); cluster = new ForkableFlinkMiniCluster(config, false); diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala index 58b100ed12183..39dd6cc53080e 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala @@ -21,6 +21,7 @@ package org.apache.flink.streaming.api.scala import com.esotericsoftware.kryo.Serializer import org.apache.flink.annotation.{Internal, PublicEvolving, Public} import org.apache.flink.api.common.io.{FileInputFormat, InputFormat} +import org.apache.flink.api.common.restartstrategy.RestartStrategies.RestartStrategyConfiguration import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer import org.apache.flink.api.scala.ClosureCleaner @@ -186,8 +187,7 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) { * operator states. Time interval between state checkpoints is specified in in millis. * * Setting this option assumes that the job is used in production and thus if not stated - * explicitly otherwise with calling with the - * [[setNumberOfExecutionRetries(int)]] method in case of + * explicitly otherwise with calling the [[setRestartStrategy]] method in case of * failure the job will be resubmitted to the cluster indefinitely. */ @deprecated @@ -229,22 +229,49 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) { */ @PublicEvolving def getStateBackend: AbstractStateBackend = javaEnv.getStateBackend() - + /** - * Sets the number of times that failed tasks are re-executed. A value of zero - * effectively disables fault tolerance. A value of "-1" indicates that the system - * default value (as defined in the configuration) should be used. - */ + * Sets the restart strategy configuration. The configuration specifies which restart strategy + * will be used for the execution graph in case of a restart. + * + * @param restartStrategyConfiguration Restart strategy configuration to be set + */ + @PublicEvolving + def setRestartStrategy(restartStrategyConfiguration: RestartStrategyConfiguration): Unit = { + javaEnv.setRestartStrategy(restartStrategyConfiguration) + } + + /** + * Returns the specified restart strategy configuration. + * + * @return The restart strategy configuration to be used + */ + @PublicEvolving + def getRestartStrategy: RestartStrategyConfiguration = { + javaEnv.getRestartStrategy() + } + + /** + * Sets the number of times that failed tasks are re-executed. A value of zero + * effectively disables fault tolerance. A value of "-1" indicates that the system + * default value (as defined in the configuration) should be used. + * + * @deprecated This method will be replaced by [[setRestartStrategy()]]. The + * FixedDelayRestartStrategyConfiguration contains the number of execution retries. + */ @PublicEvolving def setNumberOfExecutionRetries(numRetries: Int): Unit = { javaEnv.setNumberOfExecutionRetries(numRetries) } /** - * Gets the number of times the system will try to re-execute failed tasks. A value - * of "-1" indicates that the system default value (as defined in the configuration) - * should be used. - */ + * Gets the number of times the system will try to re-execute failed tasks. A value + * of "-1" indicates that the system default value (as defined in the configuration) + * should be used. + * + * @deprecated This method will be replaced by [[getRestartStrategy]]. The + * FixedDelayRestartStrategyConfiguration contains the number of execution retries. + */ @PublicEvolving def getNumberOfExecutionRetries = javaEnv.getNumberOfExecutionRetries diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java index 9eca07477719d..9d2f06225b58e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.checkpointing; import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple4; import org.apache.flink.configuration.ConfigConstants; @@ -67,7 +68,6 @@ public static void startTestCluster() { config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM / 2); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 48); - config.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "0 ms"); cluster = new ForkableFlinkMiniCluster(config, false); cluster.start(); @@ -96,7 +96,7 @@ public void testTumblingTimeWindow() { env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); env.getConfig().disableSysoutLogging(); env @@ -160,7 +160,7 @@ public void testSlidingTimeWindow() { env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); env.getConfig().disableSysoutLogging(); env @@ -221,7 +221,7 @@ public void testPreAggregatedTumblingTimeWindow() { env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); env.getConfig().disableSysoutLogging(); env @@ -289,7 +289,7 @@ public void testPreAggregatedSlidingTimeWindow() { env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); env.getConfig().disableSysoutLogging(); env diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java index d9ced4b5dba15..c6a6b28e0daed 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.checkpointing; import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.java.tuple.Tuple; @@ -70,7 +71,7 @@ import static org.junit.Assert.*; /** - * This verfies that checkpointing works correctly with event time windows. This is more + * This verifies that checkpointing works correctly with event time windows. This is more * strict than {@link WindowCheckpointingITCase} because for event-time the contents * of the emitted windows are deterministic. */ @@ -98,7 +99,6 @@ public static void startTestCluster() { config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM / 2); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 48); - config.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "0 ms"); cluster = new ForkableFlinkMiniCluster(config, false); cluster.start(); @@ -149,7 +149,7 @@ public void testTumblingTimeWindow() { env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); env.getConfig().disableSysoutLogging(); env.setStateBackend(this.stateBackend); @@ -213,7 +213,7 @@ public void testTumblingTimeWindowWithKVState() { env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); env.getConfig().disableSysoutLogging(); env.setStateBackend(this.stateBackend); @@ -282,7 +282,7 @@ public void testSlidingTimeWindow() { env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); env.getConfig().disableSysoutLogging(); env.setStateBackend(this.stateBackend); @@ -346,7 +346,7 @@ public void testPreAggregatedTumblingTimeWindow() { env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); env.getConfig().disableSysoutLogging(); env.setStateBackend(this.stateBackend); @@ -414,7 +414,7 @@ public void testPreAggregatedSlidingTimeWindow() { env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); env.getConfig().disableSysoutLogging(); env.setStateBackend(this.stateBackend); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index 58f8225b81028..b89f35a98b14a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -27,6 +27,7 @@ import org.apache.flink.api.common.ApplicationID; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; @@ -170,7 +171,7 @@ public void testTriggerSavepointAndResume() throws Exception { LOG.info("JobManager: " + jobManager + "."); // Submit the job - final JobGraph jobGraph = createJobGraph(parallelism, 0, 1000); + final JobGraph jobGraph = createJobGraph(parallelism, 0, 1000, 1000); final JobID jobId = jobGraph.getJobID(); // Wait for the source to be notified about the expected number @@ -463,7 +464,7 @@ public void testCheckpointHasBeenRemoved() throws Exception { LOG.info("JobManager: " + jobManager + "."); // Submit the job - final JobGraph jobGraph = createJobGraph(parallelism, 0, 1000); + final JobGraph jobGraph = createJobGraph(parallelism, 0, 1000, 1000); final JobID jobId = jobGraph.getJobID(); // Wait for the source to be notified about the expected number @@ -599,7 +600,7 @@ public void testCheckpointsRemovedWithJobManagerBackendOnShutdown() throws Excep LOG.info("JobManager: " + jobManager + "."); // Submit the job - final JobGraph jobGraph = createJobGraph(parallelism, 0, 1000); + final JobGraph jobGraph = createJobGraph(parallelism, 0, 1000, 1000); final JobID jobId = jobGraph.getJobID(); // Wait for the source to be notified about the expected number @@ -704,10 +705,6 @@ public void testSubmitWithUnknownSavepointPath() throws Exception { config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskManagers); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlotsPerTaskManager); - // Long delay to ensure that the test times out if the job - // manager tries to restart the job. - config.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "1 hour"); - LOG.info("Flink configuration: " + config + "."); // Start Flink @@ -725,7 +722,9 @@ public void testSubmitWithUnknownSavepointPath() throws Exception { // High value to ensure timeouts if restarted. int numberOfRetries = 1000; // Submit the job - final JobGraph jobGraph = createJobGraph(parallelism, numberOfRetries, 1000); + // Long delay to ensure that the test times out if the job + // manager tries to restart the job. + final JobGraph jobGraph = createJobGraph(parallelism, numberOfRetries, 3600000, 1000); // Set non-existing savepoint path jobGraph.setSavepointPath("unknown path"); @@ -758,13 +757,14 @@ public void testSubmitWithUnknownSavepointPath() throws Exception { private JobGraph createJobGraph( int parallelism, int numberOfRetries, + long restartDelay, int checkpointingInterval) { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(parallelism); - env.setNumberOfExecutionRetries(numberOfRetries); env.enableCheckpointing(checkpointingInterval); env.disableOperatorChaining(); + env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(numberOfRetries, restartDelay)); env.getConfig().disableSysoutLogging(); DataStream stream = env diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java index 5fa066686826b..8927f43e7e3eb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java @@ -84,7 +84,7 @@ public static void startCluster() { Configuration config = new Configuration(); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); - config.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "0 ms"); + config.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 ms"); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); cluster = new ForkableFlinkMiniCluster(config, false); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java index a1e1e40b6f1fe..0fffdb5bdfd07 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java @@ -49,7 +49,6 @@ public static void startCluster() { Configuration config = new Configuration(); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); - config.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "0 ms"); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); cluster = new ForkableFlinkMiniCluster(config, false); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java index c9286ce6a1acc..eb5ef5a03246b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.client.program.ProgramInvocationException; @@ -81,7 +82,6 @@ public static void startTestCluster() { config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM / 2); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 48); - config.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "0 ms"); cluster = new ForkableFlinkMiniCluster(config, false); cluster.start(); @@ -109,7 +109,7 @@ public void testTumblingProcessingTimeWindow() { env.setStreamTimeCharacteristic(timeCharacteristic); env.getConfig().setAutoWatermarkInterval(10); env.enableCheckpointing(100); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); env.getConfig().disableSysoutLogging(); env @@ -167,7 +167,7 @@ public void testSlidingProcessingTimeWindow() { env.setStreamTimeCharacteristic(timeCharacteristic); env.getConfig().setAutoWatermarkInterval(10); env.enableCheckpointing(100); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); env.getConfig().disableSysoutLogging(); env @@ -225,7 +225,7 @@ public void testAggregatingTumblingProcessingTimeWindow() { env.setStreamTimeCharacteristic(timeCharacteristic); env.getConfig().setAutoWatermarkInterval(10); env.enableCheckpointing(100); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); env.getConfig().disableSysoutLogging(); env @@ -273,7 +273,7 @@ public void testAggregatingSlidingProcessingTimeWindow() { env.setStreamTimeCharacteristic(timeCharacteristic); env.getConfig().setAutoWatermarkInterval(10); env.enableCheckpointing(100); - env.setNumberOfExecutionRetries(3); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0)); env.getConfig().disableSysoutLogging(); env diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java index 59868bf913c2a..4003de1d58e3b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java @@ -54,7 +54,6 @@ public void testJobsWithCustomClassLoader() { Configuration config = new Configuration(); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2); - config.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "0 s"); // we need to use the "filesystem" state backend to ensure FLINK-2543 is not happening again. config.setString(ConfigConstants.STATE_BACKEND, "filesystem"); diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java index bc7cebb4d07a4..f91582f3a51f8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java +++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java @@ -19,6 +19,7 @@ package org.apache.flink.test.classloading.jar; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.datastream.DataStream; @@ -45,7 +46,7 @@ public static void main(String[] args) throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(host, port, jarFile); env.getConfig().disableSysoutLogging(); env.enableCheckpointing(CHECKPOINT_INTERVALL); - env.setNumberOfExecutionRetries(1); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 10000)); env.disableOperatorChaining(); DataStream text = env.addSource(new SimpleStringGenerator()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java index 65036662b9abc..f6fb0ddd1feca 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java @@ -21,6 +21,7 @@ import java.util.List; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.test.util.JavaProgramTestBase; @@ -63,8 +64,7 @@ protected void testProgram() throws Exception { private void executeTask(MapFunction mapper, int retries) throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().setNumberOfExecutionRetries(retries); - env.getConfig().setExecutionRetryDelay(0); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(retries, 0)); List result = env.generateSequence(1, 9) .map(mapper) .collect(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java index 7b4c9b219f259..d3abf8b95c1eb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java @@ -122,7 +122,7 @@ public void testTaskManagerProcessFailure() { jmConfig.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, "1000 ms"); jmConfig.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, "6 s"); jmConfig.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 9); - jmConfig.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "10 s"); + jmConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "10 s"); jmConfig.setString(ConfigConstants.AKKA_ASK_TIMEOUT, "100 s"); jmActorSystem = AkkaUtils.createActorSystem(jmConfig, new Some>(localAddress)); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java index 2a139c71500b6..8a45d6259d37b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -29,6 +30,7 @@ import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.util.TestLogger; import org.junit.Test; import java.util.concurrent.atomic.AtomicInteger; @@ -36,7 +38,7 @@ import static org.junit.Assert.fail; @SuppressWarnings("serial") -public class FastFailuresITCase { +public class FastFailuresITCase extends TestLogger { static final AtomicInteger FAILURES_SO_FAR = new AtomicInteger(); static final int NUM_FAILURES = 200; @@ -54,9 +56,9 @@ public void testThis() { "localhost", cluster.getLeaderRPCPort()); env.getConfig().disableSysoutLogging(); - env.getConfig().setExecutionRetryDelay(0); env.setParallelism(4); env.enableCheckpointing(1000); + env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(200, 0)); DataStream> input = env.addSource(new RichSourceFunction>() { diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerProcessFailureBatchRecoveryITCase.java index 66565dd2c62cf..c64f8cefae5d0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerProcessFailureBatchRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerProcessFailureBatchRecoveryITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.DiscardingOutputFormat; @@ -75,7 +76,7 @@ public void testJobManagerFailure(String zkQuorum, final File coordinateDir) thr ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment( "leader", 1, config); env.setParallelism(PARALLELISM); - env.setNumberOfExecutionRetries(1); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000)); env.getConfig().setExecutionMode(executionMode); env.getConfig().disableSysoutLogging(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java index 2b49c088ae5b2..e07a1a4b2a9c9 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.client.program.ProgramInvocationException; @@ -134,7 +135,7 @@ public void run() { try { ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", jobManagerPort); env.setParallelism(2); - env.setNumberOfExecutionRetries(0); + env.setRestartStrategy(RestartStrategies.noRestart()); env.getConfig().disableSysoutLogging(); env.generateSequence(0, Long.MAX_VALUE) diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java index 2efb887c0d4fb..4437db1c522d2 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.LocalCollectionOutputFormat; import org.apache.flink.client.program.ProgramInvocationException; @@ -51,7 +52,9 @@ public static void setupCluster() { Configuration config = new Configuration(); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2); - config.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, "100 ms"); + config.setString(ConfigConstants.RESTART_STRATEGY, "fixed-delay"); + config.setInteger(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 1); + config.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "100 ms"); cluster = new ForkableFlinkMiniCluster(config, false); @@ -82,7 +85,7 @@ public void testFailedRunThenSuccessfulRun() { "localhost", cluster.getLeaderRPCPort()); env.setParallelism(4); - env.setNumberOfExecutionRetries(0); + env.setRestartStrategy(RestartStrategies.noRestart()); env.getConfig().disableSysoutLogging(); env.generateSequence(1, 10) @@ -112,7 +115,7 @@ public Long reduce(Long value1, Long value2) { "localhost", cluster.getLeaderRPCPort()); env.setParallelism(4); - env.setNumberOfExecutionRetries(0); + env.setRestartStrategy(RestartStrategies.noRestart()); env.getConfig().disableSysoutLogging(); env.generateSequence(1, 10) @@ -159,7 +162,7 @@ public void testRestart() { "localhost", cluster.getLeaderRPCPort()); env.setParallelism(4); - env.setNumberOfExecutionRetries(1); + // the default restart strategy should be taken env.getConfig().disableSysoutLogging(); env.generateSequence(1, 10) @@ -204,7 +207,7 @@ public void testRestartMultipleTimes() { "localhost", cluster.getLeaderRPCPort()); env.setParallelism(4); - env.setNumberOfExecutionRetries(5); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(5, 100)); env.getConfig().disableSysoutLogging(); env.generateSequence(1, 10) diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java index 520a0f2ab3aa6..6c621accd8d68 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java @@ -24,6 +24,7 @@ import akka.pattern.Patterns; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.LocalCollectionOutputFormat; import org.apache.flink.configuration.ConfigConstants; @@ -88,7 +89,7 @@ public void testRestartWithFailingTaskManager() { "localhost", cluster.getLeaderRPCPort()); env.setParallelism(PARALLELISM); - env.setNumberOfExecutionRetries(1); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000)); env.getConfig().disableSysoutLogging(); env.generateSequence(1, 10) diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java index 173c8ea0c7fd4..053722b7acbf0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionMode; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.junit.runner.RunWith; @@ -65,7 +66,7 @@ public void testTaskManagerFailure(int jobManagerPort, final File coordinateDir) ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", jobManagerPort); env.setParallelism(PARALLELISM); - env.setNumberOfExecutionRetries(1); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000)); env.getConfig().setExecutionMode(executionMode); env.getConfig().disableSysoutLogging(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java index 63aa96788554f..fcc3d42389592 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java @@ -26,6 +26,7 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.Checkpointed; @@ -69,7 +70,7 @@ public void testTaskManagerFailure(int jobManagerPort, final File coordinateDir) .createRemoteEnvironment("localhost", jobManagerPort); env.setParallelism(PARALLELISM); env.getConfig().disableSysoutLogging(); - env.setNumberOfExecutionRetries(1); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000)); env.enableCheckpointing(200); env.setStateBackend(new FsStateBackend(tempCheckpointDir.getAbsoluteFile().toURI())); diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java index 68b1d5aaba57c..45ee839bc1b97 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java @@ -148,7 +148,7 @@ public void testJobExecutionOnClusterWithLeaderReelection() throws Exception { // we "effectively" disable the automatic RecoverAllJobs message and sent it manually to make // sure that all TMs have registered to the JM prior to issueing the RecoverAllJobs message - configuration.setString(ConfigConstants.EXECUTION_RETRY_DELAY_KEY, AkkaUtils.INF_TIMEOUT().toString()); + configuration.setString(ConfigConstants.AKKA_ASK_TIMEOUT, AkkaUtils.INF_TIMEOUT().toString()); Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(true); diff --git a/flink-yarn-tests/src/main/scala/org/apache/flink/yarn/TestingYarnJobManager.scala b/flink-yarn-tests/src/main/scala/org/apache/flink/yarn/TestingYarnJobManager.scala index c7cd205f9b6b7..4e6b745e31c88 100644 --- a/flink-yarn-tests/src/main/scala/org/apache/flink/yarn/TestingYarnJobManager.scala +++ b/flink-yarn-tests/src/main/scala/org/apache/flink/yarn/TestingYarnJobManager.scala @@ -24,6 +24,7 @@ import akka.actor.ActorRef import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.checkpoint.{SavepointStore, CheckpointRecoveryFactory} import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager +import org.apache.flink.runtime.executiongraph.restart.RestartStrategy import org.apache.flink.runtime.instance.InstanceManager import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore import org.apache.flink.runtime.jobmanager.scheduler.Scheduler @@ -45,8 +46,7 @@ import scala.concurrent.duration.FiniteDuration * @param scheduler Scheduler to schedule Flink jobs * @param libraryCacheManager Manager to manage uploaded jar files * @param archive Archive for finished Flink jobs - * @param defaultExecutionRetries Number of default execution retries - * @param delayBetweenRetries Delay between retries + * @param restartStrategy Default restart strategy for job restarts * @param timeout Timeout for futures * @param leaderElectionService LeaderElectionService to participate in the leader election */ @@ -57,13 +57,13 @@ class TestingYarnJobManager( scheduler: Scheduler, libraryCacheManager: BlobLibraryCacheManager, archive: ActorRef, - defaultExecutionRetries: Int, - delayBetweenRetries: Long, + restartStrategy: RestartStrategy, timeout: FiniteDuration, leaderElectionService: LeaderElectionService, submittedJobGraphs : SubmittedJobGraphStore, checkpointRecoveryFactory : CheckpointRecoveryFactory, - savepointStore: SavepointStore) + savepointStore: SavepointStore, + jobRecoveryTimeout: FiniteDuration) extends YarnJobManager( flinkConfiguration, executorService, @@ -71,13 +71,13 @@ class TestingYarnJobManager( scheduler, libraryCacheManager, archive, - defaultExecutionRetries, - delayBetweenRetries, + restartStrategy, timeout, leaderElectionService, submittedJobGraphs, checkpointRecoveryFactory, - savepointStore) + savepointStore, + jobRecoveryTimeout) with TestingJobManagerLike { override val taskManagerRunnerClass = classOf[TestingYarnTaskManagerRunner] diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala index ec1fb81f11773..266345154e474 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala @@ -33,6 +33,7 @@ import org.apache.flink.api.common.JobID import org.apache.flink.configuration.{Configuration => FlinkConfiguration, ConfigConstants} import org.apache.flink.runtime.akka.AkkaUtils import org.apache.flink.runtime.checkpoint.{SavepointStore, CheckpointRecoveryFactory} +import org.apache.flink.runtime.executiongraph.restart.RestartStrategy import org.apache.flink.runtime.jobgraph.JobStatus import org.apache.flink.runtime.jobmanager.{SubmittedJobGraphStore, JobManager} import org.apache.flink.runtime.leaderelection.LeaderElectionService @@ -74,8 +75,7 @@ import scala.util.Try * @param scheduler Scheduler to schedule Flink jobs * @param libraryCacheManager Manager to manage uploaded jar files * @param archive Archive for finished Flink jobs - * @param defaultExecutionRetries Number of default execution retries - * @param delayBetweenRetries Delay between retries + * @param restartStrategy Restart strategy to be used in case of a job recovery * @param timeout Timeout for futures * @param leaderElectionService LeaderElectionService to participate in the leader election */ @@ -86,13 +86,13 @@ class YarnJobManager( scheduler: FlinkScheduler, libraryCacheManager: BlobLibraryCacheManager, archive: ActorRef, - defaultExecutionRetries: Int, - delayBetweenRetries: Long, + restartStrategy: RestartStrategy, timeout: FiniteDuration, leaderElectionService: LeaderElectionService, submittedJobGraphs : SubmittedJobGraphStore, checkpointRecoveryFactory : CheckpointRecoveryFactory, - savepointStore: SavepointStore) + savepointStore: SavepointStore, + jobRecoveryTimeout: FiniteDuration) extends JobManager( flinkConfiguration, executorService, @@ -100,13 +100,13 @@ class YarnJobManager( scheduler, libraryCacheManager, archive, - defaultExecutionRetries, - delayBetweenRetries, + restartStrategy, timeout, leaderElectionService, submittedJobGraphs, checkpointRecoveryFactory, - savepointStore) { + savepointStore, + jobRecoveryTimeout) { import context._ import scala.collection.JavaConverters._