Skip to content

Flink: add snapshot expiration reset strategy #12639

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Conversation

Guosmilesmile
Copy link
Contributor

We encountered a scenario where, when using Flink source to incrementally consume data from Iceberg, the lastSnapshotId being consumed has already been cleaned up. This can happen, for example, through Spark's expire_snapshots (CALL iceberg.system.expire_snapshots(table => 'default.my_table', older_than => TIMESTAMP '2025-03-25 00:00:00.000', retain_last => 1)) or in other cases where consumption is too slow and historical snapshots are cleaned up.

Caused by: java.lang.IllegalArgumentException: Starting snapshot (exclusive) 2444106500863389603 is not a parent ancestor of end snapshot 2357271669960485754
  at org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument(Preconditions.java:430)
  at org.apache.iceberg.BaseIncrementalScan.fromSnapshotIdExclusive(BaseIncrementalScan.java:179)
  at org.apache.iceberg.BaseIncrementalScan.planFiles(BaseIncrementalScan.java:104)
  at org.apache.iceberg.BaseIncrementalAppendScan.planTasks(BaseIncrementalAppendScan.java:61)
  at org.apache.iceberg.flink.source.FlinkSplitPlanner.planTasks(FlinkSplitPlanner.java:119)
  at org.apache.iceberg.flink.source.FlinkSplitPlanner.planIcebergSourceSplits(FlinkSplitPlanner.java:76)
  at org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl.discoverIncrementalSplits(ContinuousSplitPlannerImpl.java:135)
  at org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl.planSplits(ContinuousSplitPlannerImpl.java:83)
  at org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator.discoverSplits(ContinuousIcebergEnumerator.java:130)
  at org.apache.flink.runtime.source.coordinator.ExecutorNotifier.lambda$notifyReadyAsync$5(ExecutorNotifier.java:130)
  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
  at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)

In such scenarios, the Flink job will repeatedly restart due to the snapshot ID stored in the state being unavailable, requiring manual intervention to restart the job for recovery. We hope to have a mechanism that allows the job to recover automatically when encountering this situation, similar to how Kafka handles out-of-range offsets by automatically starting to consume from the earliest or latest available data.

This PR mainly adds a configuration option called snapshot-expiration-reset-strategy. When the lastSnapshot is not the parent ancestor of the current snapshot, it can be handled in three ways to avoid manual intervention to restart the job for recovery :

Default mode: Maintain the current behavior.
Earliest mode: Start incremental consumption from the oldest snapshot as the lastSnapshot.
Latest mode: Start incremental consumption from the current latest snapshot as the lastSnapshot.

@Guosmilesmile
Copy link
Contributor Author

@pvary @stevenzwu Could you please take some time to review whether this change is feasible? Thanks!

@pvary
Copy link
Contributor

pvary commented Mar 25, 2025

It might be only me, but SnapshotExpirationResetStrategy.LATEST, SnapshotExpirationResetStrategy.EARLIEST is a very serious data corruption issue which should require manual intervention. The only valid use-case would be a CDC table, but for those we don't have a steaming read yet.

@stevenzwu, @mxm: What are your thoughts?

@Guosmilesmile
Copy link
Contributor Author

It might be only me, but SnapshotExpirationResetStrategy.LATEST, SnapshotExpirationResetStrategy.EARLIEST is a very serious data corruption issue which should require manual intervention. The only valid use-case would be a CDC table, but for those we don't have a steaming read yet.

@stevenzwu, @mxm: What are your thoughts?

@pvary Thank you very much for your response. Here is my somewhat immature little thought.

In our daily operations, if we encounter such a scenario and manually intervene, the only way to recover is by modifying the source configuration to set the starting-strategy. However, whether we choose to recover from INCREMENTAL_FROM_EARLIEST_SNAPSHOT or INCREMENTAL_FROM_LATEST_SNAPSHOT, it will inevitably lead to data loss. Recovering from TABLE_SCAN_THEN_INCREMENTAL would result in data duplication, which could also cause a significant traffic impact downstream. None of these options are ideal, as each can introduce certain data issues.

Moreover, manual intervention may not always be timely, potentially leading to even more data loss. This is why we came up with the idea of needing such an automated recovery configuration, while also retaining the default option to support the default behavior.

@Guosmilesmile
Copy link
Contributor Author

Guosmilesmile commented Mar 25, 2025

To add further, manual intervention to recover the job requires abandoning the previous state. If downstream jobs utilize windows, this would also result in the loss of statistical data. To ensure the preservation of state data, we also hope to have an automated recovery mechanism in place.

