You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by "Dong Lin (Jira)" <ji...@apache.org> on 2021/12/03 02:40:00 UTC

[jira] [Created] (FLINK-25148) PipelinedSubpartition.resumeConsumption throws IllegalStateException

Dong Lin created FLINK-25148:
--------------------------------

             Summary: PipelinedSubpartition.resumeConsumption throws IllegalStateException
                 Key: FLINK-25148
                 URL: https://issues.apache.org/jira/browse/FLINK-25148
             Project: Flink
          Issue Type: Bug
          Components: Runtime / Checkpointing
    Affects Versions: 1.14.0
            Reporter: Dong Lin


We observed the following stacktrace when running the unit test org.apache.flink.ml.clustering.KMeansTest::testFeaturePredictionParam. See [https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true].

Note that the stacktrace contains an IllegalStateException from org.apache.flink.runtime.io.network.partition.PipelinedSubpartition.resumeConsumption(PipelinedSubpartition.java:381). Thus it is likely due to a bug in the Flink runtime.

 

{code}
 
Error:  testFeaturePredictionParam Time elapsed: 2.079 s <<< ERROR! 
[25158|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25158]java.lang.RuntimeException: Failed to fetch next result 
[25159|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25159] at org.apache.flink.streaming.api.operators.collect.CollectResultIterator.nextResultFromFetcher(CollectResultIterator.java:109) 
[25160|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25160] at org.apache.flink.streaming.api.operators.collect.CollectResultIterator.hasNext(CollectResultIterator.java:80) 
[25161|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25161] at org.apache.commons.collections.IteratorUtils.toList(IteratorUtils.java:848) 
[25162|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25162] at org.apache.commons.collections.IteratorUtils.toList(IteratorUtils.java:825) 
[25163|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25163] at org.apache.flink.ml.clustering.KMeansTest.executeAndCollect(KMeansTest.java:105) 
[25164|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25164] at org.apache.flink.ml.clustering.KMeansTest.testFeaturePredictionParam(KMeansTest.java:162) 
[25165|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25165] at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
[25166|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25166] at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
[25167|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25167] at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) 
[25168|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25168] at java.lang.reflect.Method.invoke(Method.java:498) 
[25169|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25169] at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50) 
[25170|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25170] at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) 
[25171|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25171] at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47) 
[25172|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25172] at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) 
[25173|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25173] at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) 
[25174|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25174] at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) 
[25175|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25175] at org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45) 
[25176|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25176] at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55) 
[25177|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25177] at org.junit.rules.RunRules.evaluate(RunRules.java:20) 
[25178|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25178] at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325) 
[25179|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25179] at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78) 
[25180|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25180] at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57) 
[25181|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25181] at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290) 
[25182|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25182] at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71) 
[25183|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25183] at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288) 
[25184|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25184] at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58) 
[25185|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25185] at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268) 
[25186|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25186] at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48) 
[25187|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25187] at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48) 
[25188|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25188] at org.junit.rules.RunRules.evaluate(RunRules.java:20) 
[25189|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25189] at org.junit.runners.ParentRunner.run(ParentRunner.java:363) 
[25190|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25190] at org.junit.runner.JUnitCore.run(JUnitCore.java:137) 
[25191|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25191] at org.junit.runner.JUnitCore.run(JUnitCore.java:115) 
[25192|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25192] at org.junit.vintage.engine.execution.RunnerExecutor.execute(RunnerExecutor.java:43) 
[25193|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25193] at java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) 
[25194|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25194] at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) 
[25195|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25195] at java.util.Iterator.forEachRemaining(Iterator.java:116) 
[25196|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25196] at java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801) 
[25197|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25197] at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) 
[25198|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25198] at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) 
[25199|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25199] at java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) 
[25200|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25200] at java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173) 
 
[25251|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25251] at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:78) 
[25252|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25252] at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:163) 
[25253|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25253] at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:24) 
[25254|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25254] at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:20) 
[25255|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25255] at scala.PartialFunction.applyOrElse(PartialFunction.scala:123) 
[25256|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25256] at scala.PartialFunction.applyOrElse$(PartialFunction.scala:122) 
[25257|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25257] at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:20) 
[25258|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25258] at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) 
[25259|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25259] at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) 
[25260|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25260] at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) 
[25261|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25261] at akka.actor.Actor.aroundReceive(Actor.scala:537) 
[25262|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25262] at akka.actor.Actor.aroundReceive$(Actor.scala:535) 
[25263|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25263] at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:220) 
[25264|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25264] at akka.actor.ActorCell.receiveMessage(ActorCell.scala:580) 
[25265|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25265] at akka.actor.ActorCell.invoke(ActorCell.scala:548) 
[25266|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25266] at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:270) 
[25267|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25267] at akka.dispatch.Mailbox.run(Mailbox.scala:231) 
[25268|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25268] at akka.dispatch.Mailbox.exec(Mailbox.scala:243) 
[25269|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25269] at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) 
[25270|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25270] at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) 
[25271|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25271] at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) 
[25272|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25272] at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175) 
[25273|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25273]Caused by: java.lang.IllegalStateException: Should be blocked by checkpoint. 
[25274|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25274] at org.apache.flink.util.Preconditions.checkState(Preconditions.java:193) 
[25275|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25275] at org.apache.flink.runtime.io.network.partition.PipelinedSubpartition.resumeConsumption(PipelinedSubpartition.java:381) 
[25276|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25276] at org.apache.flink.runtime.io.network.partition.PipelinedSubpartitionView.resumeConsumption(PipelinedSubpartitionView.java:79) 
[25277|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25277] at org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel.resumeConsumption(LocalInputChannel.java:283) 
[25278|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25278] at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.resumeConsumption(SingleInputGate.java:867) 
[25279|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25279] at org.apache.flink.runtime.taskmanager.InputGateWithMetrics.resumeConsumption(InputGateWithMetrics.java:67) 
[25280|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25280] at org.apache.flink.streaming.runtime.io.checkpointing.SingleCheckpointBarrierHandler.processBarrier(SingleCheckpointBarrierHandler.java:223) 
[25281|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25281] at org.apache.flink.streaming.runtime.tasks.StreamTask.triggerUnfinishedChannelsCheckpoint(StreamTask.java:1209) 
[25282|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25282] at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$triggerCheckpointAsync$12(StreamTask.java:1126) 
[25283|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25283] at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:50) 
[25284|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25284] at org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90) 
[25285|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25285] at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorImpl.yield(MailboxExecutorImpl.java:86) 
[25286|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25286] at org.apache.flink.iteration.operator.HeadOperator.endInput(HeadOperator.java:408) 
[25287|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25287] at org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.endOperatorInput(StreamOperatorWrapper.java:91) 
[25288|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25288] at org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.endInput(RegularOperatorChain.java:100) 
[25289|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25289] at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:68) 
[25290|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25290] at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:496) 
[25291|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25291] at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:203) 
[25292|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25292] at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:809) 
[25293|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25293] at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:761) 
[25294|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25294] at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:958) 
[25295|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25295] at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:937) 
[25296|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25296] at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:766) 
[25297|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25297] at org.apache.flink.runtime.taskmanager.Task.run(Task.java:575) 
[25298|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25298] at java.lang.Thread.run(Thread.java:748) 
[25299|https://github.com/apache/flink-ml/runs/4379659665?check_suite_focus=true#step:4:25299]
{code}



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