iceberg-python
iceberg-python copied to clipboard
Remove deleted data files with expire_snapshots
Feature Request / Improvement
Running an expire snapshots operation will only rewrite the metadata file without the expired snapshots (and refs/ statistics). It does not delete deleted data files referenced only by the expired snapshots. This can be observed by deleting an entire table and calling expire_snapshots - the data files still exist. Trino and spark both clean up deleted data files when all snapshots referencing them are expired.
From the spec:
When a file is replaced or deleted from the dataset, its manifest entry fields store the snapshot ID
in which the file was deleted and status 2 (deleted). The file may be deleted from the file system
when the snapshot in which it was deleted is garbage collected, assuming that older snapshots
have also been garbage collected [1].
...
1. Technically, data files can be deleted when the last snapshot that contains the file as “live” data is
garbage collected. But this is harder to detect and requires finding the diff of multiple snapshots. It
is easier to track what files are deleted in a snapshot and delete them when that snapshot expires.
It is not recommended to add a deleted file back to a table. Adding a deleted file can lead to edge
cases where incremental deletes can break table snapshots.
Happy to work on this if others agree that it should be added :)
I dont disagree there is a difference between JAVA and how pyiceberg handles the actual data files on the object store. My perspective was that not dealing with the actual file objects during the expiration process, was a feature rather than an oversight or deficiency.
In my data lake, we deal with a good amount of transactions, so I prefer to have the table not "locked-up" so to speak, as much as possible.
That being said, i dont think its a big lift to add an capability to delete data files, or not. I think it would be a good argument we could pass the maintenance constructor, or maybe a table property?? (though i know that might not be part of the spec, and not the best approach, id need to take a look).
@Anton-Tarazi thoughts?
I don't know how the java implementation does it, but if we delete the data files after the new metadata is committed that wouldn't cause a long-running transaction. Other processes would be free to write to the table, while the expire_snapshots process continues deleting the relevant data files. Doing it in this order is fine since those files are orphaned from the table.
(Once #1958 is merged one could just call remove_orphan_files after the expire_snapshots and the result would be the same, but I think its valuable to have expire_snapshots be consistent with the java version).
Making this opt-in seems reasonable. I think if we're gonna deviate from the spec its better to make it an argument rather than a table property.
Yeah, i guess i hadn't considered doing after the expiration happens - seems like a fair point.
Hey @Anton-Tarazi Thanks for raising this!
I do think cleaning up the snapshots makes sense, but it can be pretty expensive. It can be that the files that are referenced are still used by other snapshots. Of course, we could clean up the manifest-lists since they are unqiue per snapshot. We can do this on a best-effort basis; do the commit, and then delete the files.
(Once https://github.com/apache/iceberg-python/pull/1958 is merged one could just call remove_orphan_files after the expire_snapshots and the result would be the same, but I think its valuable to have expire_snapshots be consistent with the java version).
I think there is also a difference here. If we expire a snapshot, we can easily list all the files that are related to that snapshot using the metadata. The remove_orphan_files will do a list operation on the object-store, which can be pretty slow. I think if we want to clean up the data files, we could also collect a Set of the files that are in the expired snapshots, and compare that with the full metadata tree (we can use the metadata tables here).