[ https://issues.apache.org/jira/browse/FLINK-23759?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17399126#comment-17399126 ]
Yun Gao commented on FLINK-23759: --------------------------------- This should be because currently when reporting the latest completed checkpoint id with the abortion message, the restored checkpoints are not ignored. I opened a PR to fix this issue~ > notifyCheckpointComplete without corresponding snapshotState > ------------------------------------------------------------ > > Key: FLINK-23759 > URL: https://issues.apache.org/jira/browse/FLINK-23759 > Project: Flink > Issue Type: Bug > Components: Runtime / Checkpointing > Affects Versions: 1.14.0 > Reporter: Arvid Heise > Priority: Blocker > Labels: pull-request-available > Fix For: 1.14.0 > > > In a private run on AZP, I found a run where {{notifyCheckpointComplete}} was > invoked without prior {{snapshotState}} after the default > ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH was changed to true. > https://dev.azure.com/arvidheise0209/arvidheise/_build/results?buildId=1325&view=logs&j=43a593e7-535d-554b-08cc-244368da36b4&t=82d122c0-8bbf-56f3-4c0d-8e3d69630d0f > This causes the following NPE because the implementation relies on > {{notifyCheckpointComplete}} being called after a corresponding > {{snapshotState}} (valid assumption). > {noformat} > 2021-08-12T19:25:20.6237724Z Aug 12 19:25:20 > org.apache.flink.runtime.client.JobExecutionException: Job execution failed. > 2021-08-12T19:25:20.6238810Z Aug 12 19:25:20 at > org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:144) > 2021-08-12T19:25:20.6239964Z Aug 12 19:25:20 at > org.apache.flink.runtime.minicluster.MiniClusterJobClient.lambda$getJobExecutionResult$3(MiniClusterJobClient.java:137) > 2021-08-12T19:25:20.6241075Z Aug 12 19:25:20 at > java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616) > 2021-08-12T19:25:20.6242062Z Aug 12 19:25:20 at > java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591) > 2021-08-12T19:25:20.6243440Z Aug 12 19:25:20 at > java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) > 2021-08-12T19:25:20.6298828Z Aug 12 19:25:20 at > java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975) > 2021-08-12T19:25:20.6300041Z Aug 12 19:25:20 at > org.apache.flink.runtime.rpc.akka.AkkaInvocationHandler.lambda$invokeRpc$0(AkkaInvocationHandler.java:250) > 2021-08-12T19:25:20.6301124Z Aug 12 19:25:20 at > java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774) > 2021-08-12T19:25:20.6302132Z Aug 12 19:25:20 at > java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750) > 2021-08-12T19:25:20.6303136Z Aug 12 19:25:20 at > java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) > 2021-08-12T19:25:20.6304395Z Aug 12 19:25:20 at > java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975) > 2021-08-12T19:25:20.6305382Z Aug 12 19:25:20 at > org.apache.flink.util.concurrent.FutureUtils.doForward(FutureUtils.java:1389) > 2021-08-12T19:25:20.6306384Z Aug 12 19:25:20 at > org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.lambda$null$1(ClassLoadingUtils.java:93) > 2021-08-12T19:25:20.6307484Z Aug 12 19:25:20 at > org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:68) > 2021-08-12T19:25:20.6308655Z Aug 12 19:25:20 at > org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.lambda$guardCompletionWithContextClassLoader$2(ClassLoadingUtils.java:92) > 2021-08-12T19:25:20.6309720Z Aug 12 19:25:20 at > java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774) > 2021-08-12T19:25:20.6310726Z Aug 12 19:25:20 at > java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750) > 2021-08-12T19:25:20.6311740Z Aug 12 19:25:20 at > java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) > 2021-08-12T19:25:20.6312700Z Aug 12 19:25:20 at > java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975) > 2021-08-12T19:25:20.6313684Z Aug 12 19:25:20 at > org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils$1.onComplete(AkkaFutureUtils.java:47) > 2021-08-12T19:25:20.6314630Z Aug 12 19:25:20 at > akka.dispatch.OnComplete.internal(Future.scala:300) > 2021-08-12T19:25:20.6315487Z Aug 12 19:25:20 at > akka.dispatch.OnComplete.internal(Future.scala:297) > 2021-08-12T19:25:20.6316341Z Aug 12 19:25:20 at > akka.dispatch.japi$CallbackBridge.apply(Future.scala:224) > 2021-08-12T19:25:20.6317222Z Aug 12 19:25:20 at > akka.dispatch.japi$CallbackBridge.apply(Future.scala:221) > 2021-08-12T19:25:20.6318115Z Aug 12 19:25:20 at > scala.concurrent.impl.CallbackRunnable.run(Promise.scala:60) > 2021-08-12T19:25:20.6319129Z Aug 12 19:25:20 at > org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils$DirectExecutionContext.execute(AkkaFutureUtils.java:65) > 2021-08-12T19:25:20.6320196Z Aug 12 19:25:20 at > scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:68) > 2021-08-12T19:25:20.6321183Z Aug 12 19:25:20 at > scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1(Promise.scala:284) > 2021-08-12T19:25:20.6322205Z Aug 12 19:25:20 at > scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1$adapted(Promise.scala:284) > 2021-08-12T19:25:20.6323208Z Aug 12 19:25:20 at > scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:284) > 2021-08-12T19:25:20.6324131Z Aug 12 19:25:20 at > akka.pattern.PromiseActorRef.$bang(AskSupport.scala:621) > 2021-08-12T19:25:20.6325186Z Aug 12 19:25:20 at > akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:24) > 2021-08-12T19:25:20.6326204Z Aug 12 19:25:20 at > akka.pattern.PipeToSupport$PipeableFuture$$anonfun$pipeTo$1.applyOrElse(PipeToSupport.scala:23) > 2021-08-12T19:25:20.6327138Z Aug 12 19:25:20 at > scala.concurrent.Future.$anonfun$andThen$1(Future.scala:532) > 2021-08-12T19:25:20.6328036Z Aug 12 19:25:20 at > scala.concurrent.impl.Promise.liftedTree1$1(Promise.scala:29) > 2021-08-12T19:25:20.6329104Z Aug 12 19:25:20 at > scala.concurrent.impl.Promise.$anonfun$transform$1(Promise.scala:29) > 2021-08-12T19:25:20.6330028Z Aug 12 19:25:20 at > scala.concurrent.impl.CallbackRunnable.run(Promise.scala:60) > 2021-08-12T19:25:20.6330990Z Aug 12 19:25:20 at > akka.dispatch.BatchingExecutor$AbstractBatch.processBatch(BatchingExecutor.scala:63) > 2021-08-12T19:25:20.6332098Z Aug 12 19:25:20 at > akka.dispatch.BatchingExecutor$BlockableBatch.$anonfun$run$1(BatchingExecutor.scala:100) > 2021-08-12T19:25:20.6333098Z Aug 12 19:25:20 at > scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:12) > 2021-08-12T19:25:20.6334705Z Aug 12 19:25:20 at > scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:81) > 2021-08-12T19:25:20.6335508Z Aug 12 19:25:20 at > akka.dispatch.BatchingExecutor$BlockableBatch.run(BatchingExecutor.scala:100) > 2021-08-12T19:25:20.6338266Z Aug 12 19:25:20 at > akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:49) > 2021-08-12T19:25:20.6339018Z Aug 12 19:25:20 at > akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:48) > 2021-08-12T19:25:20.6339892Z Aug 12 19:25:20 at > java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) > 2021-08-12T19:25:20.6340671Z Aug 12 19:25:20 at > java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) > 2021-08-12T19:25:20.6341326Z Aug 12 19:25:20 at > java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) > 2021-08-12T19:25:20.6341960Z Aug 12 19:25:20 at > java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175) > 2021-08-12T19:25:20.6342720Z Aug 12 19:25:20 Caused by: > org.apache.flink.runtime.JobException: Recovery is suppressed by > FixedDelayRestartBackoffTimeStrategy(maxNumberRestartAttempts=1, > backoffTimeMS=0) > 2021-08-12T19:25:20.6343650Z Aug 12 19:25:20 at > org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:138) > 2021-08-12T19:25:20.6344554Z Aug 12 19:25:20 at > org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:82) > 2021-08-12T19:25:20.6345383Z Aug 12 19:25:20 at > org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:228) > 2021-08-12T19:25:20.6346140Z Aug 12 19:25:20 at > org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:218) > 2021-08-12T19:25:20.6346909Z Aug 12 19:25:20 at > org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:209) > 2021-08-12T19:25:20.6347684Z Aug 12 19:25:20 at > org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:679) > 2021-08-12T19:25:20.6348417Z Aug 12 19:25:20 at > org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:79) > 2021-08-12T19:25:20.6349129Z Aug 12 19:25:20 at > org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:438) > 2021-08-12T19:25:20.6349738Z Aug 12 19:25:20 at > sun.reflect.GeneratedMethodAccessor21.invoke(Unknown Source) > 2021-08-12T19:25:20.6350350Z Aug 12 19:25:20 at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > 2021-08-12T19:25:20.6350965Z Aug 12 19:25:20 at > java.lang.reflect.Method.invoke(Method.java:498) > 2021-08-12T19:25:20.6351611Z Aug 12 19:25:20 at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$handleRpcInvocation$1(AkkaRpcActor.java:316) > 2021-08-12T19:25:20.6352374Z Aug 12 19:25:20 at > org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:83) > 2021-08-12T19:25:20.6353131Z Aug 12 19:25:20 at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:314) > 2021-08-12T19:25:20.6353837Z Aug 12 19:25:20 at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:217) > 2021-08-12T19:25:20.6354562Z Aug 12 19:25:20 at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:78) > 2021-08-12T19:25:20.6355385Z Aug 12 19:25:20 at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:163) > 2021-08-12T19:25:20.6356022Z Aug 12 19:25:20 at > akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:24) > 2021-08-12T19:25:20.6356601Z Aug 12 19:25:20 at > akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:20) > 2021-08-12T19:25:20.6357200Z Aug 12 19:25:20 at > scala.PartialFunction.applyOrElse(PartialFunction.scala:123) > 2021-08-12T19:25:20.6357787Z Aug 12 19:25:20 at > scala.PartialFunction.applyOrElse$(PartialFunction.scala:122) > 2021-08-12T19:25:20.6358396Z Aug 12 19:25:20 at > akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:20) > 2021-08-12T19:25:20.6359000Z Aug 12 19:25:20 at > scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) > 2021-08-12T19:25:20.6359690Z Aug 12 19:25:20 at > scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) > 2021-08-12T19:25:20.6360303Z Aug 12 19:25:20 at > scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) > 2021-08-12T19:25:20.6360874Z Aug 12 19:25:20 at > akka.actor.Actor.aroundReceive(Actor.scala:537) > 2021-08-12T19:25:20.6361398Z Aug 12 19:25:20 at > akka.actor.Actor.aroundReceive$(Actor.scala:535) > 2021-08-12T19:25:20.6361967Z Aug 12 19:25:20 at > akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:220) > 2021-08-12T19:25:20.6362554Z Aug 12 19:25:20 at > akka.actor.ActorCell.receiveMessage(ActorCell.scala:580) > 2021-08-12T19:25:20.6363106Z Aug 12 19:25:20 at > akka.actor.ActorCell.invoke(ActorCell.scala:548) > 2021-08-12T19:25:20.6363737Z Aug 12 19:25:20 at > akka.dispatch.Mailbox.processMailbox(Mailbox.scala:270) > 2021-08-12T19:25:20.6364412Z Aug 12 19:25:20 at > akka.dispatch.Mailbox.run(Mailbox.scala:231) > 2021-08-12T19:25:20.6364929Z Aug 12 19:25:20 at > akka.dispatch.Mailbox.exec(Mailbox.scala:243) > 2021-08-12T19:25:20.6365363Z Aug 12 19:25:20 ... 4 more > 2021-08-12T19:25:20.6365842Z Aug 12 19:25:20 Caused by: > java.util.concurrent.ExecutionException: java.lang.NullPointerException > 2021-08-12T19:25:20.6383585Z Aug 12 19:25:20 at > java.util.concurrent.FutureTask.report(FutureTask.java:122) > 2021-08-12T19:25:20.6384218Z Aug 12 19:25:20 at > java.util.concurrent.FutureTask.get(FutureTask.java:192) > 2021-08-12T19:25:20.6384965Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.completeProcessing(SourceStreamTask.java:354) > 2021-08-12T19:25:20.6385814Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:325) > 2021-08-12T19:25:20.6386472Z Aug 12 19:25:20 Caused by: > java.lang.NullPointerException > 2021-08-12T19:25:20.6387131Z Aug 12 19:25:20 at > org.apache.flink.table.planner.runtime.utils.FailingCollectionSource.notifyCheckpointComplete(FailingCollectionSource.java:242) > 2021-08-12T19:25:20.6388036Z Aug 12 19:25:20 at > org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.notifyCheckpointComplete(AbstractUdfStreamOperator.java:126) > 2021-08-12T19:25:20.6388900Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.notifyCheckpointComplete(StreamOperatorWrapper.java:99) > 2021-08-12T19:25:20.6389784Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.notifyCheckpointComplete(SubtaskCheckpointCoordinatorImpl.java:360) > 2021-08-12T19:25:20.6390610Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.StreamTask.notifyCheckpointComplete(StreamTask.java:1406) > 2021-08-12T19:25:20.6391388Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$notifyCheckpointAbortAsync$16(StreamTask.java:1366) > 2021-08-12T19:25:20.6392191Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$notifyCheckpointOperation$17(StreamTask.java:1388) > 2021-08-12T19:25:20.6393070Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:93) > 2021-08-12T19:25:20.6394070Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90) > 2021-08-12T19:25:20.6394767Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorImpl.tryYield(MailboxExecutorImpl.java:97) > 2021-08-12T19:25:20.6395597Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.quiesceTimeServiceAndFinishOperator(StreamOperatorWrapper.java:162) > 2021-08-12T19:25:20.6396421Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.finish(StreamOperatorWrapper.java:130) > 2021-08-12T19:25:20.6397168Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.OperatorChain.finishOperators(OperatorChain.java:460) > 2021-08-12T19:25:20.6397847Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.StreamTask.endData(StreamTask.java:531) > 2021-08-12T19:25:20.6398739Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.lambda$completeProcessing$0(SourceStreamTask.java:351) > 2021-08-12T19:25:20.6399564Z Aug 12 19:25:20 at > org.apache.flink.util.function.FunctionUtils.lambda$asCallable$5(FunctionUtils.java:126) > 2021-08-12T19:25:20.6400205Z Aug 12 19:25:20 at > java.util.concurrent.FutureTask.run(FutureTask.java:266) > 2021-08-12T19:25:20.6400970Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:93) > 2021-08-12T19:25:20.6401753Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90) > 2021-08-12T19:25:20.6402520Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMailsWhenDefaultActionUnavailable(MailboxProcessor.java:338) > 2021-08-12T19:25:20.6403354Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:324) > 2021-08-12T19:25:20.6404138Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:201) > 2021-08-12T19:25:20.6404858Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:818) > 2021-08-12T19:25:20.6405556Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.StreamTask.executeInvoke(StreamTask.java:745) > 2021-08-12T19:25:20.6406273Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:784) > 2021-08-12T19:25:20.6406970Z Aug 12 19:25:20 at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:727) > 2021-08-12T19:25:20.6407583Z Aug 12 19:25:20 at > org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:786) > 2021-08-12T19:25:20.6408172Z Aug 12 19:25:20 at > org.apache.flink.runtime.taskmanager.Task.run(Task.java:572) > 2021-08-12T19:25:20.6408718Z Aug 12 19:25:20 at > java.lang.Thread.run(Thread.java:748) > {noformat} > [~gaoyunhaii][~pnowojski] -- This message was sent by Atlassian Jira (v8.3.4#803005)