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

Dawid Wysakowicz commented on FLINK-22266:
------------------------------------------

There might actually be a problem in the AdaptiveScheduler. After investigating 
some logs, I find the following stacktrace:

{code}
21:54:12,518 [flink-akka.actor.default-dispatcher-4] WARN  
org.apache.flink.runtime.minicluster.MiniCluster             [] - Error in 
MiniCluster. Shutting the MiniCluster down.
java.lang.IllegalStateException: The vertex must be in CREATED or SCHEDULED 
state to be deployed. Found state RUNNING
        at 
org.apache.flink.runtime.executiongraph.Execution.deploy(Execution.java:546) 
~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.executiongraph.ExecutionVertex.deploy(ExecutionVertex.java:427)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.Executing.deploySafely(Executing.java:139)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.Executing.deploy(Executing.java:132)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.Executing.<init>(Executing.java:63) 
~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.Executing$Factory.getState(Executing.java:363)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.Executing$Factory.getState(Executing.java:334)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.transitionToState(AdaptiveScheduler.java:1139)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.goToExecuting(AdaptiveScheduler.java:787)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.StopWithSavepoint.handleSavepointCompletion(StopWithSavepoint.java:106)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.StopWithSavepoint.lambda$null$0(StopWithSavepoint.java:89)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.runIfState(AdaptiveScheduler.java:1093)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler.lambda$runIfState$26(AdaptiveScheduler.java:1108)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[?:1.8.0_282]
        at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
~[?:1.8.0_282]
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:440)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:208)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:77)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:158)
 ~[flink-runtime_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT]
        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123) 
[scala-library-2.11.12.jar:?]
        at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170) 
[scala-library-2.11.12.jar:?]
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) 
[scala-library-2.11.12.jar:?]
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) 
[scala-library-2.11.12.jar:?]
        at akka.actor.Actor$class.aroundReceive(Actor.scala:517) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.actor.ActorCell.invoke(ActorCell.scala:561) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.dispatch.Mailbox.run(Mailbox.scala:225) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.dispatch.Mailbox.exec(Mailbox.scala:235) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at 
akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at 
akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
        at 
akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) 
[akka-actor_2.11-2.5.21.jar:2.5.21]
{code}

After the exception, the MiniCluster is being shut down, leading to random 
exceptions from other tests.

> Harden JobMasterStopWithSavepointITCase
> ---------------------------------------
>
>                 Key: FLINK-22266
>                 URL: https://issues.apache.org/jira/browse/FLINK-22266
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing, Runtime / Coordination
>    Affects Versions: 1.13.0, 1.14.0
>            Reporter: Guowei Ma
>            Assignee: Dawid Wysakowicz
>            Priority: Critical
>              Labels: test-stability
>             Fix For: 1.14.0
>
>
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=16451&view=logs&j=5c8e7682-d68f-54d1-16a2-a09310218a49&t=f508e270-48d6-5f1e-3138-42a17e0714f0&l=3884
> {code:java}
> [ERROR] 
> throwingExceptionOnCallbackWithNoRestartsShouldFailTheTerminate(org.apache.flink.runtime.jobmaster.JobMasterStopWithSavepointITCase)
>   Time elapsed: 0.154 s  <<< FAILURE!
> java.lang.AssertionError
>       at org.junit.Assert.fail(Assert.java:86)
>       at org.junit.Assert.assertTrue(Assert.java:41)
>       at org.junit.Assert.assertTrue(Assert.java:52)
>       at 
> org.apache.flink.runtime.jobmaster.JobMasterStopWithSavepointITCase.throwingExceptionOnCallbackWithoutRestartsHelper(JobMasterStopWithSavepointITCase.java:154)
>       at 
> org.apache.flink.runtime.jobmaster.JobMasterStopWithSavepointITCase.throwingExceptionOnCallbackWithNoRestartsShouldFailTheTerminate(JobMasterStopWithSavepointITCase.java:138)
>       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.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
>       at 
> org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at java.lang.Thread.run(Thread.java:748)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to