[ https://issues.apache.org/jira/browse/FLINK-9053?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16447221#comment-16447221 ]
ASF GitHub Bot commented on FLINK-9053: --------------------------------------- Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/5748 I was unsure initially, because we tried hard to avoid the checkpoint lock during shutdown/cancellation before (some timer / user thread might hold the lock)... The changes to the network stack (use fewer other locks) seem to make this one here strictly necessary though. I could not yet think of another way to do this. We also have better cancellation safety nets in place now, which should help... > Exception in RecordWriter during cleanup of StreamTask with the checkpoint > trigger running in parallel > ------------------------------------------------------------------------------------------------------ > > Key: FLINK-9053 > URL: https://issues.apache.org/jira/browse/FLINK-9053 > Project: Flink > Issue Type: Bug > Components: Network > Affects Versions: 1.5.0, 1.6.0 > Reporter: Chesnay Schepler > Assignee: Nico Kruber > Priority: Blocker > Fix For: 1.5.0 > > > https://travis-ci.org/apache/flink/jobs/356612100 > {code} > testSavepointRescalingOutKeyedStateDerivedMaxParallelism[0](org.apache.flink.test.checkpointing.RescalingITCase) > Time elapsed: 2.021 sec <<< ERROR! > org.apache.flink.runtime.client.JobExecutionException: Job execution failed. > at > org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply$mcV$sp(JobManager.scala:891) > at > org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:834) > at > org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$6.apply(JobManager.scala:834) > at > scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24) > at > scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24) > at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39) > at > akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415) > 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) > Caused by: java.util.NoSuchElementException: No value present > at java.util.Optional.get(Optional.java:135) > at > org.apache.flink.runtime.io.network.api.writer.RecordWriter.closeBufferBuilder(RecordWriter.java:218) > at > org.apache.flink.runtime.io.network.api.writer.RecordWriter.clearBuffers(RecordWriter.java:178) > at > org.apache.flink.streaming.runtime.io.StreamRecordWriter.close(StreamRecordWriter.java:99) > at > org.apache.flink.streaming.runtime.io.RecordWriterOutput.close(RecordWriterOutput.java:161) > at > org.apache.flink.streaming.runtime.tasks.OperatorChain.releaseOutputs(OperatorChain.java:239) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:402) > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:703) > at java.lang.Thread.run(Thread.java:748) > {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)