Skip to content

Allow skipping of data deletion in expire_snapshots #26213

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

Closed
wants to merge 1 commit into from

Conversation

tbaeg
Copy link
Member

@tbaeg tbaeg commented Jul 16, 2025

Description

Allow skipping of file deletion in expire_snapshots to allow for more efficient out-of-band deletes.

Release notes

( ) This is not user-visible or is docs only, and no release notes are required.
(x) Release notes are required. Please propose a release note for me.
( ) Release notes are required, with the following suggested text:

@cla-bot cla-bot bot added the cla-signed label Jul 16, 2025
@github-actions github-actions bot added the iceberg Iceberg connector label Jul 16, 2025
@tbaeg tbaeg force-pushed the expire-snapshots-preserve branch from 5f10507 to fa4285e Compare July 16, 2025 14:55
@ebyhr ebyhr added the needs-docs This pull request requires changes to the documentation label Jul 16, 2025
@tbaeg tbaeg force-pushed the expire-snapshots-preserve branch from fa4285e to bb31e19 Compare July 16, 2025 21:07
@github-actions github-actions bot added the docs label Jul 16, 2025
@@ -881,7 +881,7 @@ with the `retention_threshold` parameter.
`expire_snapshots` can be run as follows:

```sql
ALTER TABLE test_table EXECUTE expire_snapshots(retention_threshold => '7d');
ALTER TABLE test_table EXECUTE expire_snapshots(retention_threshold => '7d', delete_files => true);
Copy link
Member

Choose a reason for hiding this comment

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

I'd like to understand why this is needed. We don't see this in https://iceberg.apache.org/docs/latest/spark-procedures/#expire_snapshots
What is the "more efficient out-of-band" way of doing deletes ? Is there something can we improve about how we execute deletes in the current implementation to avoid the need to have a flag to disable it ?

Copy link
Member Author

Choose a reason for hiding this comment

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

I'd like to understand why this is needed. We don't see this in https://iceberg.apache.org/docs/latest/spark-procedures/#expire_snapshots

So a few quick reasons:

  1. To meet data retention policy requirements.
  2. Reduce coordinator work load pressure.
  3. Flexibility in how table maintenance is performed.
  4. expire_snapshots execution speed.

The main use case I am targeting is speeding up execution of expire_snapshots for a large table(s).

For a small number of tables, with reasonable amount of data, there isn't a ton of benefit (hence the default is true). But I have found at certain scales (i.e. - petabytes of data and or trillions of rows), the call to expire_snapshots can take an extremely long time; most of the time taken spent on the deletion of files.

Additionally, depending on how many tables are being curated concurrently, there can be a fair bit of pressure on the coordinator.

With this parameter, we have more control over when we actually delete the underlying data while still benefiting from minimizing metadata for query planning.

What is the "more efficient out-of-band" way of doing deletes ?

What is "more efficient" depends on underlying storage, but a simple example would be leveraging S3 bucket lifecycle policies. Assuming you had a time based retention policy for data in a table, the lifecycle policy would be a convenient and more efficient way to delete the files.

Is there something can we improve about how we execute deletes in the current implementation to avoid the need to have a flag to disable it ?

Parallelizing/distributing the deletes would certainly be an improvement. I believe the current implementation is single threaded.

Copy link
Member

Choose a reason for hiding this comment

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

Assuming you had a time based retention policy for data in a table, the lifecycle policy would be a convenient and more efficient way to delete the files.

If we want to rely on lifecycle policy then we need to "soft delete" the files by adding a deleted tag to the files which the lifecycle policy can then act on. Relying on just time based policy has the danger of either deleting live files or keeping unnecessary files around for too long. Its not easy to keep the expiration interval of the table always aligned with a time based lifecycle policy.

Parallelizing/distributing the deletes would certainly be an improvement. I believe the current implementation is single threaded.

Why don't we use a threadpool in org.apache.iceberg.ExpireSnapshots#executeDeleteWith to use more parallelism in deletes ?

Copy link
Member Author

Choose a reason for hiding this comment

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

If we want to rely on lifecycle policy then we need to "soft delete" the files by adding a deleted tag to the files which the lifecycle policy can then act on. Relying on just time based policy has the danger of either deleting live files or keeping unnecessary files around for too long. Its not easy to keep the expiration interval of the table always aligned with a time based lifecycle policy.

Yeah, definitely some nuance I intentionally didn't touch on but in principal the point remains.

Why don't we use a threadpool in org.apache.iceberg.ExpireSnapshots#executeDeleteWith to use more parallelism in deletes ?

I think we should but the property still adds value for points 1, 2, and 3. I am also happy to throw something together to use a threadpool as well.

@tbaeg tbaeg force-pushed the expire-snapshots-preserve branch from bb31e19 to 26fdfd2 Compare July 16, 2025 22:02
@ebyhr ebyhr removed the needs-docs This pull request requires changes to the documentation label Jul 17, 2025
@tbaeg tbaeg force-pushed the expire-snapshots-preserve branch from 26fdfd2 to 838515d Compare July 21, 2025 13:12
@tbaeg tbaeg force-pushed the expire-snapshots-preserve branch from 838515d to 03ed2e4 Compare July 28, 2025 16:45
@@ -2184,6 +2186,7 @@ private void executeExpireSnapshots(ConnectorSession session, IcebergTableExecut

// ForwardingFileIo handles bulk operations so no separate function implementation is needed
table.expireSnapshots()
.cleanExpiredFiles(expireSnapshotsHandle.deleteFiles())
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 the table harder to clean up in the future, since the first maintenance task for removing expired snapshots is always slow?

Copy link
Member Author

Choose a reason for hiding this comment

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

I'm sure I am following what you mean, could clarify?


List<String> updatedDataFiles = getAllDataFilesFromTableDirectory(table.getName());
assertThat(updatedDataFiles).containsExactlyInAnyOrderElementsOf(initialFiles);
assertQuery("SELECT sum(col2) FROM " + table.getName(), "SELECT 1101");
Copy link
Contributor

Choose a reason for hiding this comment

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

Please add a comment why the result is 1101 -- it's not obvious otherwise considering using the SELECT col2 .. to check each row

Copy link
Member Author

Choose a reason for hiding this comment

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

This is the same test as testExpireSnapshotsPartitionedTable. It also feels a redundant to list out and explain each sql query that was executed to reach this value. Could this be done in a follow up?

@tbaeg tbaeg force-pushed the expire-snapshots-preserve branch from 03ed2e4 to 36d420a Compare August 11, 2025 13:03
@raunaqmorarka
Copy link
Member

Is expire_snapshots still slow after recent improvements ?
I don't see an option to skip deletes in spark procedure https://iceberg.apache.org/docs/latest/spark-procedures/#expire_snapshots
My preference is we either find ways to make expire_snapshots/remove_orphans faster without skipping deletes.
Or we tag files as "deleted" similar to org.apache.iceberg.aws.s3.S3FileIOProperties#DELETE_TAGS_PREFIX and org.apache.iceberg.aws.s3.S3FileIOProperties#DELETE_ENABLED and let users configure tag based object lifecycle policies in S3.
Just skipping the deletes altogether and relying completely on time based lifecycle policy on storage layer doesn't sound safe to me as there is no guarantee that it won't delete live files.
Also, I don't think the solution can be tied to just expire_snapshots procedure, it should be generically targeting deletes in iceberg from any operation. I suspect that deletes from remove_orphans are more of a bottleneck than the ones from expire_snapshots.

@tbaeg
Copy link
Member Author

tbaeg commented Aug 11, 2025

Is expire_snapshots still slow after recent improvements ?

I haven't had a chance to test the improvements as we don't update unless it's targeting a release version.

I don't see an option to skip deletes in spark procedure https://iceberg.apache.org/docs/latest/spark-procedures/#expire_snapshots My preference is we either find ways to make expire_snapshots/remove_orphans faster without skipping deletes.

Not sure lack of support in spark is necessarily an indicator against adding this.

Or we tag files as "deleted" similar to org.apache.iceberg.aws.s3.S3FileIOProperties#DELETE_TAGS_PREFIX and org.apache.iceberg.aws.s3.S3FileIOProperties#DELETE_ENABLED and let users configure tag based object lifecycle policies in S3. Just skipping the deletes altogether and relying completely on time based lifecycle policy on storage layer doesn't sound safe to me as there is no guarantee that it won't delete live files.

To be clear, I'm not saying time based policy is the end all be all; that was just a simple example to illustrate the general intent. I do think tagging is useful and worthwhile to implement. I can take a look to see what that might look like.

Also, I don't think the solution can be tied to just expire_snapshots procedure, it should be generically targeting deletes in iceberg from any operation. I suspect that deletes from remove_orphans are more of a bottleneck than the ones from expire_snapshots.

Other than iterative adding features/options to prevent deletion, I'm not sure of a way to target it holistically. I do agree remove_orphan_files will be more of a bottleneck than others and I have a few ideas around implementing improvements.

All that said, it sounds like there needs to be a more use cases from the community. Will close this PR for now until there are other(s) that find this useful.

@tbaeg tbaeg closed this Aug 11, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Development

Successfully merging this pull request may close these issues.

4 participants