Hi Matt,

this seems interesting, I'm aware of some possible inconsistency issues
with unstable connections [1], but I have to yet find out if this could be
related. I'll do some research on this and will get back to you.

In the meantime, can you see anything relevant in the zookeeper logs? Also
which ZK version are you using?

[1] https://issues.apache.org/jira/browse/FLINK-24543

Best,
D.

On Tue, Oct 19, 2021 at 7:49 AM LeVeck, Matt <matt_lev...@intuit.com> wrote:

> My team and I could use some help debugging the following issue, and may
> understanding Flink's full checkpoint recovery decision tree:
>
> We've seen a few times a scenario where a task restarts (but not the job
> manager), a recent checkpoint is saved.  But upon coming back up Flink
> chooses a much older checkpoint.  Below is a log of one such event.  In it
> checkpoint 64996 is written (the log indicates this, and checking S3
> confirms, but the job restarts with 61634.  Looking at the log I'm
> wondering:
>
>    1. Is it likely that Flink failed to update Zookeeper, despite writing
>    the checkpoint to S3?
>    2. In the event where Flink fails to find an entry in Zookeeper, what
>    is its fallback algorithm (where does it look next for a recovery point?)
>    3. It seems to ultimately have ended up in the checkpoint that existed
>    at the time when the job started.  Is there a configuration that would
>    allow the fallback checkpoint to be something more recent?
>
>    Thanks,
>    Matt
>
>
> 2021/10/11 12:22:28.137 INFO  c.i.strmprocess.ArgsPreprocessor - 
> latestSavepointPrefix:desanalytics-7216-doc-comprehension-analytics-7216-prd/4/savepoints/savepoint-000000-abb450590ca7/_metadata
>  LastModified:2021-10-11T12:03:47Z
>
> 2021/10/11 12:22:43.188 INFO  o.a.f.r.c.CheckpointCoordinator - Starting job 
> 00000000000000000000000000000000 from savepoint 
> s3://spp-state-371299021277-tech-core-idx/desanalytics-7216-doc-comprehension-analytics-7216-prd/4/savepoints/savepoint-000000-abb450590ca7
>  (allowing non restored state)
>
> Starting standalonejob as a console application on host 
> doc-comprehension-analytics-7216-7dbfbc4bbd-rzvpw.
>
> args: --job-id 00000000000000000000000000000000 --allowNonRestoredState 
> --job-classname 
> com.intuit.ifdp.doccomprehension.analytics.DocComprehensionAnalyticsProcessor 
>  --fromSavepoint 
> s3://spp-state-371299021277-tech-core-idx/desanalytics-7216-doc-comprehension-analytics-7216-prd/4/savepoints/savepoint-000000-abb450590ca7
>
> 2021/10/11 12:22:51.777 INFO  o.a.f.r.c.CheckpointCoordinator - Reset the 
> checkpoint ID of job 00000000000000000000000000000000 to 61634.
>
> 2021/10/11 12:22:51.777 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Recovering checkpoints from ZooKeeper.
>
> 2021/10/11 12:22:51.788 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Found 1 checkpoints in ZooKeeper.
>
> 2021/10/11 12:22:51.788 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Trying to fetch 1 checkpoints from storage.
>
> 2021/10/11 12:22:51.788 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Trying to retrieve checkpoint 61633.
>
> 2021/10/11 12:22:51.895 INFO  o.a.f.r.c.CheckpointCoordinator - Restoring job 
> 00000000000000000000000000000000 from latest valid checkpoint: Checkpoint 
> 61633 @ 0 for 00000000000000000000000000000000.
>
> <<<<<<<< long run here, with a few errors & recover
>
> >>>>>>>> NO restarts on JM
>
> 2021/10/13 23:10:37.918 INFO  o.a.f.r.c.CheckpointCoordinator - Triggering 
> checkpoint 64996 @ 1634166637914 for job 00000000000000000000000000000000.
>
> 2021/10/13 23:10:49.933 INFO  o.a.f.r.c.CheckpointCoordinator - Completed 
> checkpoint 64996 for job 00000000000000000000000000000000 (20015670 bytes in 
> 11759 ms).
>
> 2021/10/13 23:10:59.200 INFO  o.a.f.s.z.o.a.zookeeper.ClientCnxn - Unable to 
> read additional data from server sessionid 0x17c099f647f1e69, likely server 
> has closed socket, closing socket connection and attempting reconnect
>
> 2021/10/13 23:10:59.301 INFO  o.a.f.s.c.o.a.c.f.s.ConnectionStateManager - 
> State change: SUSPENDED
>
> 2021/10/13 23:10:59.301 WARN  o.a.f.r.l.ZooKeeperLeaderRetrievalService - 
> Connection to ZooKeeper suspended. Can no longer retrieve the leader from 
> ZooKeeper.
>
> 2021/10/13 23:10:59.323 INFO  o.a.f.r.e.ExecutionGraph - Could not restart 
> the job doc-comprehension-analytics-7216 (00000000000000000000000000000000) 
> because the restart strategy prevented it.
>
> 2021/10/13 23:10:59.543 INFO  o.a.f.r.z.ZooKeeperStateHandleStore - Removing 
> /flink/prd/desanalytics-7216/doc-comprehension-analytics-7216/checkpoints/00000000000000000000000000000000
>  from ZooKeeper
>
> 2021/10/13 23:10:59.555 INFO  o.a.f.r.e.ExecutionGraph - Job recovers via 
> failover strategy: full graph restart
>
> 2021/10/13 23:10:59.622 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Recovering checkpoints from ZooKeeper.
>
> 2021/10/13 23:10:59.643 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Found 0 checkpoints in ZooKeeper.
>
> 2021/10/13 23:10:59.643 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Trying to fetch 0 checkpoints from storage.
>
> 2021/10/13 23:10:59.643 INFO  o.a.f.r.c.CheckpointCoordinator - Starting job 
> 00000000000000000000000000000000 from savepoint 
> s3://spp-state-371299021277-tech-core-idx/desanalytics-7216-doc-comprehension-analytics-7216-prd/4/savepoints/savepoint-000000-abb450590ca7
>  (allowing non restored state)
>
> 2021/10/13 23:10:59.744 INFO  o.a.f.r.r.StandaloneResourceManager - 
> Registering TaskManager with ResourceID 085da1dcfff1103b0f054dbd8a275357 
> (akka.ssl.tcp://flink@10.35.132.216:41531/user/taskmanager_0) at 
> ResourceManager
>
> 2021/10/13 23:11:01.550 INFO  o.a.f.r.r.StandaloneResourceManager - Ignoring 
> outdated TaskExecutorGateway connection.
>
> 2021/10/13 23:11:01.550 INFO  o.a.f.r.r.StandaloneResourceManager - 
> Registering TaskManager with ResourceID 7606c3879f564a3d8936c6a1db86e611 
> (akka.ssl.tcp://flink@10.35.164.209:37357/user/taskmanager_0) at 
> ResourceManager
>
> 2021/10/13 23:11:07.215 INFO  o.a.f.r.c.CheckpointCoordinator - Reset the 
> checkpoint ID of job 00000000000000000000000000000000 to 61634.
>
> 2021/10/13 23:11:07.215 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Recovering checkpoints from ZooKeeper.
>
> 2021/10/13 23:11:07.227 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Found 1 checkpoints in ZooKeeper.
>
> 2021/10/13 23:11:07.227 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Trying to fetch 1 checkpoints from storage.
>
> 2021/10/13 23:11:07.227 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Trying to retrieve checkpoint 61633.
>
> 2021/10/13 23:11:07.317 INFO  o.a.f.r.c.CheckpointCoordinator - Restoring job 
> 00000000000000000000000000000000 from latest valid checkpoint: Checkpoint 
> 61633 @ 0 for 00000000000000000000000000000000.
>
> 2021/10/13 23:11:07.331 INFO  o.a.f.r.c.CheckpointCoordinator - No master 
> state to restore
>
> 2021/10/13 23:11:38.522 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Found 1 checkpoints in ZooKeeper.
>
> 2021/10/13 23:11:38.522 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Trying to fetch 1 checkpoints from storage.
>
> 2021/10/13 23:11:38.522 INFO  o.a.f.r.c.ZooKeeperCompletedCheckpointStore - 
> Trying to retrieve checkpoint 61634.
>
> 2021/10/13 23:11:38.531 INFO  o.a.f.r.r.StandaloneResourceManager - 
> Registering TaskManager with ResourceID 085da1dcfff1103b0f054dbd8a275357 
> (akka.ssl.tcp://flink@10.35.132.216:41531/user/taskmanager_0) at 
> ResourceManager
>
> 2021/10/13 23:11:38.591 INFO  o.a.f.r.c.CheckpointCoordinator - Restoring job 
> 00000000000000000000000000000000 from latest valid checkpoint: Checkpoint 
> 61634 @ 1634166669489 for 00000000000000000000000000000000.
>
> 2021/10/13 23:11:38.605 INFO  o.a.f.r.c.CheckpointCoordinator - No master 
> state to restore
>
>
>

Reply via email to