-
Notifications
You must be signed in to change notification settings - Fork 3.4k
Improve file delete performance for expire_snapshots #26230
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
9e77d23 to
9ffdd87
Compare
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java
Outdated
Show resolved
Hide resolved
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java
Outdated
Show resolved
Hide resolved
.../trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMinioConnectorSmokeTest.java
Outdated
Show resolved
Hide resolved
896fd0d to
e0cd470
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Pull Request Overview
This PR improves the performance of the expire_snapshots procedure in Iceberg by introducing parallel file deletion capabilities and delegating file cleanup operations to Iceberg's native bulk delete functionality.
Key changes include:
- Introduces a new configurable thread pool for file deletion operations
- Replaces custom file deletion logic with Iceberg's native bulk delete operations
- Updates thread pool sizing defaults with reasonable upper bounds
Reviewed Changes
Copilot reviewed 12 out of 12 changed files in this pull request and generated 2 comments.
Show a summary per file
| File | Description |
|---|---|
| ForIcebergFileDelete.java | Adds new binding annotation for file delete executor dependency injection |
| IcebergConfig.java | Adds configuration for file delete thread pool size |
| IcebergExecutorModule.java | Creates new executor service for file deletion operations |
| IcebergMetadata.java | Replaces custom file deletion with Iceberg's native bulk delete operations |
| IcebergMetadataFactory.java | Integrates file delete executor into metadata factory |
| TestIcebergConfig.java | Updates tests for new configuration and thread pool defaults |
| BaseIcebergMinioConnectorSmokeTest.java | Updates test expectations for bulk delete behavior |
| Various test files | Adds file delete executor parameter to test constructors |
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java
Show resolved
Hide resolved
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java
Show resolved
Hide resolved
|
@tbaeg are you able to share any benchmark results from trying this change ? |
At the moment, I do not have any benchmarks to share. For our testing environment I'd first need to back port the change. I can try to get something this week. |
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java
Show resolved
Hide resolved
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java
Show resolved
Hide resolved
ebyhr
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks good except for missing documentation.
e0cd470 to
c0402c7
Compare
|
@tbaeg if it is not much trouble, could you add in the description of the PR an informal benchmark with and without parallelism for |
.../trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMinioConnectorSmokeTest.java
Outdated
Show resolved
Hide resolved
c0402c7 to
4dc4d6c
Compare
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java
Outdated
Show resolved
Hide resolved
4dc4d6c to
5bc97de
Compare
I was trying to get something informal together on a test cluster with more substantial data, but it looks like it might not be possible. I can write a simple test and profile it locally, but not sure how worthwhile that is. |
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java
Show resolved
Hide resolved
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergExecutorModule.java
Show resolved
Hide resolved
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java
Show resolved
Hide resolved
dc1c0d2 to
8845baa
Compare
|
Did a POC that parallelizes |
353c0eb to
49a2a12
Compare
raunaqmorarka
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
.
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/FileIoModule.java
Outdated
Show resolved
Hide resolved
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/FileIoFactory.java
Outdated
Show resolved
Hide resolved
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingFileIo.java
Outdated
Show resolved
Hide resolved
...in/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/RegisterTableProcedure.java
Outdated
Show resolved
Hide resolved
d39a0a4 to
1be0c5a
Compare
raunaqmorarka
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
lgtm overall
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java
Show resolved
Hide resolved
plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java
Outdated
Show resolved
Hide resolved
plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java
Outdated
Show resolved
Hide resolved
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/FileIoModule.java
Outdated
Show resolved
Hide resolved
chenjian2664
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks good % comments
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingFileIo.java
Outdated
Show resolved
Hide resolved
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java
Show resolved
Hide resolved
63d7f0d to
4bf7faf
Compare
plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java
Outdated
Show resolved
Hide resolved
- Add ForwardingFileIoFactory so delete specific ExecutorService is not injected in multiple places - Remove direct usage of ForwardingFileIo in favor of a FileIoFactory implementation - Add parallelized deletes in ForwardingFileIo
4bf7faf to
28a31e5
Compare
Description
Improve file delete performance for expire_snapshots.
Crude benchmark in a test. 10 per deletes per request (manually set the delete batch size).
OLD:
NEW:
Release notes
( ) This is not user-visible or is docs only, and no release notes are required.
( ) Release notes are required. Please propose a release note for me.
(x) Release notes are required, with the following suggested text: