You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Zhipeng Zhang (Jira)" <ji...@apache.org> on 2022/07/15 03:47:00 UTC

[jira] [Commented] (FLINK-26801) Support duplicate checkpoint aborted messages

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

Zhipeng Zhang commented on FLINK-26801:
---------------------------------------

[https://github.com/apache/flink-ml/runs/7351668847?check_suite_focus=true]

 

The bug appears again..

> Support duplicate checkpoint aborted messages
> ---------------------------------------------
>
>                 Key: FLINK-26801
>                 URL: https://issues.apache.org/jira/browse/FLINK-26801
>             Project: Flink
>          Issue Type: Bug
>          Components: Library / Machine Learning
>    Affects Versions: ml-2.0.0
>            Reporter: Zhipeng Zhang
>            Assignee: Yun Gao
>            Priority: Critical
>              Labels: pull-request-available, test-stability
>             Fix For: ml-2.1.0
>
>
> The flink-ml run fails at the following case [1]:
> ```
> Error: Tests run: 8, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 71.967 s <<< FAILURE! - in org.apache.flink.ml.classification.LogisticRegressionTest 
> [27997|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:27997]Error: testGetModelData Time elapsed: 3.221 s <<< ERROR! 
> [27998|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:27998]java.lang.RuntimeException: Failed to fetch next result 
> [27999|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:27999] at org.apache.flink.streaming.api.operators.collect.CollectResultIterator.nextResultFromFetcher(CollectResultIterator.java:109) 
> [28000|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28000] at org.apache.flink.streaming.api.operators.collect.CollectResultIterator.next(CollectResultIterator.java:88) 
> [28001|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28001] at org.apache.flink.ml.classification.LogisticRegressionTest.testGetModelData(LogisticRegressionTest.java:251) 
> [28002|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28002] at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
> [28003|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28003] at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
> [28004|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28004] at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) 
> [28005|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28005] at java.lang.reflect.Method.invoke(Method.java:498) 
> [28006|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28006] at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50) 
> [28007|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28007] at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) 
> [28008|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28008] at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47) 
> [28009|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28009] at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) 
> [28010|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28010] at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) 
> [28011|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28011] at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48) 
> [28012|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28012] at org.junit.rules.RunRules.evaluate(RunRules.java:20) 
> [28013|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28013] at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325) 
> [28014|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28014] at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78) 
> [28015|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28015] at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57) 
> [28016|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28016] at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290) 
> [28017|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28017] at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71) 
> [28018|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28018] at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288) 
> [28019|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28019] at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58) 
> [28020|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28020] at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268) 
> [28021|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28021] at org.junit.runners.ParentRunner.run(ParentRunner.java:363) 
> [28022|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28022] at org.junit.runner.JUnitCore.run(JUnitCore.java:137) 
> [28023|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28023] at org.junit.runner.JUnitCore.run(JUnitCore.java:115) 
> [28024|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28024] at org.junit.vintage.engine.execution.RunnerExecutor.execute(RunnerExecutor.java:43) 
> [28025|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28025] at java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) 
> [28026|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28026] at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) 
> [28027|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28027] at java.util.Iterator.forEachRemaining(Iterator.java:116) 
> [28028|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28028] at java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801) 
> [28029|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28029] at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) 
> [28030|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28030] at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) 
> [28031|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28031] at java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) 
> [28032|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28032] at java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173) 
> [28033|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28033] at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) 
> [28034|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28034] at java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:485) 
> [28035|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28035] at org.junit.vintage.engine.VintageTestEngine.executeAllChildren(VintageTestEngine.java:82) 
> [28036|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28036] at org.junit.vintage.engine.VintageTestEngine.execute(VintageTestEngine.java:73) 
> [28037|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28037] at org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:220) 
> [28038|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28038] at org.junit.platform.launcher.core.DefaultLauncher.lambda$execute$6(DefaultLauncher.java:188) 
> [28039|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28039] at org.junit.platform.launcher.core.DefaultLauncher.withInterceptedStreams(DefaultLauncher.java:202) 
> [28040|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28040] at org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:181) 
> [28041|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28041] at org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:128) 
> [28042|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28042] at org.apache.maven.surefire.junitplatform.JUnitPlatformProvider.invokeAllTests(JUnitPlatformProvider.java:142) 
> [28043|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28043] at org.apache.maven.surefire.junitplatform.JUnitPlatformProvider.invoke(JUnitPlatformProvider.java:109) 
> [28044|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28044] at org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:384) 
> [28045|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28045] at org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:345) 
> [28046|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28046] at org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:126) 
> [28047|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28047] at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:418) 
> [28048|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28048]Caused by: java.io.IOException: Failed to fetch job execution result 
> [28049|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28049] at org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.getAccumulatorResults(CollectResultFetcher.java:177) 
> [28050|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28050] at org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.next(CollectResultFetcher.java:120) 
> [28051|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28051] at org.apache.flink.streaming.api.operators.collect.CollectResultIterator.nextResultFromFetcher(CollectResultIterator.java:106) 
> [28052|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28052] ... 48 more 
> [28053|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28053]Caused by: java.util.concurrent.ExecutionException: org.apache.flink.runtime.client.JobExecutionException: Job execution failed. 
> [28054|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28054] at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357) 
> [28055|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28055] at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928) 
> [28056|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28056] at org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.getAccumulatorResults(CollectResultFetcher.java:175) 
> [28057|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28057] ... 50 more 
> [28058|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28058]Caused by: org.apache.flink.runtime.client.JobExecutionException: Job execution failed. 
> [28059|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28059] at org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:144) 
> [28060|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28060] at org.apache.flink.runtime.minicluster.MiniClusterJobClient.lambda$getJobExecutionResult$3(MiniClusterJobClient.java:137) 
> [28061|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28061] at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616) 
> [28062|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28062] at java.util.concurrent.CompletableFuture.uniApplyStage(CompletableFuture.java:628) 
> [28063|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28063] at java.util.concurrent.CompletableFuture.thenApply(CompletableFuture.java:1996) 
> [28064|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28064] at org.apache.flink.runtime.minicluster.MiniClusterJobClient.getJobExecutionResult(MiniClusterJobClient.java:134) 
> [28065|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28065] at org.apache.flink.streaming.api.operators.collect.CollectResultFetcher.getAccumulatorResults(CollectResultFetcher.java:174) 
> [28066|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28066] ... 50 more 
> [28067|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28067]Caused by: org.apache.flink.runtime.JobException: Recovery is suppressed by NoRestartBackoffTimeStrategy 
> [28068|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28068] at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:138) 
> [28069|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28069] at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:82) 
> [28070|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28070] at org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:228) 
> [28071|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28071] at org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:218) 
> [28072|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28072] at org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:209) 
> [28073|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28073] at org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:679) 
> [28074|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28074] at org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:79) 
> [28075|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28075] at org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:444) 
> [28076|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28076] at sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source) 
> [28077|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28077] at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) 
> [28078|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28078] at java.lang.reflect.Method.invoke(Method.java:498) 
> [28079|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28079] at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$handleRpcInvocation$1(AkkaRpcActor.java:316) 
> [28080|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28080] at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:83) 
> [28081|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28081] at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:314) 
> [28082|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28082] at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:217) 
> [28083|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28083] at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:78) 
> [28084|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28084] at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:163) 
> [28085|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28085] at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:24) 
> [28086|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28086] at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:20) 
> [28087|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28087] at scala.PartialFunction.applyOrElse(PartialFunction.scala:123) 
> [28088|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28088] at scala.PartialFunction.applyOrElse$(PartialFunction.scala:122) 
> [28089|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28089] at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:20) 
> [28090|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28090] at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) 
> [28091|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28091] at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) 
> [28092|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28092] at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) 
> [28093|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28093] at akka.actor.Actor.aroundReceive(Actor.scala:537) 
> [28094|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28094] at akka.actor.Actor.aroundReceive$(Actor.scala:535) 
> [28095|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28095] at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:220) 
> [28096|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28096] at akka.actor.ActorCell.receiveMessage(ActorCell.scala:580) 
> [28097|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28097] at akka.actor.ActorCell.invoke(ActorCell.scala:548) 
> [28098|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28098] at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:270) 
> [28099|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28099] at akka.dispatch.Mailbox.run(Mailbox.scala:231) 
> [28100|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28100] at akka.dispatch.Mailbox.exec(Mailbox.scala:243) 
> [28101|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28101] at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) 
> [28102|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28102] at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) 
> [28103|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28103] at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) 
> [28104|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28104] at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175) 
> [28105|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28105]Caused by: java.lang.IllegalStateException: Should be blocked by checkpoint. 
> [28106|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28106] at org.apache.flink.util.Preconditions.checkState(Preconditions.java:193) 
> [28107|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28107] at org.apache.flink.runtime.io.network.partition.PipelinedSubpartition.resumeConsumption(PipelinedSubpartition.java:381) 
> [28108|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28108] at org.apache.flink.runtime.io.network.partition.PipelinedSubpartitionView.resumeConsumption(PipelinedSubpartitionView.java:79) 
> [28109|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28109] at org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel.resumeConsumption(LocalInputChannel.java:283) 
> [28110|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28110] at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.resumeConsumption(SingleInputGate.java:867) 
> [28111|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28111] at org.apache.flink.runtime.taskmanager.InputGateWithMetrics.resumeConsumption(InputGateWithMetrics.java:67) 
> [28112|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28112] at org.apache.flink.streaming.runtime.io.checkpointing.SingleCheckpointBarrierHandler.processBarrier(SingleCheckpointBarrierHandler.java:223) 
> [28113|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28113] at org.apache.flink.streaming.runtime.tasks.StreamTask.triggerUnfinishedChannelsCheckpoint(StreamTask.java:1209) 
> [28114|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28114] at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$triggerCheckpointAsync$12(StreamTask.java:1126) 
> [28115|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28115] at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:50) 
> [28116|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28116] at org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90) 
> [28117|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28117] at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorImpl.yield(MailboxExecutorImpl.java:86) 
> [28118|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28118] at org.apache.flink.iteration.operator.HeadOperator.endInput(HeadOperator.java:408) 
> [28119|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28119] at org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.endOperatorInput(StreamOperatorWrapper.java:91) 
> [28120|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28120] at org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.endInput(RegularOperatorChain.java:100) 
> [28121|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28121] at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:68) 
> [28122|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28122] at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:496) 
> [28123|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28123] at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:203) 
> [28124|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28124] at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:809) 
> [28125|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28125] at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:761) 
> [28126|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28126] at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:958) 
> [28127|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28127] at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:937) 
> [28128|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28128] at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:766) 
> [28129|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28129] at org.apache.flink.runtime.taskmanager.Task.run(Task.java:575) 
> [28130|https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true#step:4:28130] at java.lang.Thread.run(Thread.java:750)
> ```
>  
> This bug cannot be repeated after 100 local runs.
>  
>  [1]https://github.com/apache/flink-ml/runs/5638224415?check_suite_focus=true



--
This message was sent by Atlassian Jira
(v8.20.10#820010)