[
https://issues.apache.org/jira/browse/FLINK-22345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17330164#comment-17330164
]
Dawid Wysakowicz commented on FLINK-22345:
------------------------------------------
The problem is in the test. The {{EventSendingCoordinator}} uses a field
{{failedBefore}} to check if it should throw an exception or not. This works
under the assumption that an instance of the coordinator is created only once
and survives restarts. That is not the case for the {{AdaptiveScheduler}},
which recreates the whole {{ExecutionGraph}} and creates a new coordinator with
the {{failedBefore}} flag reset. Therefore we fail indefinitely.
> CoordinatorEventsExactlyOnceITCase hangs on azure
> -------------------------------------------------
>
> Key: FLINK-22345
> URL: https://issues.apache.org/jira/browse/FLINK-22345
> Project: Flink
> Issue Type: Bug
> Components: API / DataStream
> Affects Versions: 1.13.0, 1.14.0
> Reporter: Dawid Wysakowicz
> Assignee: Stephan Ewen
> Priority: Blocker
> Labels: test-stability
> Fix For: 1.13.0, 1.12.3
>
> Attachments: screenshot-1.png
>
>
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=16731&view=logs&j=02c4e775-43bf-5625-d1cc-542b5209e072&t=e5961b24-88d9-5c77-efd3-955422674c25&l=9896
> {code}
> "main" #1 prio=5 os_prio=0 tid=0x00007fa8c800b800 nid=0x58b3 waiting on
> condition [0x00007fa8cfd1c000]
> java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for <0x000000008147a7e8> (a
> java.util.concurrent.CompletableFuture$Signaller)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> at
> java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1707)
> at
> java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
> at
> java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1742)
> at
> java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
> at
> org.apache.flink.runtime.minicluster.MiniCluster.executeJobBlocking(MiniCluster.java:802)
> at
> org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.test(CoordinatorEventsExactlyOnceITCase.java:187)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
> at
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
> at
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
> at
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
> at
> org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45)
> at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
> at org.junit.rules.RunRules.evaluate(RunRules.java:20)
> at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
> at
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
> at
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
> at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
> at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
> at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
> at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
> at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
> at
> org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
> at
> org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
> at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
> at
> org.apache.maven.surefire.junit4.JUnit4Provider.execute(JUnit4Provider.java:365)
> at
> org.apache.maven.surefire.junit4.JUnit4Provider.executeWithRerun(JUnit4Provider.java:273)
> at
> org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:238)
> at
> org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:159)
> at
> org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:384)
> at
> org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:345)
> at
> org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:126)
> at
> org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:418)
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)