You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2016/10/26 22:46:53 UTC
flink git commit: [FLINK-4283] Use new InfiniteDelayRestartStrategy
instead of FixedDelayRestartStrategy to avoid blocking threads
Repository: flink
Updated Branches:
refs/heads/master fa664e5b9 -> b5938b052
[FLINK-4283] Use new InfiniteDelayRestartStrategy instead of FixedDelayRestartStrategy to avoid blocking threads
This closes #2661.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b5938b05
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b5938b05
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b5938b05
Branch: refs/heads/master
Commit: b5938b052dfd1b4e9450c430446062852bca8850
Parents: fa664e5
Author: Alexander Shoshin <Al...@epam.com>
Authored: Tue Oct 18 13:21:51 2016 +0300
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Oct 27 00:46:36 2016 +0200
----------------------------------------------------------------------
.../ExecutionGraphRestartTest.java | 11 +++---
.../restart/InfiniteDelayRestartStrategy.java | 41 ++++++++++++++++++++
2 files changed, 47 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/b5938b05/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java
----------------------------------------------------------------------
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 0d09e38..3743adb 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
@@ -32,6 +32,7 @@ import org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy
import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy;
import org.apache.flink.runtime.executiongraph.restart.FailureRateRestartStrategy;
import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
+import org.apache.flink.runtime.executiongraph.restart.InfiniteDelayRestartStrategy;
import org.apache.flink.runtime.instance.Instance;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobgraph.JobStatus;
@@ -186,7 +187,7 @@ public class ExecutionGraphRestartTest extends TestLogger {
@Test
public void testCancelWhileRestarting() throws Exception {
// We want to manually control the restart and delay
- FixedDelayRestartStrategy restartStrategy = new FixedDelayRestartStrategy(Integer.MAX_VALUE, Long.MAX_VALUE);
+ RestartStrategy restartStrategy = new InfiniteDelayRestartStrategy();
Tuple2<ExecutionGraph, Instance> executionGraphInstanceTuple = createExecutionGraph(restartStrategy);
ExecutionGraph executionGraph = executionGraphInstanceTuple.f0;
Instance instance = executionGraphInstanceTuple.f1;
@@ -234,7 +235,7 @@ public class ExecutionGraphRestartTest extends TestLogger {
new SerializedValue<>(new ExecutionConfig()),
AkkaUtils.getDefaultTimeout(),
// We want to manually control the restart and delay
- new FixedDelayRestartStrategy(Integer.MAX_VALUE, Long.MAX_VALUE));
+ new InfiniteDelayRestartStrategy());
JobVertex jobVertex = new JobVertex("NoOpInvokable");
jobVertex.setInvokableClass(Tasks.NoOpInvokable.class);
@@ -277,7 +278,7 @@ public class ExecutionGraphRestartTest extends TestLogger {
@Test
public void testCancelWhileFailing() throws Exception {
// We want to manually control the restart and delay
- FixedDelayRestartStrategy restartStrategy = new FixedDelayRestartStrategy(Integer.MAX_VALUE, Long.MAX_VALUE);
+ RestartStrategy restartStrategy = new InfiniteDelayRestartStrategy();
Tuple2<ExecutionGraph, Instance> executionGraphInstanceTuple = createSpyExecutionGraph(restartStrategy);
ExecutionGraph executionGraph = executionGraphInstanceTuple.f0;
Instance instance = executionGraphInstanceTuple.f1;
@@ -440,7 +441,7 @@ public class ExecutionGraphRestartTest extends TestLogger {
JobGraph jobGraph = new JobGraph("Test Job", vertex);
jobGraph.setExecutionConfig(executionConfig);
- ExecutionGraph eg = newExecutionGraph(new FixedDelayRestartStrategy(1, 1000000));
+ ExecutionGraph eg = newExecutionGraph(new InfiniteDelayRestartStrategy());
eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources());
@@ -485,7 +486,7 @@ public class ExecutionGraphRestartTest extends TestLogger {
JobGraph jobGraph = new JobGraph("Test Job", vertex);
jobGraph.setExecutionConfig(executionConfig);
- ExecutionGraph eg = newExecutionGraph(new FixedDelayRestartStrategy(1, 1000000));
+ ExecutionGraph eg = newExecutionGraph(new InfiniteDelayRestartStrategy());
eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources());
http://git-wip-us.apache.org/repos/asf/flink/blob/b5938b05/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/restart/InfiniteDelayRestartStrategy.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/restart/InfiniteDelayRestartStrategy.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/restart/InfiniteDelayRestartStrategy.java
new file mode 100644
index 0000000..4be0b96
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/restart/InfiniteDelayRestartStrategy.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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Testing restart strategy which promise to restart {@link ExecutionGraph} after the infinite time delay.
+ * Actually {@link ExecutionGraph} will never be restarted. No additional threads will be used.
+ */
+public class InfiniteDelayRestartStrategy implements RestartStrategy {
+ private static final Logger LOG = LoggerFactory.getLogger(InfiniteDelayRestartStrategy.class);
+
+ @Override
+ public boolean canRestart() {
+ return true;
+ }
+
+ @Override
+ public void restart(ExecutionGraph executionGraph) {
+ LOG.info("Delaying retry of job execution forever");
+ }
+}