[ https://issues.apache.org/jira/browse/FLINK-19816?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17226460#comment-17226460 ]
Steven Zhen Wu edited comment on FLINK-19816 at 11/5/20, 3:53 AM: ------------------------------------------------------------------ This happened again for the same job in production. I noticed both failures started zookeeper failure. Here are some observations on the sequence of events. My hypothesis is that the race condition / interactions between recovering from zk failure and failure-rate restart-strategy caused this problem of restoring a wrong and very old checkpoint. [~trohrmann] any comment? 1. initially, there were some problems with zookeeper that caused the job to fail {code} 2020-10-25 02:35:59,266 WARN org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.ClientCnxn [main-SendThread(ip-100-81-150-64.us-west-2.compute.internal:2181)] - Client session timed out, have not heard from server in 26676 ms for sessionid 0x363850d0d034d9d 2020-10-25 02:35:59,268 INFO org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.ClientCnxn [main-SendThread(ip-100-81-150-64.us-west-2.compute.internal:2181)] - Client session timed out, have not heard from server in 26676 ms for sessionid 0x363850d0d034d9d, closing socket connection and attempting reconnect 2020-10-25 02:35:59,282 INFO com.netflix.bdp.s3fs.BdpS3FileSystem [cluster-io-thread-25] - Deleting path: s3://us-west-2.spaas.prod/checkpoints/r7E1/clapp_avro-clapp_avro_nontvui/1593/233b4938179c06974e4 535ac8a868675/chk-210758/16d4e138-4199-4fd2-a014-4b394189f72b 2020-10-25 02:35:59,369 INFO org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateManager [main-EventThread] - State change: SUSPENDED {code} 2. This job is configured with `restart-strategy=failure-rate`. and there are enough task restarts to trigger the terminal condition canRestart() to return false. This should eventually lead the Flink job to halt/terminal state. {code} 2020-10-25 02:35:59,641 INFO org.apache.flink.runtime.executiongraph.ExecutionGraph [flink-akka.actor.default-dispatcher-81991] - Job clapp-avro-nontvui (233b4938179c06974e4535ac8a868675) switched from state FAILING to FA ILED. org.apache.flink.runtime.JobException: Recovery is suppressed by FailureRateRestartBackoffTimeStrategy(FailureRateRestartBackoffTimeStrategy(failuresIntervalMS=1800000,backoffTimeMS=30000,maxFailuresPerInterval=20) at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:116) at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:78) at org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:203) at org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:185) at org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:179) at org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:508) at org.apache.flink.runtime.scheduler.UpdateSchedulerNgOnInternalFailuresListener.notifyTaskFailure(UpdateSchedulerNgOnInternalFailuresListener.java:49) at org.apache.flink.runtime.executiongraph.ExecutionGraph.notifySchedulerNgAboutInternalTaskFailure(ExecutionGraph.java:1725) at org.apache.flink.runtime.executiongraph.Execution.processFail(Execution.java:1287) at org.apache.flink.runtime.executiongraph.Execution.processFail(Execution.java:1255) at org.apache.flink.runtime.executiongraph.Execution.fail(Execution.java:954) at org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot.signalPayloadRelease(SingleLogicalSlot.java:173) at org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot.release(SingleLogicalSlot.java:165) at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$SingleTaskSlot.release(SlotSharingManager.java:732) at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.release(SlotSharingManager.java:537) at org.apache.flink.runtime.jobmaster.slotpool.AllocatedSlot.releasePayload(AllocatedSlot.java:149) at org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl.releaseTaskManagerInternal(SlotPoolImpl.java:818) at org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl.releaseTaskManager(SlotPoolImpl.java:777) at org.apache.flink.runtime.jobmaster.JobMaster.disconnectTaskManager(JobMaster.java:435) at org.apache.flink.runtime.jobmaster.JobMaster.onStop(JobMaster.java:352) at org.apache.flink.runtime.rpc.RpcEndpoint.internalCallOnStop(RpcEndpoint.java:216) at org.apache.flink.runtime.rpc.akka.AkkaRpcActor$StartedState.terminate(AkkaRpcActor.java:514) at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleControlMessage(AkkaRpcActor.java:176) at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26) at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21) at scala.PartialFunction.applyOrElse(PartialFunction.scala:123) at scala.PartialFunction.applyOrElse$(PartialFunction.scala:122) at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21) at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) at akka.actor.Actor.aroundReceive(Actor.scala:517) at akka.actor.Actor.aroundReceive$(Actor.scala:515) at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225) at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592) at akka.actor.ActorCell.invoke(ActorCell.scala:561) at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258) at akka.dispatch.Mailbox.run(Mailbox.scala:225) at akka.dispatch.Mailbox.exec(Mailbox.scala:235) at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) Caused by: org.apache.flink.util.FlinkException: Stopping JobMaster for job clapp-avro-nontvui(233b4938179c06974e4535ac8a868675). at org.apache.flink.runtime.jobmaster.JobMaster.onStop(JobMaster.java:349) ... 22 more 2020-10-25 02:35:59,641 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator [flink-akka.actor.default-dispatcher-81991] - Stopping checkpoint coordinator for job 233b4938179c06974e4535ac8a868675. 2020-10-25 02:35:59,641 INFO org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore [flink-akka.actor.default-dispatcher-81991] - Shutting down {code} 3. Flink then removes the checkpoint-counter from zk, as it tries to fail/halt the job {code} 2020-10-25 02:36:03,262 INFO org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore [flink-akka.actor.default-dispatcher-81991] - Removing /spaas/clapp_avro-clapp_avro_nontvui/1593/default/checkpoints/233b4938179c06974e45 35ac8a868675 from ZooKeeper 2020-10-25 02:36:03,269 INFO org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointIDCounter [flink-akka.actor.default-dispatcher-81991] - Shutting down. 2020-10-25 02:36:03,269 INFO org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointIDCounter [flink-akka.actor.default-dispatcher-81991] - Removing /checkpoint-counter/233b4938179c06974e4535ac8a868675 from ZooKeeper {code} 4. but somehow Flink tries to start the job again even though failure-rate restart-strategy reached terminal state for canRestart(). maybe it is because the zookeeper reconnect caused another path of job recovery? because the proper checkpoint info in zookeeper is already cleaned in previous step, now Flink recovered a wrong checkpoint. {code} 2020-10-25 02:36:03,301 INFO org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess [cluster-io-thread-3] - Start SessionDispatcherLeaderProcess. 2020-10-25 02:36:03,302 INFO org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess [cluster-io-thread-5] - Recover all persisted job graphs. 2020-10-25 02:36:03,307 INFO org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess [cluster-io-thread-5] - Trying to recover job with job id 233b4938179c06974e4535ac8a868675. ... 2020-10-25 02:36:03,427 INFO org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore [flink-akka.actor.default-dispatcher-82003] - Recovering checkpoints from ZooKeeper. 2020-10-25 02:36:03,432 INFO org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore [flink-akka.actor.default-dispatcher-82003] - Found 0 checkpoints in ZooKeeper. {code} was (Author: stevenz3wu): This happened again for the same job in production. I noticed both failures started zookeeper failure. Here are some observations on the sequence of events. My hypothesis is that the race condition / interactions between recovering from zk failure and failure-rate restart-strategy caused this problem of restoring a wrong and very old checkpoint. [~trohrmann] 1. initially, there were some problems with zookeeper that caused the job to fail {code} 2020-10-25 02:35:59,266 WARN org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.ClientCnxn [main-SendThread(ip-100-81-150-64.us-west-2.compute.internal:2181)] - Client session timed out, have not heard from server in 26676 ms for sessionid 0x363850d0d034d9d 2020-10-25 02:35:59,268 INFO org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.ClientCnxn [main-SendThread(ip-100-81-150-64.us-west-2.compute.internal:2181)] - Client session timed out, have not heard from server in 26676 ms for sessionid 0x363850d0d034d9d, closing socket connection and attempting reconnect 2020-10-25 02:35:59,282 INFO com.netflix.bdp.s3fs.BdpS3FileSystem [cluster-io-thread-25] - Deleting path: s3://us-west-2.spaas.prod/checkpoints/r7E1/clapp_avro-clapp_avro_nontvui/1593/233b4938179c06974e4 535ac8a868675/chk-210758/16d4e138-4199-4fd2-a014-4b394189f72b 2020-10-25 02:35:59,369 INFO org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateManager [main-EventThread] - State change: SUSPENDED {code} 2. This job is configured with `restart-strategy=failure-rate`. and there are enough task restarts to trigger the terminal condition canRestart() to return false. This should eventually lead the Flink job to halt/terminal state. {code} 2020-10-25 02:35:59,641 INFO org.apache.flink.runtime.executiongraph.ExecutionGraph [flink-akka.actor.default-dispatcher-81991] - Job clapp-avro-nontvui (233b4938179c06974e4535ac8a868675) switched from state FAILING to FA ILED. org.apache.flink.runtime.JobException: Recovery is suppressed by FailureRateRestartBackoffTimeStrategy(FailureRateRestartBackoffTimeStrategy(failuresIntervalMS=1800000,backoffTimeMS=30000,maxFailuresPerInterval=20) at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:116) at org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:78) at org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:203) at org.apache.flink.runtime.scheduler.DefaultScheduler.maybeHandleTaskFailure(DefaultScheduler.java:185) at org.apache.flink.runtime.scheduler.DefaultScheduler.updateTaskExecutionStateInternal(DefaultScheduler.java:179) at org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:508) at org.apache.flink.runtime.scheduler.UpdateSchedulerNgOnInternalFailuresListener.notifyTaskFailure(UpdateSchedulerNgOnInternalFailuresListener.java:49) at org.apache.flink.runtime.executiongraph.ExecutionGraph.notifySchedulerNgAboutInternalTaskFailure(ExecutionGraph.java:1725) at org.apache.flink.runtime.executiongraph.Execution.processFail(Execution.java:1287) at org.apache.flink.runtime.executiongraph.Execution.processFail(Execution.java:1255) at org.apache.flink.runtime.executiongraph.Execution.fail(Execution.java:954) at org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot.signalPayloadRelease(SingleLogicalSlot.java:173) at org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot.release(SingleLogicalSlot.java:165) at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$SingleTaskSlot.release(SlotSharingManager.java:732) at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.release(SlotSharingManager.java:537) at org.apache.flink.runtime.jobmaster.slotpool.AllocatedSlot.releasePayload(AllocatedSlot.java:149) at org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl.releaseTaskManagerInternal(SlotPoolImpl.java:818) at org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl.releaseTaskManager(SlotPoolImpl.java:777) at org.apache.flink.runtime.jobmaster.JobMaster.disconnectTaskManager(JobMaster.java:435) at org.apache.flink.runtime.jobmaster.JobMaster.onStop(JobMaster.java:352) at org.apache.flink.runtime.rpc.RpcEndpoint.internalCallOnStop(RpcEndpoint.java:216) at org.apache.flink.runtime.rpc.akka.AkkaRpcActor$StartedState.terminate(AkkaRpcActor.java:514) at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleControlMessage(AkkaRpcActor.java:176) at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26) at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21) at scala.PartialFunction.applyOrElse(PartialFunction.scala:123) at scala.PartialFunction.applyOrElse$(PartialFunction.scala:122) at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21) at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171) at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172) at akka.actor.Actor.aroundReceive(Actor.scala:517) at akka.actor.Actor.aroundReceive$(Actor.scala:515) at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225) at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592) at akka.actor.ActorCell.invoke(ActorCell.scala:561) at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258) at akka.dispatch.Mailbox.run(Mailbox.scala:225) at akka.dispatch.Mailbox.exec(Mailbox.scala:235) at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) Caused by: org.apache.flink.util.FlinkException: Stopping JobMaster for job clapp-avro-nontvui(233b4938179c06974e4535ac8a868675). at org.apache.flink.runtime.jobmaster.JobMaster.onStop(JobMaster.java:349) ... 22 more 2020-10-25 02:35:59,641 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator [flink-akka.actor.default-dispatcher-81991] - Stopping checkpoint coordinator for job 233b4938179c06974e4535ac8a868675. 2020-10-25 02:35:59,641 INFO org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore [flink-akka.actor.default-dispatcher-81991] - Shutting down {code} 3. Flink then removes the checkpoint-counter from zk, as it tries to fail/halt the job {code} 2020-10-25 02:36:03,262 INFO org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore [flink-akka.actor.default-dispatcher-81991] - Removing /spaas/clapp_avro-clapp_avro_nontvui/1593/default/checkpoints/233b4938179c06974e45 35ac8a868675 from ZooKeeper 2020-10-25 02:36:03,269 INFO org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointIDCounter [flink-akka.actor.default-dispatcher-81991] - Shutting down. 2020-10-25 02:36:03,269 INFO org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointIDCounter [flink-akka.actor.default-dispatcher-81991] - Removing /checkpoint-counter/233b4938179c06974e4535ac8a868675 from ZooKeeper {code} 4. but somehow Flink tries to start the job again even though failure-rate restart-strategy reached terminal state for canRestart(). maybe it is because the zookeeper reconnect caused another path of job recovery? because the proper checkpoint info in zookeeper is already cleaned in previous step, now Flink recovered a wrong checkpoint. {code} 2020-10-25 02:36:03,291 INFO org.apache.flink.runtime.dispatcher.StandaloneDispatcher [flink-akka.actor.default-dispatcher-82003] - Stopped dispatcher akka.tcp://flink@100.115.174.33:36481/user/rpc/dispatcher_1. 2020-10-25 02:36:03,301 INFO org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess [cluster-io-thread-3] - Start SessionDispatcherLeaderProcess. 2020-10-25 02:36:03,302 INFO org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess [cluster-io-thread-5] - Recover all persisted job graphs. 2020-10-25 02:36:03,307 INFO org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess [cluster-io-thread-5] - Trying to recover job with job id 233b4938179c06974e4535ac8a868675. {code} > Flink restored from a wrong checkpoint (a very old one and not the last > completed one) > -------------------------------------------------------------------------------------- > > Key: FLINK-19816 > URL: https://issues.apache.org/jira/browse/FLINK-19816 > Project: Flink > Issue Type: Bug > Components: Runtime / Checkpointing > Affects Versions: 1.11.0 > Reporter: Steven Zhen Wu > Priority: Major > > h2. Summary > Upon failure, it seems that Flink didn't restore from the last completed > checkpoint. Instead, it restored from a very old checkpoint. As a result, > Kafka offsets are invalid and caused the job to replay from the beginning as > Kafka consumer "auto.offset.reset" was set to "EARLIEST". > This is an embarrassingly parallel stateless job. Parallelism is over 1,000. > I have the full log file from jobmanager at INFO level available upon request. > h2. Sequence of events from the logs > Just before the failure, checkpoint *210768* completed. > {code} > 2020-10-25 02:35:05,970 INFO > org.apache.flink.runtime.checkpoint.CheckpointCoordinator > [jobmanager-future-thread-5] - Completed checkpoint 210768 for job > 233b4938179c06974e4535ac8a868675 (4623776 bytes in 120402 ms). > {code} > During restart, somehow it decided to restore from a very old checkpoint > *203531*. > {code:java} > 2020-10-25 02:36:03,301 INFO > org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess > [cluster-io-thread-3] - Start SessionDispatcherLeaderProcess. > 2020-10-25 02:36:03,302 INFO > org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess > [cluster-io-thread-5] - Recover all persisted job graphs. > 2020-10-25 02:36:03,304 INFO com.netflix.bdp.s3fs.BdpS3FileSystem > [cluster-io-thread-25] - Deleting path: > s3://<bucket>/checkpoints/XM3B/clapp_avro-clapp_avro_nontvui/1593/233b4938179c06974e4535ac8a868675/chk-210758/c31aec1e-07a7-4193-aa00-3fbe83f9e2e6 > 2020-10-25 02:36:03,307 INFO > org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess > [cluster-io-thread-5] - Trying to recover job with job id > 233b4938179c06974e4535ac8a868675. > 2020-10-25 02:36:03,381 INFO com.netflix.bdp.s3fs.BdpS3FileSystem > [cluster-io-thread-25] - Deleting path: > s3://<bucket>/checkpoints/Hh86/clapp_avro-clapp_avro_nontvui/1593/233b4938179c06974e4535ac8a868675/chk-210758/4ab92f70-dfcd-4212-9b7f-bdbecb9257fd > ... > 2020-10-25 02:36:03,427 INFO > org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore > [flink-akka.actor.default-dispatcher-82003] - Recovering checkpoints from > ZooKeeper. > 2020-10-25 02:36:03,432 INFO > org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore > [flink-akka.actor.default-dispatcher-82003] - Found 0 checkpoints in > ZooKeeper. > 2020-10-25 02:36:03,432 INFO > org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore > [flink-akka.actor.default-dispatcher-82003] - Trying to fetch 0 checkpoints > from storage. > 2020-10-25 02:36:03,432 INFO > org.apache.flink.runtime.checkpoint.CheckpointCoordinator > [flink-akka.actor.default-dispatcher-82003] - Starting job > 233b4938179c06974e4535ac8a868675 from savepoint > s3://<bucket>/checkpoints/metadata/clapp_avro-clapp_avro_nontvui/1113/47e2a25a8d0b696c7d0d423722bb6f54/chk-203531/_metadata > () > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005)