You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Stephan Ewen (JIRA)" <ji...@apache.org> on 2016/01/20 17:32:39 UTC

[jira] [Commented] (FLINK-3268) Unstable test JobManagerSubmittedJobGraphsRecoveryITCase

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

Stephan Ewen commented on FLINK-3268:
-------------------------------------

The actual test failure is that the cluster does not properly shut down. Shutdown is triggered while one blocking job is still running.

The log of the shutdown period is below:
{code}
15:28:35,960 INFO  org.apache.flink.runtime.jobmanager.JobManagerSubmittedJobGraphsRecoveryITCase  - Wait that the non-leader removes the submitted job.
15:28:35,961 INFO  org.apache.flink.runtime.testingUtils.TestingTaskManager      - Stopping TaskManager akka://flink/user/taskmanager_1#-949637839.
15:28:35,961 INFO  org.apache.flink.runtime.testingUtils.TestingJobManager       - Stopping JobManager akka.tcp://flink@127.0.0.1:54921/user/jobmanager.
15:28:35,961 INFO  org.apache.flink.runtime.testingUtils.TestingTaskManager      - Cancelling all computations and discarding all cached data.
15:28:35,962 INFO  org.apache.flink.runtime.taskmanager.Task                     - Attempting to fail task externally Blocking Vertex (1/1)
15:28:35,962 INFO  org.apache.flink.runtime.taskmanager.Task                     - Blocking Vertex (1/1) switched to FAILED with exception.
java.lang.Exception: TaskManager is shutting down.
	at org.apache.flink.runtime.taskmanager.TaskManager.postStop(TaskManager.scala:216)
	at akka.actor.Actor$class.aroundPostStop(Actor.scala:475)
	at org.apache.flink.runtime.taskmanager.TaskManager.aroundPostStop(TaskManager.scala:119)
	at akka.actor.dungeon.FaultHandling$class.akka$actor$dungeon$FaultHandling$$finishTerminate(FaultHandling.scala:210)
	at akka.actor.dungeon.FaultHandling$class.terminate(FaultHandling.scala:172)
	at akka.actor.ActorCell.terminate(ActorCell.scala:369)
	at akka.actor.ActorCell.invokeAll$1(ActorCell.scala:462)
	at akka.actor.ActorCell.systemInvoke(ActorCell.scala:478)
	at akka.dispatch.Mailbox.processAllSystemMessages(Mailbox.scala:279)
	at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)
	at akka.dispatch.Mailbox.run(Mailbox.scala:221)
	at akka.dispatch.Mailbox.exec(Mailbox.scala:231)
	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
15:28:35,963 INFO  org.apache.flink.runtime.testingUtils.TestingJobManager       - Stopping JobManager akka.tcp://flink@127.0.0.1:54922/user/jobmanager.
15:28:35,964 INFO  org.apache.flink.runtime.taskmanager.Task                     - Triggering cancellation of task code Blocking Vertex (1/1) (7e54466b8bfc1f8063e7a145126688ae).
15:28:35,965 INFO  org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  - Stopping ZooKeeperLeaderElectionService.
15:28:35,965 INFO  org.apache.flink.runtime.testingUtils.TestingTaskManager      - Disassociating from JobManager
15:28:35,965 INFO  org.apache.flink.runtime.blob.BlobCache                       - Shutting down BlobCache
15:28:35,965 INFO  org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping ZooKeeperLeaderRetrievalService.
15:28:35,973 INFO  org.apache.flink.runtime.taskmanager.Task                     - Freeing task resources for Blocking Vertex (1/1)
15:28:40,135 ERROR org.apache.curator.framework.recipes.cache.PathChildrenCache  - 
java.lang.IllegalStateException: instance must be started before calling this method
	at org.apache.flink.shaded.com.google.common.base.Preconditions.checkState(Preconditions.java:173)
	at org.apache.curator.framework.imps.CuratorFrameworkImpl.getChildren(CuratorFrameworkImpl.java:379)
	at org.apache.curator.framework.recipes.cache.PathChildrenCache.refresh(PathChildrenCache.java:502)
	at org.apache.curator.framework.recipes.cache.RefreshOperation.invoke(RefreshOperation.java:35)
	at org.apache.curator.framework.recipes.cache.PathChildrenCache$9.run(PathChildrenCache.java:759)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
15:28:40,136 INFO  org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Removed job graph 1b1fc2fc95ce37e47702697bb4b4e84d from ZooKeeper.
15:28:40,138 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Blocking Vertex (1/1) (7e54466b8bfc1f8063e7a145126688ae) switched from RUNNING to CANCELING
15:28:40,138 INFO  org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService  - Stopping ZooKeeperLeaderElectionService.
15:28:44,729 INFO  org.apache.curator.framework.state.ConnectionStateManager     - State change: SUSPENDED
15:28:45,962 ERROR org.apache.flink.runtime.jobmanager.JobManagerSubmittedJobGraphsRecoveryITCase  - 
--------------------------------------------------------------------------------
Test testSubmitJobToNonLeader(org.apache.flink.runtime.jobmanager.JobManagerSubmittedJobGraphsRecoveryITCase) failed with:
java.util.concurrent.TimeoutException: Futures timed out after [10000 milliseconds]
	at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:219)
	at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:153)
	at scala.concurrent.Await$$anonfun$ready$1.apply(package.scala:86)
	at scala.concurrent.Await$$anonfun$ready$1.apply(package.scala:86)
	at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:53)
	at scala.concurrent.Await$.ready(package.scala:86)
	at org.apache.flink.runtime.minicluster.FlinkMiniCluster.shutdown(FlinkMiniCluster.scala:321)
	at org.apache.flink.runtime.jobmanager.JobManagerSubmittedJobGraphsRecoveryITCase.testSubmitJobToNonLeader(JobManagerSubmittedJobGraphsRecoveryITCase.java:219)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:606)
	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
	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:271)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:70)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:50)
	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238)
	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63)
	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236)
	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53)
	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229)
	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
	at org.junit.runners.ParentRunner.run(ParentRunner.java:309)
	at org.apache.maven.surefire.junit4.JUnit4Provider.execute(JUnit4Provider.java:283)
	at org.apache.maven.surefire.junit4.JUnit4Provider.executeWithRerun(JUnit4Provider.java:173)
	at org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:153)
	at org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:128)
	at org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:203)
	at org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:155)
	at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:103)
{code}

> Unstable test JobManagerSubmittedJobGraphsRecoveryITCase
> --------------------------------------------------------
>
>                 Key: FLINK-3268
>                 URL: https://issues.apache.org/jira/browse/FLINK-3268
>             Project: Flink
>          Issue Type: Bug
>          Components: Tests
>    Affects Versions: 0.10.1
>            Reporter: Stephan Ewen
>            Priority: Critical
>             Fix For: 1.0.0
>
>
> Logs for the failed test: https://s3.amazonaws.com/archive.travis-ci.org/jobs/103625073/log.txt



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)