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

Anton Kalashnikov commented on FLINK-25427:
-------------------------------------------

[~trohrmann], yes, I have a little progress. I successfully isolated 
`testTriggerSavepointAndResumeWithNoClaim` and right now, I at least have the 
explicit exception:
{noformat}
06:54:23,926 [flink-akka.actor.default-dispatcher-17] WARN  
org.apache.flink.runtime.resourcemanager.slotmanager.DeclarativeSlotManager [] 
- Could not fulfill resource requirements of job 
0c6696a94a9c5efb3fa74a58f23cbb08. Free slots: 0
06:54:24,627 [flink-akka.actor.default-dispatcher-18] INFO  
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler [] - Failed to go 
from CreatingExecutionGraph to Executing because the ExecutionGraph cre
ation failed.
org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Not 
enough resources available for scheduling.
        at 
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.lambda$determineParallelism$21(AdaptiveScheduler.java:743)
 ~[flink-runtime-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
        at java.util.Optional.orElseThrow(Optional.java:290) ~[?:1.8.0_292]
        at 
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.determineParallelism(AdaptiveScheduler.java:741)
 ~[flink-runtime-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.createExecutionGraphWithAvailableResourcesAsync(AdaptiveScheduler.java:915)
 ~[flink-runtime-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.goToCreatingExecutionGraph(AdaptiveScheduler.java:902)
 ~[flink-runtime-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.WaitingForResources.createExecutionGraphWithAvailableResources(WaitingForResources.java:178)
 ~[flink-runtime-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.WaitingForResources.resourceTimeout(WaitingForResources.java:174)
 ~[flink-runtime-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.runIfState(AdaptiveScheduler.java:1106)
 ~[flink-runtime-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.lambda$runIfState$26(AdaptiveScheduler.java:1121)
 ~[flink-runtime-1.15-SNAPSHOT.jar:1.15-SNAPSHOT]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[?:1.8.0_292]
        at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
~[?:1.8.0_292]
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$handleRunAsync$4(AkkaRpcActor.java:455)
 ~[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at 
org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:68)
 ~[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:455)
 ~[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:213)
 ~[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at 
org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:78)
 ~[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:163)
 ~[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:24) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:20) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at scala.PartialFunction.applyOrElse(PartialFunction.scala:123) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at scala.PartialFunction.applyOrElse$(PartialFunction.scala:122) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:20) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at akka.actor.Actor.aroundReceive(Actor.scala:537) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at akka.actor.Actor.aroundReceive$(Actor.scala:535) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:220) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:580) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at akka.actor.ActorCell.invoke(ActorCell.scala:548) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:270) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at akka.dispatch.Mailbox.run(Mailbox.scala:231) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at akka.dispatch.Mailbox.exec(Mailbox.scala:243) 
[flink-rpc-akka_0433dc93-b31b-44d6-8efc-777bd9bd7aa2.jar:1.15-SNAPSHOT]
        at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) 
[?:1.8.0_292]
        at 
java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) 
[?:1.8.0_292]
        at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) 
[?:1.8.0_292]
        at 
java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175) 
[?:1.8.0_292]
{noformat}

It happens when we try to submit the second job to the same minicluster after 
the canceling the first one. As you can see `Free slots: 0`. So right now, I am 
trying to figure out why it happens.

