Details
-
Bug
-
Status: Resolved
-
Critical
-
Resolution: Fixed
-
None
-
None
Description
- 1.20 Java 11 / Test (module: tests) https://github.com/apache/flink/actions/runs/9217608897/job/25360103124#step:10:8641
ResumeCheckpointManuallyITCase.testExternalizedIncrementalRocksDBCheckpointsWithLocalRecoveryZookeeper throws a NullPointerException when it tries to restore state handles:
Error: 02:57:52 02:57:52.551 [ERROR] Tests run: 48, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 268.6 s <<< FAILURE! -- in org.apache.flink.test.checkpointing.ResumeCheckpointManuallyITCase Error: 02:57:52 02:57:52.551 [ERROR] org.apache.flink.test.checkpointing.ResumeCheckpointManuallyITCase.testExternalizedIncrementalRocksDBCheckpointsWithLocalRecoveryZookeeper[RestoreMode = CLAIM] -- Time elapsed: 3.145 s <<< ERROR! May 24 02:57:52 org.apache.flink.runtime.JobException: Recovery is suppressed by NoRestartBackoffTimeStrategy May 24 02:57:52 at org.apache.flink.runtime.executiongraph.failover.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:219) May 24 02:57:52 at org.apache.flink.runtime.executiongraph.failover.ExecutionFailureHandler.handleFailureAndReport(ExecutionFailureHandler.java:166) May 24 02:57:52 at org.apache.flink.runtime.executiongraph.failover.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:121) May 24 02:57:52 at org.apache.flink.runtime.scheduler.DefaultScheduler.recordTaskFailure(DefaultScheduler.java:279) May 24 02:57:52 at org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:270) May 24 02:57:52 at org.apache.flink.runtime.scheduler.DefaultScheduler.onTaskFailed(DefaultScheduler.java:263) May 24 02:57:52 at org.apache.flink.runtime.scheduler.SchedulerBase.onTaskExecutionStateUpdate(SchedulerBase.java:788) May 24 02:57:52 at org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:765) May 24 02:57:52 at org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:83) May 24 02:57:52 at org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:496) May 24 02:57:52 at jdk.internal.reflect.GeneratedMethodAccessor29.invoke(Unknown Source) May 24 02:57:52 at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) May 24 02:57:52 at java.base/java.lang.reflect.Method.invoke(Method.java:566) May 24 02:57:52 at org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.lambda$handleRpcInvocation$1(PekkoRpcActor.java:318) May 24 02:57:52 at org.apache.flink.runtime.concurrent.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:83) May 24 02:57:52 at org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.handleRpcInvocation(PekkoRpcActor.java:316) May 24 02:57:52 at org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.handleRpcMessage(PekkoRpcActor.java:229) May 24 02:57:52 at org.apache.flink.runtime.rpc.pekko.FencedPekkoRpcActor.handleRpcMessage(FencedPekkoRpcActor.java:88) May 24 02:57:52 at org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.handleMessage(PekkoRpcActor.java:174) May 24 02:57:52 at org.apache.pekko.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:33) May 24 02:57:52 at org.apache.pekko.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:29) May 24 02:57:52 at scala.PartialFunction.applyOrElse(PartialFunction.scala:127) May 24 02:57:52 at scala.PartialFunction.applyOrElse$(PartialFunction.scala:126) May 24 02:57:52 at org.apache.pekko.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:29) May 24 02:57:52 at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:175) May 24 02:57:52 at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176) May 24 02:57:52 at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176) May 24 02:57:52 at org.apache.pekko.actor.Actor.aroundReceive(Actor.scala:547) May 24 02:57:52 at org.apache.pekko.actor.Actor.aroundReceive$(Actor.scala:545) May 24 02:57:52 at org.apache.pekko.actor.AbstractActor.aroundReceive(AbstractActor.scala:229) May 24 02:57:52 at org.apache.pekko.actor.ActorCell.receiveMessage(ActorCell.scala:590) May 24 02:57:52 at org.apache.pekko.actor.ActorCell.invoke(ActorCell.scala:557) May 24 02:57:52 at org.apache.pekko.dispatch.Mailbox.processMailbox(Mailbox.scala:280) May 24 02:57:52 at org.apache.pekko.dispatch.Mailbox.run(Mailbox.scala:241) May 24 02:57:52 at org.apache.pekko.dispatch.Mailbox.exec(Mailbox.scala:253) May 24 02:57:52 at java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:290) May 24 02:57:52 at java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1020) May 24 02:57:52 at java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1656) May 24 02:57:52 at java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1594) May 24 02:57:52 at java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:183) May 24 02:57:52 Caused by: java.lang.NullPointerException May 24 02:57:52 at org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManagerBase.isManagedByFileMergingManager(FileMergingSnapshotManagerBase.java:733) May 24 02:57:52 at org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManagerBase.lambda$restoreStateHandles$4(FileMergingSnapshotManagerBase.java:687) May 24 02:57:52 at java.base/java.util.HashMap.computeIfAbsent(HashMap.java:1134) May 24 02:57:52 at org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManagerBase.lambda$restoreStateHandles$5(FileMergingSnapshotManagerBase.java:683) May 24 02:57:52 at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) May 24 02:57:52 at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) May 24 02:57:52 at java.base/java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:177) May 24 02:57:52 at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) May 24 02:57:52 at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) May 24 02:57:52 at java.base/java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:177) May 24 02:57:52 at java.base/java.util.stream.Streams$StreamBuilderImpl.forEachRemaining(Streams.java:411) May 24 02:57:52 at java.base/java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:658) May 24 02:57:52 at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:274) May 24 02:57:52 at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948) May 24 02:57:52 at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484) May 24 02:57:52 at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474) May 24 02:57:52 at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) May 24 02:57:52 at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173) May 24 02:57:52 at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) May 24 02:57:52 at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:497) May 24 02:57:52 at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:274) May 24 02:57:52 at java.base/java.util.ArrayList$Itr.forEachRemaining(ArrayList.java:1033) May 24 02:57:52 at java.base/java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801) May 24 02:57:52 at java.base/java.util.stream.Streams$ConcatSpliterator.forEachRemaining(Streams.java:734) May 24 02:57:52 at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484) May 24 02:57:52 at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474) May 24 02:57:52 at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) May 24 02:57:52 at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173) May 24 02:57:52 at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) May 24 02:57:52 at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:497) May 24 02:57:52 at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:274) May 24 02:57:52 at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948) May 24 02:57:52 at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484) May 24 02:57:52 at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474) May 24 02:57:52 at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) May 24 02:57:52 at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173) May 24 02:57:52 at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) May 24 02:57:52 at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:497) May 24 02:57:52 at org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManagerBase.restoreStateHandles(FileMergingSnapshotManagerBase.java:680) May 24 02:57:52 at org.apache.flink.runtime.checkpoint.filemerging.SubtaskFileMergingManagerRestoreOperation.restore(SubtaskFileMergingManagerRestoreOperation.java:102) May 24 02:57:52 at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.registerRestoredStateToFileMergingManager(StreamTaskStateInitializerImpl.java:353) May 24 02:57:52 at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:163) May 24 02:57:52 at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:267) May 24 02:57:52 at org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.initializeStateAndOpenOperators(RegularOperatorChain.java:106) May 24 02:57:52 at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreStateAndGates(StreamTask.java:858) May 24 02:57:52 at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$restoreInternal$5(StreamTask.java:812) May 24 02:57:52 at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.call(StreamTaskActionExecutor.java:55) May 24 02:57:52 at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:812) May 24 02:57:52 at org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:771) May 24 02:57:52 at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:970) May 24 02:57:52 at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:939) May 24 02:57:52 at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:763) May 24 02:57:52 at org.apache.flink.runtime.taskmanager.Task.run(Task.java:575) May 24 02:57:52 at java.base/java.lang.Thread.run(Thread.java:829)
This looks like a similar error that happened in FLINK-35382, but for a different reason.
Attachments
Issue Links
- relates to
-
FLINK-35457 EventTimeWindowCheckpointingITCase fails on AZP as NPE
- Resolved
-
FLINK-35456 Many tests fails on AZP as NPE related to FileMerging
- Closed
- links to