This PR mainly to avoid manual intervention to restart the job for recovery

@pvary
Copy link
Contributor

pvary commented Mar 25, 2025

@Guosmilesmile: I hope my answer were not too harsh, I did not intend to hurt. All I was trying to say is that in my experience the correctness is the most important feature for a job. YMMW

manual intervention to recover the job requires abandoning the previous state.

Could you use uid to drop the source state, and keep the state of the other operators?

@Guosmilesmile
Copy link
Contributor Author

@Guosmilesmile: I hope my answer were not too harsh, I did not intend to hurt. All I was trying to say is that in my experience the correctness is the most important feature for a job. YMMW

manual intervention to recover the job requires abandoning the previous state.

Could you use uid to drop the source state, and keep the state of the other operators?

Peter, your feedback is very insightful and timely, and I am very glad to receive your response. I would love to communicate with you more; I just worry that my replies may not fully express my thoughts, so I would like to provide additional information.

Modifying the UID can resolve my scenario, but it requires manual intervention each time, especially in the middle of the night, which is why I came up with this PR. I apologize if my previous replies have caused any misunderstandings.

@pvary
Copy link
Contributor

pvary commented Mar 25, 2025

Good to hear!
Let's hear what others think about the possibilities. If all else fails, setting the uid could be done through environmental variables, or job properties, and that could solve your immediate problem.

@Guosmilesmile
Copy link
Contributor Author

Guosmilesmile commented Mar 26, 2025

Yes, you are right. Change uid can help me to save the state.

I think this PR is primarily aimed at providing an automatic recovery mechanism to avoid the need for manual intervention, while the configuration switch can also preserve the original behavior.

Collections.emptyList(), lastPosition, newPosition);
}
case EARLIEST:
default:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will this make DEFAULT case to be the same as EARLIEST ? Is this your expectation ? I thought that for DEFAULT we shouldn't do anything

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for your review! Since there is a check above to determine whether it is the DEFAULT strategy, at this point, only EARLIEST and LATEST .

if (scanContext.snapshotExpirationResetStrategy() != SnapshotExpirationResetStrategy.DEFAULT
          && lastConsumedSnapshotId != null
          && !SnapshotUtil.isParentAncestorOf(
              table, currentSnapshot.snapshotId(), lastConsumedSnapshotId))

However, my implementation could indeed be misleading, and I have already modified this part.

@rodmeneses
Copy link
Contributor

Hi @Guosmilesmile
Thanks so much for the contribution! I have a very basic question (sorry if it’s very basic, but I wanted to give it a shot):

How should we use this new property? Are we planning to set it for jobs that are failing with the “is not a parent ancestor of end snapshot” error, when we suspect that’s because of a race condition on the expire_snapshot spark action?... In such cases, I find this property to be really helpful!

But if we set it once, and we don't remove it, and then we still get this kind of error, but because of a different issue, I’m afraid that we might be missing the real cause of the data corruption. So my point being that maybe manual intervention, even though it is manual, could still be a good option?
Thanks again!

@Guosmilesmile
Copy link
Contributor Author

@rodmeneses Thank you very much for your reply. This PR can provide an auto-recovery mechanism, but it may also cause some issues, like missing the real cause of data corruption.

I think if we have this strategy, we can manage data at different levels. For data with lower reliability requirements, we can enable this strategy for auto-recovery. For data with higher requirements, we can keep it as is and choose manual intervention.

Can we explain the pros and cons in the docs and let the user decide whether to use this config?

Copy link

This pull request has been marked as stale due to 30 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the [email protected] list. Thank you for your contributions.

@github-actions github-actions bot added the stale label May 12, 2025
@mxm
Copy link
Contributor

mxm commented May 16, 2025

I tend to lean towards what @pvary pointed out, that this is a critical condition which warrants manual intervention. However, I think the use case is valid and I agree that there should be an option to skip the dropped snapshot, instead of getting stuck on it.

@github-actions github-actions bot removed the stale label May 17, 2025
@pvary
Copy link
Contributor

pvary commented May 19, 2025

@gyfora: What's your take?

Copy link

This pull request has been marked as stale due to 30 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the [email protected] list. Thank you for your contributions.

@github-actions github-actions bot added the stale label Jun 19, 2025
@Guosmilesmile
Copy link
Contributor Author

Based on the current situation, the demand for this feature is not very high, so it will be temporarily disabled and can be enabled again if there is a need.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants