[ https://issues.apache.org/jira/browse/FLINK-7216?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16094670#comment-16094670 ]
ASF GitHub Bot commented on FLINK-7216: --------------------------------------- Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/4364#discussion_r128512192 --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/restart/FailureRateRestartStrategyTest.java --- @@ -0,0 +1,128 @@ +/* + * 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.time.Time; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; +import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; + +import org.junit.After; +import org.junit.Test; + +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Unit test for the {@link FailureRateRestartStrategy}. + */ +public class FailureRateRestartStrategyTest { + + public final ScheduledExecutorService executorService = Executors.newScheduledThreadPool(4); + + public final ScheduledExecutor executor = new ScheduledExecutorServiceAdapter(executorService); + + @After + public void shutdownExecutor() { + executorService.shutdownNow(); + } + + // ------------------------------------------------------------------------ + + @Test + public void testManyFailuresWithinRate() throws Exception { + final int numAttempts = 10; + final int intervalMillis = 1; + + final FailureRateRestartStrategy restartStrategy = + new FailureRateRestartStrategy(1, Time.milliseconds(intervalMillis), Time.milliseconds(0)); + + for (int attempsLeft = numAttempts; attempsLeft > 0; --attempsLeft) { + assertTrue(restartStrategy.canRestart()); + restartStrategy.restart(new NoOpRestarter(), executor); + sleepGuaranteed(2 * intervalMillis); + } + + assertTrue(restartStrategy.canRestart()); + } + + @Test + public void testFailuresExceedingRate() throws Exception { + final int numFailures = 3; + final int intervalMillis = 10_000; + + final FailureRateRestartStrategy restartStrategy = + new FailureRateRestartStrategy(numFailures, Time.milliseconds(intervalMillis), Time.milliseconds(0)); + + for (int failuresLeft = numFailures; failuresLeft > 0; --failuresLeft) { + assertTrue(restartStrategy.canRestart()); + restartStrategy.restart(new NoOpRestarter(), executor); + } + + // now the rate should be exceeded + assertFalse(restartStrategy.canRestart()); + } + + @Test + public void testDelay() throws Exception { + final long restartDelay = 2; + final int numberRestarts = 10; + + final FailureRateRestartStrategy strategy = + new FailureRateRestartStrategy(numberRestarts + 1, Time.milliseconds(1), Time.milliseconds(restartDelay)); + + for (int restartsLeft = numberRestarts; restartsLeft > 0; --restartsLeft) { + assertTrue(strategy.canRestart()); + + final OneShotLatch sync = new OneShotLatch(); + final RestartCallback restarter = new LatchedRestarter(sync); + + final long time = System.nanoTime(); + strategy.restart(restarter, executor); + sync.await(); + + final long elapsed = System.nanoTime() - time; + assertTrue("Not enough delay", elapsed >= restartDelay * 1_000_000); + } + } + + // ------------------------------------------------------------------------ + + /** + * This method makes sure that the actual interval and is not spuriously waking up. --- End diff -- Then the whole method and test anyways aborts exceptionally. > ExecutionGraph can perform concurrent global restarts to scheduling > ------------------------------------------------------------------- > > Key: FLINK-7216 > URL: https://issues.apache.org/jira/browse/FLINK-7216 > Project: Flink > Issue Type: Bug > Components: Distributed Coordination > Affects Versions: 1.2.1, 1.3.1 > Reporter: Stephan Ewen > Assignee: Stephan Ewen > Priority: Blocker > Fix For: 1.4.0, 1.3.2 > > > Because ExecutionGraph restarts happen asynchronously and possibly delayed, > it can happen in rare corner cases that two restarts are attempted > concurrently, in which case some structures on the Execution Graph undergo a > concurrent access: > Sample stack trace: > {code} > WARN org.apache.flink.runtime.executiongraph.ExecutionGraph - Failed > to restart the job. > java.lang.IllegalStateException: SlotSharingGroup cannot clear task > assignment, group still has allocated resources. > at > org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup.clearTaskAssignment(SlotSharingGroup.java:78) > at > org.apache.flink.runtime.executiongraph.ExecutionJobVertex.resetForNewExecution(ExecutionJobVertex.java:535) > at > org.apache.flink.runtime.executiongraph.ExecutionGraph.restart(ExecutionGraph.java:1151) > at > org.apache.flink.runtime.executiongraph.restart.ExecutionGraphRestarter$1.call(ExecutionGraphRestarter.java:40) > at akka.dispatch.Futures$$anonfun$future$1.apply(Future.scala:95) > at > scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24) > at > scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:748) > {code} > The solution is to strictly guard against "subsumed" restarts via the > {{globalModVersion}} in a similar way as we fence local restarts against > global restarts. -- This message was sent by Atlassian JIRA (v6.4.14#64029)