> SavepointITCase.testTriggerSavepointAndResumeWithNoClaim fails on AZP
> ---------------------------------------------------------------------
>
>                 Key: FLINK-25427
>                 URL: https://issues.apache.org/jira/browse/FLINK-25427
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing
>    Affects Versions: 1.15.0
>            Reporter: Till Rohrmann
>            Assignee: Anton Kalashnikov
>            Priority: Blocker
>              Labels: pull-request-available, test-stability
>             Fix For: 1.15.0
>
>
> The test {{SavepointITCase.testTriggerSavepointAndResumeWithNoClaim}} fails 
> on AZP with
> {code}
> 2021-12-23T03:10:26.4240179Z Dec 23 03:10:26 [ERROR] 
> org.apache.flink.test.checkpointing.SavepointITCase.testTriggerSavepointAndResumeWithNoClaim
>   Time elapsed: 62.289 s  <<< ERROR!
> 2021-12-23T03:10:26.4240998Z Dec 23 03:10:26 
> java.util.concurrent.TimeoutException: Condition was not met in given timeout.
> 2021-12-23T03:10:26.4241716Z Dec 23 03:10:26  at 
> org.apache.flink.runtime.testutils.CommonTestUtils.waitUntilCondition(CommonTestUtils.java:166)
> 2021-12-23T03:10:26.4242643Z Dec 23 03:10:26  at 
> org.apache.flink.runtime.testutils.CommonTestUtils.waitUntilCondition(CommonTestUtils.java:144)
> 2021-12-23T03:10:26.4243295Z Dec 23 03:10:26  at 
> org.apache.flink.runtime.testutils.CommonTestUtils.waitUntilCondition(CommonTestUtils.java:136)
> 2021-12-23T03:10:26.4244433Z Dec 23 03:10:26  at 
> org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning(CommonTestUtils.java:210)
> 2021-12-23T03:10:26.4245166Z Dec 23 03:10:26  at 
> org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning(CommonTestUtils.java:184)
> 2021-12-23T03:10:26.4245830Z Dec 23 03:10:26  at 
> org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning(CommonTestUtils.java:172)
> 2021-12-23T03:10:26.4246870Z Dec 23 03:10:26  at 
> org.apache.flink.test.checkpointing.SavepointITCase.testTriggerSavepointAndResumeWithNoClaim(SavepointITCase.java:446)
> 2021-12-23T03:10:26.4247813Z Dec 23 03:10:26  at 
> sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 2021-12-23T03:10:26.4248808Z Dec 23 03:10:26  at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> 2021-12-23T03:10:26.4249426Z Dec 23 03:10:26  at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 2021-12-23T03:10:26.4250192Z Dec 23 03:10:26  at 
> java.lang.reflect.Method.invoke(Method.java:498)
> 2021-12-23T03:10:26.4251196Z Dec 23 03:10:26  at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
> 2021-12-23T03:10:26.4252160Z Dec 23 03:10:26  at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
> 2021-12-23T03:10:26.4252888Z Dec 23 03:10:26  at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
> 2021-12-23T03:10:26.4253547Z Dec 23 03:10:26  at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
> 2021-12-23T03:10:26.4254142Z Dec 23 03:10:26  at 
> org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
> 2021-12-23T03:10:26.4254932Z Dec 23 03:10:26  at 
> org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
> 2021-12-23T03:10:26.4255513Z Dec 23 03:10:26  at 
> org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
> 2021-12-23T03:10:26.4256091Z Dec 23 03:10:26  at 
> org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45)
> 2021-12-23T03:10:26.4256636Z Dec 23 03:10:26  at 
> org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:61)
> 2021-12-23T03:10:26.4257165Z Dec 23 03:10:26  at 
> org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
> 2021-12-23T03:10:26.4257744Z Dec 23 03:10:26  at 
> org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
> 2021-12-23T03:10:26.4258312Z Dec 23 03:10:26  at 
> org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
> 2021-12-23T03:10:26.4258884Z Dec 23 03:10:26  at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
> 2021-12-23T03:10:26.4259488Z Dec 23 03:10:26  at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
> 2021-12-23T03:10:26.4260049Z Dec 23 03:10:26  at 
> org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
> 2021-12-23T03:10:26.4260579Z Dec 23 03:10:26  at 
> org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
> 2021-12-23T03:10:26.4261108Z Dec 23 03:10:26  at 
> org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
> 2021-12-23T03:10:26.4261648Z Dec 23 03:10:26  at 
> org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
> 2021-12-23T03:10:26.4262183Z Dec 23 03:10:26  at 
> org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
> 2021-12-23T03:10:26.4262794Z Dec 23 03:10:26  at 
> org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
> 2021-12-23T03:10:26.4263312Z Dec 23 03:10:26  at 
> org.junit.runners.ParentRunner.run(ParentRunner.java:413)
> 2021-12-23T03:10:26.4263813Z Dec 23 03:10:26  at 
> org.junit.runner.JUnitCore.run(JUnitCore.java:137)
> 2021-12-23T03:10:26.4264377Z Dec 23 03:10:26  at 
> org.junit.runner.JUnitCore.run(JUnitCore.java:115)
> 2021-12-23T03:10:26.4264909Z Dec 23 03:10:26  at 
> org.junit.vintage.engine.execution.RunnerExecutor.execute(RunnerExecutor.java:42)
> 2021-12-23T03:10:26.4265529Z Dec 23 03:10:26  at 
> org.junit.vintage.engine.VintageTestEngine.executeAllChildren(VintageTestEngine.java:80)
> 2021-12-23T03:10:26.4266171Z Dec 23 03:10:26  at 
> org.junit.vintage.engine.VintageTestEngine.execute(VintageTestEngine.java:72)
> 2021-12-23T03:10:26.4266815Z Dec 23 03:10:26  at 
> org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:107)
> 2021-12-23T03:10:26.4267505Z Dec 23 03:10:26  at 
> org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:88)
> 2021-12-23T03:10:26.4268212Z Dec 23 03:10:26  at 
> org.junit.platform.launcher.core.EngineExecutionOrchestrator.lambda$execute$0(EngineExecutionOrchestrator.java:54)
> 2021-12-23T03:10:26.4268957Z Dec 23 03:10:26  at 
> org.junit.platform.launcher.core.EngineExecutionOrchestrator.withInterceptedStreams(EngineExecutionOrchestrator.java:67)
> 2021-12-23T03:10:26.4269809Z Dec 23 03:10:26  at 
> org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:52)
> 2021-12-23T03:10:26.4270462Z Dec 23 03:10:26  at 
> org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:114)
> 2021-12-23T03:10:26.4271133Z Dec 23 03:10:26  at 
> org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:86)
> 2021-12-23T03:10:26.4271799Z Dec 23 03:10:26  at 
> org.junit.platform.launcher.core.DefaultLauncherSession$DelegatingLauncher.execute(DefaultLauncherSession.java:86)
> 2021-12-23T03:10:26.4272578Z Dec 23 03:10:26  at 
> org.junit.platform.launcher.core.SessionPerRequestLauncher.execute(SessionPerRequestLauncher.java:53)
> 2021-12-23T03:10:26.4273257Z Dec 23 03:10:26  at 
> org.apache.maven.surefire.junitplatform.JUnitPlatformProvider.execute(JUnitPlatformProvider.java:188)
> 2021-12-23T03:10:26.4273953Z Dec 23 03:10:26  at 
> org.apache.maven.surefire.junitplatform.JUnitPlatformProvider.invokeAllTests(JUnitPlatformProvider.java:154)
> 2021-12-23T03:10:26.4274712Z Dec 23 03:10:26  at 
> org.apache.maven.surefire.junitplatform.JUnitPlatformProvider.invoke(JUnitPlatformProvider.java:124)
> 2021-12-23T03:10:26.4275360Z Dec 23 03:10:26  at 
> org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:428)
> 2021-12-23T03:10:26.4275973Z Dec 23 03:10:26  at 
> org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:162)
> 2021-12-23T03:10:26.4276552Z Dec 23 03:10:26  at 
> org.apache.maven.surefire.booter.ForkedBooter.run(ForkedBooter.java:562)
> 2021-12-23T03:10:26.4277106Z Dec 23 03:10:26  at 
> org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:548)
> {code}
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=28502&view=logs&j=8fd9202e-fd17-5b26-353c-ac1ff76c8f28&t=ea7cf968-e585-52cb-e0fc-f48de023a7ca&l=9810



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to