├── .gitignore ├── Cargo.lock ├── Cargo.toml ├── LICENSE ├── README.md ├── codegen ├── Cargo.toml ├── build.rs └── src │ └── lib.rs ├── core ├── Cargo.toml ├── proto │ └── iceberg.proto └── src │ ├── common │ └── mod.rs │ ├── compaction │ └── mod.rs │ ├── config │ └── mod.rs │ ├── error │ └── mod.rs │ ├── executor │ ├── datafusion │ │ ├── datafusion_processor.rs │ │ ├── file_scan_task_table_provider.rs │ │ ├── iceberg_file_task_scan.rs │ │ └── mod.rs │ ├── iceberg_writer │ │ ├── mod.rs │ │ └── rolling_iceberg_writer.rs │ ├── mock.rs │ └── mod.rs │ ├── lib.rs │ └── parser │ ├── mod.rs │ └── proto.rs ├── docker └── compactor │ ├── Dockerfile │ ├── config.yaml │ └── docker-compose.yml ├── rust-toolchain └── services └── compactor ├── Cargo.toml ├── config.yaml └── src ├── bin └── main.rs ├── config.rs ├── lib.rs ├── rpc.rs └── server.rs /.gitignore: -------------------------------------------------------------------------------- 1 | # macOS 2 | **/*.DS_Store 3 | 4 | # Visual Studio Code 5 | .vscode/* 6 | !.vscode/*.example 7 | .devcontainer/ 8 | 9 | # JetBrains 10 | .idea/ 11 | *.iml 12 | .fleet/ 13 | 14 | # GitPod 15 | .gitpod.yml 16 | 17 | # CLion 18 | cmake-build-debug/ 19 | /cmake-build-wsl_profile/ 20 | 21 | # build 22 | *.exe 23 | *.app 24 | build/ 25 | 26 | # Rust 27 | core/**/target 28 | core/**/target_tarpaulin 29 | codegen/**/target 30 | codegen/src/** 31 | !codegen/src/lib.rs 32 | sevices/compactor/**/target 33 | target/ 34 | target-bisector-*/ 35 | 36 | # ctags 37 | **/tags 38 | 39 | log/ 40 | 41 | *.log -------------------------------------------------------------------------------- /Cargo.toml: -------------------------------------------------------------------------------- 1 | [workspace] 2 | resolver = "2" 3 | members = ["core", "codegen", "services/compactor"] 4 | 5 | [workspace.dependencies] 6 | # Async runtime and utilities 7 | async-stream = "0.3.5" 8 | async-trait = "0.1.86" 9 | futures = "0.3.17" 10 | futures-async-stream = "0.2.9" 11 | tokio = { version = "1", default-features = false } 12 | 13 | # Data processing and storage 14 | iceberg = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "b1619c554c2922ddad00cc0564fa79eaec62af5b", features = [ 15 | "storage-s3", 16 | "storage-gcs", 17 | ] } 18 | iceberg-datafusion = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "b1619c554c2922ddad00cc0564fa79eaec62af5b" } 19 | parquet = { version = "54", features = ["async"] } 20 | 21 | # gRPC and Protocol Buffers 22 | prost = "0.12" 23 | tonic = "0.11.0" 24 | 25 | # Serialization 26 | serde = { version = "1", features = ["rc"] } 27 | serde_with = { version = "3", features = ["json"] } 28 | serde_json = { version = "1" } 29 | 30 | # Error handling 31 | thiserror = "2.0.11" 32 | 33 | # Utilities 34 | url = "2" 35 | 36 | # Local workspace members 37 | bergloom-core = { path = "./core" } 38 | bergloom-codegen = { path = "./codegen" } 39 | bergloom-service-compactor = { path = "./services/compactor" } 40 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "[]" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright [yyyy] [name of copyright owner] 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # BergLoom 2 | 3 | BergLoom is a high-performance Iceberg table compaction runtime built on Rust, optimized specifically for large-scale data lakes. The project name, which means "iceberg weaver," symbolizes its core function of weaving scattered Iceberg data files into an efficient and orderly data structure. 4 | 5 | [![License](https://img.shields.io/badge/License-Apache%202.0-blue.svg)](https://opensource.org/licenses/Apache-2.0) 6 | 7 | ## 🌟 Core Highlights 8 | 9 | - **Rust-Native Performance**: Low-latency, high-throughput compaction with memory safety guarantees 10 | - **DataFusion Engine**: Leverages Apache DataFusion for query planning and vectorized execution 11 | - **Iceberg Native Support**: Full compliance with Iceberg table formats via iceberg-rs 12 | - **Multi-Cloud Ready**: Currently supports AWS S3, with plans for Azure Blob Storage and GCP Cloud Storage 13 | 14 | ## 🛠️ Basic Functionality 15 | 16 | - **Full Compaction**: Merges all data files in an Iceberg table and removes old files 17 | - **Deletion Support**: 18 | - Positional deletions (POS_DELETE) 19 | - Equality deletions (EQ_DELETE) 20 | 21 | ## 📝 Code Example 22 | 23 | ```rust 24 | ``` 25 | 26 | ## 🗺️ Roadmap 27 | 28 | ### Runtime Enhancements 29 | - [ ] Incremental compaction support 30 | - [ ] Merge-on-read performance optimization 31 | - [ ] Standalone scheduler component 32 | 33 | ### Iceberg Features 34 | - [ ] Partition evolution support 35 | - [ ] Schema evolution support 36 | 37 | ### Cloud Support 38 | - [ ] Azure Blob Storage integration 39 | - [ ] GCP Cloud Storage integration -------------------------------------------------------------------------------- /codegen/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "bergloom-codegen" 3 | version = "0.1.0" 4 | edition = "2024" 5 | description = "Protocol Buffers code generation for IC project" 6 | 7 | [lib] 8 | name = "bergloom_codegen" 9 | path = "src/lib.rs" 10 | 11 | [dependencies] 12 | prost = { workspace = true } 13 | tonic = { workspace = true } 14 | 15 | [build-dependencies] 16 | prost-build = "0.12" 17 | tonic-build = "0.11" 18 | 19 | -------------------------------------------------------------------------------- /codegen/build.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | fn main() -> Result<(), Box> { 18 | let proto_dir = std::path::Path::new("../core/proto").canonicalize()?; 19 | let proto_file = proto_dir.join("iceberg.proto"); 20 | 21 | tonic_build::configure() 22 | .build_server(true) 23 | .protoc_arg("--experimental_allow_proto3_optional") 24 | .out_dir("src") 25 | .compile(&[proto_file], &[proto_dir]) 26 | .expect("Failed to compile protos"); 27 | 28 | Ok(()) 29 | } 30 | -------------------------------------------------------------------------------- /codegen/src/lib.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | //! Protocol Buffers code generation for IC project 18 | //! 19 | //! This crate is responsible for generating Rust code from Protocol Buffers definitions. 20 | 21 | pub mod compactor; 22 | -------------------------------------------------------------------------------- /core/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "bergloom-core" 3 | version = "0.1.0" 4 | edition = "2024" 5 | 6 | [dependencies] 7 | async-stream = { workspace = true } 8 | async-trait = { workspace = true } 9 | bergloom-codegen = { workspace = true } 10 | backon = "1.2.0" 11 | datafusion = "45.0.0" 12 | futures = { workspace = true } 13 | futures-async-stream = { workspace = true } 14 | iceberg = { workspace = true } 15 | iceberg-datafusion = { workspace = true } 16 | mixtrics = "0.1.0" 17 | parquet = { workspace = true } 18 | prost = { workspace = true } 19 | serde = { workspace = true } 20 | serde_with = { workspace = true } 21 | sqlx = { version = "0.8.2", default-features = false, features = [ 22 | "bigdecimal", 23 | "chrono", 24 | "json", 25 | "mysql", 26 | "postgres", 27 | "runtime-tokio-native-tls", 28 | "rust_decimal", 29 | "sqlite", 30 | "time", 31 | "uuid", 32 | ] } 33 | thiserror = { workspace = true } 34 | tokio = { workspace = true } 35 | tracing = "0.1" 36 | url = { workspace = true } 37 | serde_json = { workspace = true } 38 | -------------------------------------------------------------------------------- /core/proto/iceberg.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | 3 | package compactor; 4 | 5 | enum DataContentType { 6 | DATA = 0; 7 | POSITION_DELETES = 1; 8 | EQUALIRY_DELETES = 2; 9 | } 10 | 11 | enum DataFileFormat { 12 | AVRO = 0; 13 | ORC = 1; 14 | PARQUET = 2; 15 | } 16 | 17 | // Transform represents the transformation applied to a source column to produce a partition value 18 | message Transform { 19 | enum TransformWithoutInner { 20 | // Source value, unmodified 21 | IDENTITY = 0; 22 | // Extract a date or timestamp year, as years from 1970 23 | YEAR = 1; 24 | // Extract a date or timestamp month, as months from 1970-01-01 25 | MONTH = 2; 26 | // Extract a date or timestamp day, as days from 1970-01-01 27 | DAY = 3; 28 | // Extract a timestamp hour, as hours from 1970-01-01 00:00:00 29 | HOUR = 4; 30 | // Always produces null 31 | VOID = 5; 32 | // Used to represent some customized transform that can't be recognized or supported now 33 | UNKNOWN = 6; 34 | } 35 | 36 | 37 | // Parameters for the transformation 38 | oneof params { 39 | TransformWithoutInner transform_without_inner = 1; 40 | // For Bucket transform: number of buckets 41 | uint32 bucket = 2; 42 | // For Truncate transform: width to truncate to 43 | uint32 truncate = 3; 44 | } 45 | } 46 | 47 | // PartitionField represents a field used for partitioning data 48 | message PartitionField { 49 | int32 source_id = 1; 50 | // A partition field id that is unique within a partition spec 51 | optional int32 field_id = 2; 52 | // A partition name 53 | string name = 3; 54 | // A transform that is applied to the source column to produce a partition value 55 | Transform transform = 4; 56 | } 57 | 58 | message PartitionSpec { 59 | int32 spec_id = 1; 60 | repeated PartitionField partition_fields = 2; 61 | } 62 | 63 | message PrimitiveType { 64 | message Decimal { 65 | uint32 precision = 1; 66 | uint32 scale = 2; 67 | } 68 | enum KindWithoutInner{ 69 | BOOLEAN = 0; 70 | INT =1; 71 | LONG = 2; 72 | FLOAT = 3; 73 | DOUBLE = 4; 74 | DATE = 5; 75 | TIME = 6; 76 | TIMESTAMP = 7; 77 | TIMESTAMPTZ = 8; 78 | TIMESTAMP_NS = 9; 79 | TIMESTAMPTZ_NS = 10; 80 | STRING = 11; 81 | UUID = 12; 82 | BINARY = 13; 83 | } 84 | oneof kind { 85 | KindWithoutInner kind_without_inner =1; 86 | Decimal decimal = 2; 87 | uint64 fixed = 3; 88 | } 89 | } 90 | 91 | message StructType { 92 | repeated NestedFieldDescriptor fields = 1; 93 | } 94 | 95 | message MapType { 96 | NestedFieldDescriptor key_field = 1; 97 | NestedFieldDescriptor value_field = 2; 98 | } 99 | 100 | message NestedFieldDescriptor{ 101 | int32 id = 1; 102 | string name = 2; 103 | bool required = 3; 104 | oneof field_type { 105 | PrimitiveType primitive = 4; 106 | StructType struct = 5; 107 | NestedFieldDescriptor list = 6; 108 | MapType map = 7; 109 | } 110 | } 111 | 112 | message SchemaDescriptor { 113 | uint64 schema_id = 1; 114 | repeated NestedFieldDescriptor fields = 2; 115 | } 116 | 117 | message FileScanTaskDescriptor { 118 | uint64 start = 1; 119 | uint64 length = 2; 120 | uint64 record_count = 3; 121 | string data_file_path = 4; 122 | DataContentType data_file_content = 5; 123 | DataFileFormat data_file_format = 6; 124 | repeated int32 project_field_ids = 7; 125 | int64 sequence_number = 8; 126 | repeated int32 equality_ids = 9; 127 | } 128 | 129 | message FileIoBuilder { 130 | string scheme_str = 1; 131 | map props = 2; 132 | } 133 | 134 | message RewriteFilesRequest { 135 | repeated FileScanTaskDescriptor file_scan_task_descriptor = 1; 136 | map rewrite_file_config = 2; 137 | string dir_path = 3; 138 | FileIoBuilder file_io_builder = 4; 139 | SchemaDescriptor schema = 5; 140 | PartitionSpec partition_spec = 6; 141 | } 142 | 143 | message PrimitiveLiteral { 144 | enum KindWithoutInnerLiteral { 145 | ABOVE_MAX = 0; 146 | BELOW_MIN = 1; 147 | } 148 | oneof kindLiteral { 149 | bool boolean = 1; 150 | int32 int = 2; 151 | int64 long = 3; 152 | float float = 4; 153 | double double = 5; 154 | string string = 6; 155 | bytes binary = 7; 156 | bytes int128 = 8; 157 | bytes uint128 = 9; 158 | KindWithoutInnerLiteral kind_without_inner_literal = 10; 159 | } 160 | } 161 | 162 | message OptionalLiteral { 163 | optional Literal value = 1; 164 | } 165 | 166 | message StructLiteralDescriptor{ 167 | repeated OptionalLiteral inner = 1; 168 | } 169 | 170 | message MapLiteral { 171 | repeated Literal keys = 1; 172 | repeated OptionalLiteral values = 2; 173 | } 174 | 175 | message Literal{ 176 | oneof literal { 177 | PrimitiveLiteral primitive = 1; 178 | StructLiteralDescriptor struct = 2; 179 | StructLiteralDescriptor list = 3; 180 | MapLiteral map = 4; 181 | } 182 | } 183 | 184 | message DataFile { 185 | DataContentType content = 1; 186 | string file_path = 2; 187 | DataFileFormat file_format = 3; 188 | StructLiteralDescriptor partition = 4; 189 | uint64 record_count = 5; 190 | uint64 file_size_in_bytes = 6; 191 | map column_sizes = 7; 192 | map value_counts = 8; 193 | map null_value_counts = 9; 194 | map nan_value_counts = 10; 195 | map lower_bounds = 11; 196 | map upper_bounds = 12; 197 | optional bytes key_metadata = 13; 198 | repeated int64 split_offsets = 14; 199 | repeated int32 equality_ids = 15; 200 | optional int32 sort_order_id = 16; 201 | int32 partition_spec_id = 17; 202 | } 203 | 204 | message RewriteFilesStat { 205 | uint32 rewritten_files_count = 2; 206 | uint32 added_files_count = 3; 207 | uint64 rewritten_bytes = 4; 208 | uint32 failed_data_files_count = 5; 209 | } 210 | 211 | message RewriteFilesResponse { 212 | repeated DataFile data_files = 1; 213 | RewriteFilesStat stat = 2; 214 | } 215 | 216 | message EchoRequest { 217 | string message = 1; 218 | } 219 | 220 | message EchoResponse { 221 | string message = 1; 222 | } 223 | 224 | service CompactorService { 225 | rpc RewriteFiles (RewriteFilesRequest) returns (RewriteFilesResponse); 226 | rpc Echo (EchoRequest) returns (EchoResponse); 227 | } -------------------------------------------------------------------------------- /core/src/common/mod.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use mixtrics::metrics::{BoxedCounterVec, BoxedHistogramVec, BoxedRegistry, Buckets}; 18 | 19 | pub struct Metrics { 20 | pub compaction_commit_counter: BoxedCounterVec, 21 | pub compaction_duration: BoxedHistogramVec, 22 | 23 | pub compaction_rewritten_bytes: BoxedCounterVec, 24 | pub compaction_rewritten_files_count: BoxedCounterVec, 25 | pub compaction_added_files_count: BoxedCounterVec, 26 | pub compaction_failed_data_files_count: BoxedCounterVec, 27 | 28 | pub compaction_commit_duration: BoxedHistogramVec, 29 | 30 | pub compaction_commit_failed_counter: BoxedCounterVec, 31 | pub compaction_executor_error_counter: BoxedCounterVec, 32 | } 33 | 34 | impl Metrics { 35 | pub fn new(registry: BoxedRegistry) -> Self { 36 | let compaction_commit_counter = registry.register_counter_vec( 37 | "compaction_commit_counter".into(), 38 | "BergLoom compaction total commit counts".into(), 39 | &["catalog_name", "table_ident"], 40 | ); 41 | 42 | let compaction_duration = registry.register_histogram_vec_with_buckets( 43 | "compaction_duration".into(), 44 | "BergLoom compaction duration in seconds".into(), 45 | &["catalog_name", "table_ident"], 46 | Buckets::exponential( 47 | 1.0, 2.0, 20, // Start at 1 second, double each bucket, up to 20 buckets 48 | ), 49 | ); 50 | 51 | let compaction_rewritten_bytes = registry.register_counter_vec( 52 | "compaction_rewritten_bytes".into(), 53 | "BergLoom compaction rewritten bytes".into(), 54 | &["catalog_name", "table_ident"], 55 | ); 56 | 57 | let compaction_rewritten_files_count = registry.register_counter_vec( 58 | "compaction_rewritten_files_count".into(), 59 | "BergLoom compaction rewritten files count".into(), 60 | &["catalog_name", "table_ident"], 61 | ); 62 | 63 | let compaction_added_files_count = registry.register_counter_vec( 64 | "compaction_added_files_count".into(), 65 | "BergLoom compaction added files count".into(), 66 | &["catalog_name", "table_ident"], 67 | ); 68 | 69 | let compaction_failed_data_files_count = registry.register_counter_vec( 70 | "compaction_failed_data_files_count".into(), 71 | "BergLoom compaction failed data files count".into(), 72 | &["catalog_name", "table_ident"], 73 | ); 74 | 75 | let compaction_commit_duration = registry.register_histogram_vec_with_buckets( 76 | "compaction_commit_duration".into(), 77 | "BergLoom compaction commit duration in milliseconds".into(), 78 | &["catalog_name", "table_ident"], 79 | Buckets::exponential( 80 | 1.0, 2.0, 20, // Start at 1 millisecond, double each bucket, up to 20 buckets 81 | ), 82 | ); 83 | 84 | let compaction_commit_failed_counter = registry.register_counter_vec( 85 | "compaction_commit_failed_counter".into(), 86 | "BergLoom compaction commit failed counts".into(), 87 | &["catalog_name", "table_ident"], 88 | ); 89 | 90 | let compaction_executor_error_counter = registry.register_counter_vec( 91 | "compaction_executor_error_counter".into(), 92 | "BergLoom compaction executor error counts".into(), 93 | &["catalog_name", "table_ident"], 94 | ); 95 | 96 | Self { 97 | compaction_commit_counter, 98 | compaction_duration, 99 | compaction_rewritten_bytes, 100 | compaction_rewritten_files_count, 101 | compaction_added_files_count, 102 | compaction_failed_data_files_count, 103 | compaction_commit_duration, 104 | compaction_commit_failed_counter, 105 | compaction_executor_error_counter, 106 | } 107 | } 108 | } 109 | -------------------------------------------------------------------------------- /core/src/compaction/mod.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use bergloom_codegen::compactor::RewriteFilesStat; 18 | use iceberg::spec::DataFile; 19 | use iceberg::{Catalog, ErrorKind, TableIdent}; 20 | use mixtrics::metrics::BoxedRegistry; 21 | use mixtrics::registry::noop::NoopMetricsRegistry; 22 | 23 | use crate::CompactionError; 24 | use crate::Result; 25 | use crate::common::Metrics; 26 | use crate::executor::{ 27 | ExecutorType, InputFileScanTasks, RewriteFilesRequest, RewriteFilesResponse, 28 | create_compaction_executor, 29 | }; 30 | use crate::{CompactionConfig, CompactionExecutor}; 31 | use futures_async_stream::for_await; 32 | use iceberg::scan::FileScanTask; 33 | use iceberg::table::Table; 34 | use iceberg::transaction::Transaction; 35 | use iceberg::writer::file_writer::location_generator::DefaultLocationGenerator; 36 | use std::collections::HashMap; 37 | use std::sync::Arc; 38 | use std::time::Duration; 39 | 40 | use backon::ExponentialBuilder; 41 | use backon::Retryable; 42 | 43 | pub enum CompactionType { 44 | Full, 45 | } 46 | 47 | /// Builder for creating Compaction instances with flexible configuration 48 | pub struct CompactionBuilder { 49 | config: Option>, 50 | executor_type: ExecutorType, 51 | catalog: Option>, 52 | registry: BoxedRegistry, 53 | table_ident: Option, 54 | compaction_type: Option, 55 | catalog_name: Option, 56 | commit_retry_config: RewriteDataFilesCommitManagerRetryConfig, 57 | } 58 | 59 | impl CompactionBuilder { 60 | /// Create a new CompactionBuilder with default settings 61 | pub fn new() -> Self { 62 | Self { 63 | config: None, 64 | executor_type: ExecutorType::DataFusion, // Default executor type 65 | catalog: None, 66 | registry: Box::new(NoopMetricsRegistry), 67 | table_ident: None, 68 | compaction_type: None, 69 | catalog_name: None, 70 | commit_retry_config: RewriteDataFilesCommitManagerRetryConfig::default(), 71 | } 72 | } 73 | 74 | /// Set the compaction configuration 75 | pub fn with_config(mut self, config: Arc) -> Self { 76 | self.config = Some(config); 77 | self 78 | } 79 | 80 | /// Set the executor type (defaults to DataFusion) 81 | pub fn with_executor_type(mut self, executor_type: ExecutorType) -> Self { 82 | self.executor_type = executor_type; 83 | self 84 | } 85 | 86 | /// Set the catalog 87 | pub fn with_catalog(mut self, catalog: Arc) -> Self { 88 | self.catalog = Some(catalog); 89 | self 90 | } 91 | 92 | /// Set the metrics registry (optional, defaults to NoopMetricsRegistry) 93 | pub fn with_registry(mut self, registry: BoxedRegistry) -> Self { 94 | self.registry = registry; 95 | self 96 | } 97 | 98 | pub fn with_table_ident(mut self, table_ident: TableIdent) -> Self { 99 | self.table_ident = Some(table_ident); 100 | self 101 | } 102 | 103 | pub fn with_compaction_type(mut self, compaction_type: CompactionType) -> Self { 104 | self.compaction_type = Some(compaction_type); 105 | self 106 | } 107 | 108 | pub fn with_catalog_name(mut self, catalog_name: String) -> Self { 109 | self.catalog_name = Some(catalog_name); 110 | self 111 | } 112 | 113 | pub fn with_retry_config( 114 | mut self, 115 | retry_config: RewriteDataFilesCommitManagerRetryConfig, 116 | ) -> Self { 117 | self.commit_retry_config = retry_config; 118 | self 119 | } 120 | 121 | /// Build the Compaction instance 122 | pub async fn build(self) -> Result { 123 | let config = self.config.ok_or_else(|| { 124 | crate::error::CompactionError::Execution("CompactionConfig is required".to_string()) 125 | })?; 126 | 127 | let catalog = self.catalog.ok_or_else(|| { 128 | crate::error::CompactionError::Execution("Catalog is required".to_string()) 129 | })?; 130 | 131 | let table_ident = self.table_ident.ok_or_else(|| { 132 | crate::error::CompactionError::Execution("TableIdent is required".to_string()) 133 | })?; 134 | 135 | let compaction_type = self.compaction_type.unwrap_or(CompactionType::Full); 136 | 137 | if !catalog.table_exists(&table_ident).await? { 138 | return Err(crate::error::CompactionError::Execution( 139 | "Table does not exist".to_string(), 140 | )); 141 | } 142 | 143 | let executor = create_compaction_executor(self.executor_type); 144 | 145 | let metrics = Arc::new(Metrics::new(self.registry)); 146 | 147 | let catalog_name = self.catalog_name.unwrap_or_default(); 148 | 149 | let commit_retry_config = self.commit_retry_config; 150 | 151 | Ok(Compaction { 152 | config, 153 | executor, 154 | catalog, 155 | metrics, 156 | table_ident, 157 | compaction_type, 158 | catalog_name, 159 | commit_retry_config, 160 | }) 161 | } 162 | } 163 | 164 | impl Default for CompactionBuilder { 165 | fn default() -> Self { 166 | Self::new() 167 | } 168 | } 169 | 170 | /// A Proxy for the compaction process, which handles the configuration, executor, and catalog. 171 | pub struct Compaction { 172 | pub config: Arc, 173 | pub executor: Box, 174 | pub catalog: Arc, 175 | pub metrics: Arc, 176 | pub table_ident: TableIdent, 177 | pub compaction_type: CompactionType, 178 | pub catalog_name: String, 179 | 180 | pub commit_retry_config: RewriteDataFilesCommitManagerRetryConfig, 181 | } 182 | 183 | impl Compaction { 184 | /// Create a new CompactionBuilder for flexible configuration 185 | pub fn builder() -> CompactionBuilder { 186 | CompactionBuilder::new() 187 | } 188 | 189 | pub async fn compact(&self) -> Result { 190 | match self.compaction_type { 191 | CompactionType::Full => self.full_compact().await, 192 | } 193 | } 194 | 195 | async fn full_compact(&self) -> Result { 196 | let table_label: std::borrow::Cow<'static, str> = self.table_ident.to_string().into(); 197 | let catalog_name_label: std::borrow::Cow<'static, str> = self.catalog_name.clone().into(); 198 | let label_vec: [std::borrow::Cow<'static, str>; 2] = [catalog_name_label, table_label]; 199 | 200 | let now = std::time::Instant::now(); 201 | 202 | let table = self.catalog.load_table(&self.table_ident).await?; 203 | let (data_files, delete_files) = get_old_files_from_table(table.clone()).await?; 204 | let input_file_scan_tasks = get_tasks_from_table(table.clone()).await?; 205 | 206 | let file_io = table.file_io().clone(); 207 | let schema = table.metadata().current_schema(); 208 | let default_location_generator = 209 | DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); 210 | let rewrite_files_request = RewriteFilesRequest { 211 | file_io, 212 | schema: schema.clone(), 213 | input_file_scan_tasks, 214 | config: self.config.clone(), 215 | dir_path: default_location_generator.dir_path, 216 | partition_spec: table.metadata().default_partition_spec().clone(), 217 | }; 218 | let RewriteFilesResponse { 219 | data_files: output_data_files, 220 | stat, 221 | } = match self.executor.rewrite_files(rewrite_files_request).await { 222 | Ok(response) => response, 223 | Err(e) => { 224 | self.metrics 225 | .compaction_executor_error_counter 226 | .counter(&label_vec) 227 | .increase(1); 228 | return Err(e); 229 | } 230 | }; 231 | 232 | let commit_manager = RewriteDataFilesCommitManager::new( 233 | self.commit_retry_config.clone(), 234 | self.catalog.clone(), 235 | self.table_ident.clone(), 236 | table.metadata().current_snapshot_id().unwrap(), 237 | true, 238 | self.catalog_name.clone(), 239 | self.metrics.clone(), 240 | ); 241 | 242 | let commit_now = std::time::Instant::now(); 243 | commit_manager 244 | .rewrite_files( 245 | output_data_files, 246 | data_files.into_iter().chain(delete_files.into_iter()), 247 | ) 248 | .await?; 249 | 250 | self.metrics 251 | .compaction_commit_duration 252 | .histogram(&label_vec) 253 | .record(commit_now.elapsed().as_secs_f64()); 254 | 255 | self.metrics 256 | .compaction_duration 257 | .histogram(&label_vec) 258 | .record(now.elapsed().as_secs_f64()); 259 | 260 | self.metrics 261 | .compaction_rewritten_bytes 262 | .counter(&label_vec) 263 | .increase(stat.rewritten_bytes); 264 | 265 | self.metrics 266 | .compaction_rewritten_files_count 267 | .counter(&label_vec) 268 | .increase(stat.rewritten_files_count as u64); 269 | 270 | self.metrics 271 | .compaction_added_files_count 272 | .counter(&label_vec) 273 | .increase(stat.added_files_count as u64); 274 | 275 | self.metrics 276 | .compaction_failed_data_files_count 277 | .counter(&label_vec) 278 | .increase(stat.failed_data_files_count as u64); 279 | 280 | Ok(RewriteFilesStat { 281 | rewritten_files_count: stat.rewritten_files_count, 282 | added_files_count: stat.added_files_count, 283 | rewritten_bytes: stat.rewritten_bytes, 284 | failed_data_files_count: stat.failed_data_files_count, 285 | }) 286 | } 287 | 288 | pub async fn expire_snapshot(&self, table_ident: TableIdent) -> Result<()> { 289 | let table = self.catalog.load_table(&table_ident).await?; 290 | let txn = Transaction::new(&table); 291 | let txn = txn.expire_snapshot().apply().await?; 292 | txn.commit(self.catalog.as_ref()).await?; 293 | Ok(()) 294 | } 295 | } 296 | 297 | async fn get_old_files_from_table(table: Table) -> Result<(Vec, Vec)> { 298 | let manifest_list = table 299 | .metadata() 300 | .current_snapshot() 301 | .unwrap() 302 | .load_manifest_list(table.file_io(), table.metadata()) 303 | .await 304 | .unwrap(); 305 | 306 | let mut data_file = vec![]; 307 | let mut delete_file = vec![]; 308 | for manifest_file in manifest_list.entries() { 309 | let a = manifest_file.load_manifest(table.file_io()).await.unwrap(); 310 | let (entry, _) = a.into_parts(); 311 | for i in entry { 312 | match i.content_type() { 313 | iceberg::spec::DataContentType::Data => { 314 | data_file.push(i.data_file().clone()); 315 | } 316 | iceberg::spec::DataContentType::EqualityDeletes => { 317 | delete_file.push(i.data_file().clone()); 318 | } 319 | iceberg::spec::DataContentType::PositionDeletes => { 320 | delete_file.push(i.data_file().clone()); 321 | } 322 | } 323 | } 324 | } 325 | Ok((data_file, delete_file)) 326 | } 327 | 328 | async fn get_tasks_from_table(table: Table) -> Result { 329 | let snapshot_id = table.metadata().current_snapshot_id().unwrap(); 330 | 331 | let scan = table 332 | .scan() 333 | .snapshot_id(snapshot_id) 334 | .with_delete_file_processing_enabled(true) 335 | .build()?; 336 | let file_scan_stream = scan.plan_files().await?; 337 | 338 | let mut position_delete_files = HashMap::new(); 339 | let mut data_files = vec![]; 340 | let mut equality_delete_files = HashMap::new(); 341 | 342 | #[for_await] 343 | for task in file_scan_stream { 344 | let task: FileScanTask = task?; 345 | match task.data_file_content { 346 | iceberg::spec::DataContentType::Data => { 347 | for delete_task in task.deletes.iter() { 348 | match &delete_task.data_file_content { 349 | iceberg::spec::DataContentType::PositionDeletes => { 350 | let mut delete_task = delete_task.clone(); 351 | delete_task.project_field_ids = vec![]; 352 | position_delete_files 353 | .insert(delete_task.data_file_path.clone(), delete_task); 354 | } 355 | iceberg::spec::DataContentType::EqualityDeletes => { 356 | let mut delete_task = delete_task.clone(); 357 | delete_task.project_field_ids = delete_task.equality_ids.clone(); 358 | equality_delete_files 359 | .insert(delete_task.data_file_path.clone(), delete_task); 360 | } 361 | _ => { 362 | unreachable!() 363 | } 364 | } 365 | } 366 | data_files.push(task); 367 | } 368 | _ => { 369 | unreachable!() 370 | } 371 | } 372 | } 373 | Ok(InputFileScanTasks { 374 | data_files, 375 | position_delete_files: position_delete_files.into_values().collect(), 376 | equality_delete_files: equality_delete_files.into_values().collect(), 377 | }) 378 | } 379 | 380 | /// Configuration for the commit manager, including retry strategies. 381 | #[derive(Debug, Clone)] 382 | pub struct RewriteDataFilesCommitManagerRetryConfig { 383 | pub max_retries: u32, // This can be used to configure the backon strategy 384 | pub retry_initial_delay: Duration, // For exponential backoff 385 | pub retry_max_delay: Duration, // For exponential backoff 386 | } 387 | 388 | impl Default for RewriteDataFilesCommitManagerRetryConfig { 389 | fn default() -> Self { 390 | Self { 391 | max_retries: 3, 392 | retry_initial_delay: Duration::from_secs(1), 393 | retry_max_delay: Duration::from_secs(10), 394 | } 395 | } 396 | } 397 | 398 | // Manages the commit process with retries 399 | pub struct RewriteDataFilesCommitManager { 400 | config: RewriteDataFilesCommitManagerRetryConfig, 401 | catalog: Arc, 402 | table_ident: TableIdent, 403 | starting_snapshot_id: i64, // The snapshot ID to start from, used for consistency 404 | use_starting_sequence_number: bool, // Whether to use the starting sequence number for commits 405 | 406 | catalog_name: String, // Catalog name for metrics 407 | metrics: Arc, // Metrics for tracking commit operations 408 | } 409 | 410 | /// Manages the commit process with retries 411 | impl RewriteDataFilesCommitManager { 412 | pub fn new( 413 | config: RewriteDataFilesCommitManagerRetryConfig, 414 | catalog: Arc, 415 | table_ident: TableIdent, 416 | starting_snapshot_id: i64, 417 | use_starting_sequence_number: bool, 418 | catalog_name: String, 419 | metrics: Arc, 420 | ) -> Self { 421 | Self { 422 | config, 423 | catalog, 424 | table_ident, 425 | starting_snapshot_id, 426 | use_starting_sequence_number, 427 | catalog_name, 428 | metrics, 429 | } 430 | } 431 | 432 | /// Rewrites files in the table, handling retries and errors. 433 | pub async fn rewrite_files( 434 | &self, 435 | data_files: impl IntoIterator, 436 | delete_files: impl IntoIterator, 437 | ) -> Result<()> { 438 | let data_files: Vec = data_files.into_iter().collect(); 439 | let delete_files: Vec = delete_files.into_iter().collect(); 440 | let operation = || { 441 | let catalog = self.catalog.clone(); 442 | let table_ident = self.table_ident.clone(); 443 | let data_files = data_files.clone(); 444 | let delete_files = delete_files.clone(); 445 | let use_starting_sequence_number = self.use_starting_sequence_number; 446 | let starting_snapshot_id = self.starting_snapshot_id; 447 | let metrics = self.metrics.clone(); 448 | 449 | let table_label: std::borrow::Cow<'static, str> = self.table_ident.to_string().into(); 450 | let catalog_name_label: std::borrow::Cow<'static, str> = 451 | self.catalog_name.clone().into(); 452 | let label_vec: [std::borrow::Cow<'static, str>; 2] = [catalog_name_label, table_label]; 453 | 454 | async move { 455 | // reload the table to get the latest state 456 | let table = catalog.load_table(&table_ident).await?; 457 | let txn = Transaction::new(&table); 458 | 459 | // TODO: support validation of data files and delete files with starting snapshot before applying the rewrite 460 | let rewrite_action = if use_starting_sequence_number { 461 | // TODO: avoid retry if the snapshot_id is not found 462 | if let Some(snapshot) = table.metadata().snapshot_by_id(starting_snapshot_id) { 463 | txn.rewrite_files(None, vec![])? 464 | .add_data_files(data_files)? 465 | .delete_files(delete_files)? 466 | .new_data_file_sequence_number(snapshot.sequence_number())? 467 | } else { 468 | return Err(iceberg::Error::new( 469 | ErrorKind::Unexpected, 470 | format!( 471 | "No snapshot found with the given snapshot_id {}", 472 | starting_snapshot_id 473 | ), 474 | )); 475 | } 476 | } else { 477 | txn.rewrite_files(None, vec![])? 478 | .add_data_files(data_files)? 479 | .delete_files(delete_files)? 480 | }; 481 | 482 | let txn = rewrite_action.apply().await?; 483 | match txn.commit(catalog.as_ref()).await { 484 | Ok(_) => { 485 | // Update metrics after a successful commit 486 | metrics 487 | .compaction_commit_counter 488 | .counter(&label_vec) 489 | .increase(1); 490 | Ok(()) 491 | } 492 | Err(commit_err) => { 493 | metrics 494 | .compaction_commit_failed_counter 495 | .counter(&label_vec) 496 | .increase(1); 497 | 498 | tracing::error!( 499 | "Commit attempt failed for table '{}': {:?}. Will retry if applicable.", 500 | table_ident, 501 | commit_err 502 | ); 503 | Err(commit_err) 504 | } 505 | } 506 | } 507 | }; 508 | 509 | let retry_strategy = ExponentialBuilder::default() 510 | .with_min_delay(self.config.retry_initial_delay) 511 | .with_max_delay(self.config.retry_max_delay) 512 | .with_max_times(self.config.max_retries as usize); 513 | 514 | operation 515 | .retry(retry_strategy) 516 | .when(|e| { 517 | matches!(e.kind(), iceberg::ErrorKind::DataInvalid) 518 | || matches!(e.kind(), iceberg::ErrorKind::Unexpected) 519 | }) 520 | .notify(|e, d| { 521 | // Notify the user about the error 522 | // TODO: add metrics 523 | tracing::info!("Retrying Compaction failed {:?} after {:?}", e, d); 524 | }) 525 | .await 526 | .map_err(|e: iceberg::Error| CompactionError::from(e)) // Convert backon::Error to your CompactionError 527 | } 528 | } 529 | -------------------------------------------------------------------------------- /core/src/config/mod.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use serde::Deserialize; 18 | use serde_with::serde_as; 19 | 20 | const DEFAULT_PREFIX: &str = "10"; 21 | 22 | #[serde_as] 23 | #[derive(Debug, Deserialize)] 24 | pub struct CompactionConfig { 25 | #[serde(default = "default_batch_parallelism")] 26 | pub batch_parallelism: usize, 27 | #[serde(default = "default_target_partitions")] 28 | pub target_partitions: usize, 29 | #[serde(default = "default_data_file_prefix")] 30 | pub data_file_prefix: String, 31 | #[serde(default = "default_target_file_size")] 32 | pub target_file_size: usize, 33 | } 34 | fn default_batch_parallelism() -> usize { 35 | 4 36 | } 37 | fn default_target_partitions() -> usize { 38 | 4 39 | } 40 | fn default_data_file_prefix() -> String { 41 | DEFAULT_PREFIX.to_string() 42 | } 43 | fn default_target_file_size() -> usize { 44 | 1024 * 1024 * 1024 // 1 GB 45 | } 46 | -------------------------------------------------------------------------------- /core/src/error/mod.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use thiserror::Error; 18 | 19 | #[derive(Error, Debug)] 20 | pub enum CompactionError { 21 | #[error("IO error: {0}")] 22 | Io(#[from] std::io::Error), 23 | 24 | #[error("Invalid configuration: {0}")] 25 | Config(String), 26 | 27 | #[error("Execution failed: {0}")] 28 | Execution(String), 29 | 30 | #[error("Iceberg error: {0}")] 31 | Iceberg(#[from] iceberg::Error), 32 | 33 | #[error("DataFusion error: {0}")] 34 | DataFusion(#[from] datafusion::error::DataFusionError), 35 | } 36 | 37 | pub type Result = std::result::Result; 38 | -------------------------------------------------------------------------------- /core/src/executor/datafusion/datafusion_processor.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 IC 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use std::sync::Arc; 18 | 19 | use crate::error::{CompactionError, Result}; 20 | use datafusion::{ 21 | execution::SendableRecordBatchStream, 22 | physical_plan::{ 23 | ExecutionPlan, ExecutionPlanProperties, Partitioning, execute_stream_partitioned, 24 | repartition::RepartitionExec, 25 | }, 26 | prelude::SessionContext, 27 | }; 28 | use iceberg::{ 29 | arrow::schema_to_arrow_schema, 30 | io::FileIO, 31 | scan::FileScanTask, 32 | spec::{NestedField, PrimitiveType, Schema, Type}, 33 | }; 34 | 35 | use super::file_scan_task_table_provider::IcebergFileScanTaskTableProvider; 36 | 37 | pub const SYS_HIDDEN_SEQ_NUM: &str = "sys_hidden_seq_num"; 38 | pub const SYS_HIDDEN_FILE_PATH: &str = "sys_hidden_file_path"; 39 | pub const SYS_HIDDEN_POS: &str = "sys_hidden_pos"; 40 | const SYS_HIDDEN_COLS: [&str; 3] = [SYS_HIDDEN_SEQ_NUM, SYS_HIDDEN_FILE_PATH, SYS_HIDDEN_POS]; 41 | 42 | const DATA_FILE_TABLE: &str = "data_file_table"; 43 | const POSITION_DELETE_TABLE: &str = "position_delete_table"; 44 | const EQUALITY_DELETE_TABLE: &str = "equality_delete_table"; 45 | 46 | pub struct DatafusionProcessor { 47 | datafusion_task_ctx: DataFusionTaskContext, 48 | table_register: DatafusionTableRegister, 49 | batch_parallelism: usize, 50 | target_partitions: usize, 51 | ctx: Arc, 52 | } 53 | 54 | impl DatafusionProcessor { 55 | pub fn new( 56 | ctx: Arc, 57 | datafusion_task_ctx: DataFusionTaskContext, 58 | batch_parallelism: usize, 59 | target_partitions: usize, 60 | file_io: FileIO, 61 | ) -> Self { 62 | let table_register = DatafusionTableRegister::new(file_io, ctx.clone()); 63 | Self { 64 | datafusion_task_ctx, 65 | table_register, 66 | batch_parallelism, 67 | target_partitions, 68 | ctx, 69 | } 70 | } 71 | 72 | pub fn register_tables(&mut self) -> Result<()> { 73 | if let Some(datafile_schema) = self.datafusion_task_ctx.data_file_schema.take() { 74 | self.table_register.register_data_table_provider( 75 | &datafile_schema, 76 | self.datafusion_task_ctx.data_files.take().unwrap(), 77 | DATA_FILE_TABLE, 78 | self.datafusion_task_ctx.need_seq_num(), 79 | self.datafusion_task_ctx.need_file_path_and_pos(), 80 | self.batch_parallelism, 81 | )?; 82 | } 83 | 84 | if let Some(position_delete_schema) = self.datafusion_task_ctx.position_delete_schema.take() 85 | { 86 | self.table_register.register_delete_table_provider( 87 | &position_delete_schema, 88 | self.datafusion_task_ctx 89 | .position_delete_files 90 | .take() 91 | .unwrap(), 92 | POSITION_DELETE_TABLE, 93 | self.batch_parallelism, 94 | )?; 95 | } 96 | 97 | if let Some(equality_delete_metadatas) = 98 | self.datafusion_task_ctx.equality_delete_metadatas.take() 99 | { 100 | for EqualityDeleteMetadata { 101 | equality_delete_schema, 102 | equality_delete_table_name, 103 | file_scan_tasks, 104 | } in equality_delete_metadatas 105 | { 106 | self.table_register.register_delete_table_provider( 107 | &equality_delete_schema, 108 | file_scan_tasks, 109 | &equality_delete_table_name, 110 | self.batch_parallelism, 111 | )?; 112 | } 113 | } 114 | Ok(()) 115 | } 116 | 117 | pub async fn execute(&mut self) -> Result<(Vec, Schema)> { 118 | self.register_tables()?; 119 | let df = self.ctx.sql(&self.datafusion_task_ctx.exec_sql).await?; 120 | let physical_plan = df.create_physical_plan().await?; 121 | let batchs = 122 | if physical_plan.output_partitioning().partition_count() != self.target_partitions { 123 | let physical_plan: Arc = 124 | Arc::new(RepartitionExec::try_new( 125 | physical_plan, 126 | Partitioning::RoundRobinBatch(self.target_partitions), 127 | )?); 128 | execute_stream_partitioned(physical_plan, self.ctx.task_ctx())? 129 | } else { 130 | execute_stream_partitioned(physical_plan, self.ctx.task_ctx())? 131 | }; 132 | Ok(( 133 | batchs, 134 | self.datafusion_task_ctx.input_schema.take().unwrap(), 135 | )) 136 | } 137 | } 138 | 139 | pub struct DatafusionTableRegister { 140 | file_io: FileIO, 141 | ctx: Arc, 142 | } 143 | 144 | impl DatafusionTableRegister { 145 | pub fn new(file_io: FileIO, ctx: Arc) -> Self { 146 | DatafusionTableRegister { file_io, ctx } 147 | } 148 | 149 | pub fn register_data_table_provider( 150 | &self, 151 | schema: &Schema, 152 | file_scan_tasks: Vec, 153 | table_name: &str, 154 | need_seq_num: bool, 155 | need_file_path_and_pos: bool, 156 | batch_parallelism: usize, 157 | ) -> Result<()> { 158 | self.register_table_provider_impl( 159 | schema, 160 | file_scan_tasks, 161 | table_name, 162 | need_seq_num, 163 | need_file_path_and_pos, 164 | batch_parallelism, 165 | ) 166 | } 167 | 168 | pub fn register_delete_table_provider( 169 | &self, 170 | schema: &Schema, 171 | file_scan_tasks: Vec, 172 | table_name: &str, 173 | batch_parallelism: usize, 174 | ) -> Result<()> { 175 | self.register_table_provider_impl( 176 | schema, 177 | file_scan_tasks, 178 | table_name, 179 | false, 180 | false, 181 | batch_parallelism, 182 | ) 183 | } 184 | 185 | #[allow(clippy::too_many_arguments)] 186 | fn register_table_provider_impl( 187 | &self, 188 | schema: &Schema, 189 | file_scan_tasks: Vec, 190 | table_name: &str, 191 | need_seq_num: bool, 192 | need_file_path_and_pos: bool, 193 | batch_parallelism: usize, 194 | ) -> Result<()> { 195 | let schema = schema_to_arrow_schema(schema)?; 196 | let data_file_table_provider = IcebergFileScanTaskTableProvider::new( 197 | file_scan_tasks, 198 | Arc::new(schema), 199 | self.file_io.clone(), 200 | need_seq_num, 201 | need_file_path_and_pos, 202 | batch_parallelism, 203 | ); 204 | 205 | self.ctx 206 | .register_table(table_name, Arc::new(data_file_table_provider)) 207 | .unwrap(); 208 | Ok(()) 209 | } 210 | } 211 | 212 | /// SQL Builder for generating merge-on-read SQL queries 213 | struct SqlBuilder<'a> { 214 | /// Column names to be projected in the query 215 | project_names: &'a Vec, 216 | 217 | /// Position delete table name 218 | position_delete_table_name: Option, 219 | 220 | /// Data file table name 221 | data_file_table_name: Option, 222 | 223 | /// Flag indicating if file path and position columns are needed 224 | equality_delete_metadatas: &'a Vec, 225 | 226 | /// Flag indicating if position delete files are needed 227 | need_file_path_and_pos: bool, 228 | } 229 | 230 | impl<'a> SqlBuilder<'a> { 231 | /// Creates a new SQL Builder with the specified parameters 232 | fn new( 233 | project_names: &'a Vec, 234 | position_delete_table_name: Option, 235 | data_file_table_name: Option, 236 | equality_delete_metadatas: &'a Vec, 237 | need_file_path_and_pos: bool, 238 | ) -> Self { 239 | Self { 240 | project_names, 241 | position_delete_table_name, 242 | data_file_table_name, 243 | equality_delete_metadatas, 244 | need_file_path_and_pos, 245 | } 246 | } 247 | 248 | /// Builds a merge-on-read SQL query 249 | /// 250 | /// This method constructs a SQL query that: 251 | /// 1. Selects the specified columns from the data file table 252 | /// 2. Optionally joins with position delete files to exclude deleted rows 253 | /// 3. Optionally joins with equality delete files to exclude rows based on equality conditions 254 | pub fn build_merge_on_read_sql(self) -> Result { 255 | let data_file_table_name = self.data_file_table_name.as_ref().ok_or_else(|| { 256 | CompactionError::Config("Data file table name is not provided".to_string()) 257 | })?; 258 | // Start with a basic SELECT query from the data file table 259 | let mut sql = format!( 260 | "SELECT {} FROM {}", 261 | self.project_names.join(","), 262 | data_file_table_name 263 | ); 264 | 265 | // Add position delete join if needed 266 | // This excludes rows that have been deleted by position 267 | if self.need_file_path_and_pos { 268 | let position_delete_table_name = 269 | self.position_delete_table_name.as_ref().ok_or_else(|| { 270 | CompactionError::Config( 271 | "Position delete table name is not provided".to_string(), 272 | ) 273 | })?; 274 | sql.push_str(&format!( 275 | " LEFT ANTI JOIN {position_delete_table_name} ON {data_file_table_name}.{SYS_HIDDEN_FILE_PATH} = {position_delete_table_name}.{SYS_HIDDEN_FILE_PATH} AND {data_file_table_name}.{SYS_HIDDEN_POS} = {position_delete_table_name}.{SYS_HIDDEN_POS}", 276 | )); 277 | } 278 | 279 | // Add equality delete join if needed 280 | // This excludes rows that match the equality conditions in the delete files 281 | if !self.equality_delete_metadatas.is_empty() { 282 | for metadata in self.equality_delete_metadatas { 283 | // LEFT ANTI JOIN ON equality delete table 284 | sql.push_str(&format!( 285 | " LEFT ANTI JOIN {} ON {}", 286 | metadata.equality_delete_table_name, 287 | metadata 288 | .equality_delete_join_names() 289 | .iter() 290 | .map(|name| format!( 291 | "{data_file_table_name}.{name} = {}.{name}", 292 | metadata.equality_delete_table_name 293 | )) 294 | .collect::>() 295 | .join(" AND ") 296 | )); 297 | 298 | // Add sequence number comparison if needed 299 | // This ensures that only newer deletes are applied 300 | sql.push_str(&format!( 301 | " AND {data_file_table_name}.{SYS_HIDDEN_SEQ_NUM} < {}.{SYS_HIDDEN_SEQ_NUM}", 302 | metadata.equality_delete_table_name 303 | )); 304 | } 305 | } 306 | 307 | Ok(sql) 308 | } 309 | } 310 | 311 | pub struct DataFusionTaskContext { 312 | pub(crate) data_file_schema: Option, 313 | pub(crate) input_schema: Option, 314 | pub(crate) data_files: Option>, 315 | pub(crate) position_delete_files: Option>, 316 | pub(crate) equality_delete_files: Option>, 317 | pub(crate) position_delete_schema: Option, 318 | pub(crate) equality_delete_metadatas: Option>, 319 | pub(crate) exec_sql: String, 320 | } 321 | 322 | pub struct DataFusionTaskContextBuilder { 323 | schema: Arc, 324 | data_files: Vec, 325 | position_delete_files: Vec, 326 | equality_delete_files: Vec, 327 | } 328 | 329 | impl DataFusionTaskContextBuilder { 330 | pub fn with_schema(mut self, schema: Arc) -> Self { 331 | self.schema = schema; 332 | self 333 | } 334 | 335 | pub fn with_datafile(mut self, data_files: Vec) -> Self { 336 | self.data_files = data_files; 337 | self 338 | } 339 | 340 | pub fn with_position_delete_files(mut self, position_delete_files: Vec) -> Self { 341 | self.position_delete_files = position_delete_files; 342 | self 343 | } 344 | 345 | pub fn with_equality_delete_files(mut self, equality_delete_files: Vec) -> Self { 346 | self.equality_delete_files = equality_delete_files; 347 | self 348 | } 349 | 350 | fn build_position_schema() -> Result { 351 | let position_delete_schema = Schema::builder() 352 | .with_fields(vec![ 353 | Arc::new(NestedField::new( 354 | 1, 355 | SYS_HIDDEN_FILE_PATH, 356 | Type::Primitive(PrimitiveType::String), 357 | true, 358 | )), 359 | Arc::new(NestedField::new( 360 | 2, 361 | SYS_HIDDEN_POS, 362 | Type::Primitive(PrimitiveType::Long), 363 | true, 364 | )), 365 | ]) 366 | .build()?; 367 | Ok(position_delete_schema) 368 | } 369 | 370 | // build data fusion task context 371 | pub fn build_merge_on_read(self) -> Result { 372 | let mut highest_field_id = self.schema.highest_field_id(); 373 | // Build scheam for position delete file, file_path + pos 374 | let position_delete_schema = Self::build_position_schema()?; 375 | // Build schema for equality delete file, equality_ids + seq_num 376 | let mut equality_ids: Option> = None; 377 | let mut equality_delete_metadatas = Vec::new(); 378 | let mut table_idx = 0; 379 | for task in &self.equality_delete_files { 380 | if equality_ids 381 | .as_ref() 382 | .is_none_or(|ids| !ids.eq(&task.equality_ids)) 383 | { 384 | // If ids are different or not assigned, create a new metadata 385 | let equality_delete_schema = 386 | self.build_equality_delete_schema(&task.equality_ids, &mut highest_field_id)?; 387 | let equality_delete_table_name = format!("{}_{}", EQUALITY_DELETE_TABLE, table_idx); 388 | equality_delete_metadatas.push(EqualityDeleteMetadata::new( 389 | equality_delete_schema, 390 | equality_delete_table_name, 391 | )); 392 | equality_ids = Some(task.equality_ids.clone()); 393 | table_idx += 1; 394 | } 395 | 396 | // Add the file scan task to the last metadata 397 | if let Some(last_metadata) = equality_delete_metadatas.last_mut() { 398 | last_metadata.add_file_scan_task(task.clone()); 399 | } 400 | } 401 | 402 | let need_file_path_and_pos = !self.position_delete_files.is_empty(); 403 | let need_seq_num = !equality_delete_metadatas.is_empty(); 404 | 405 | // Build schema for data file, old schema + seq_num + file_path + pos 406 | let project_names: Vec<_> = self 407 | .schema 408 | .as_struct() 409 | .fields() 410 | .iter() 411 | .map(|i| i.name.clone()) 412 | .collect(); 413 | let highest_field_id = self.schema.highest_field_id(); 414 | let mut add_schema_fields = vec![]; 415 | // add sequence number column if needed 416 | if need_seq_num { 417 | add_schema_fields.push(Arc::new(NestedField::new( 418 | highest_field_id + 1, 419 | SYS_HIDDEN_SEQ_NUM, 420 | Type::Primitive(PrimitiveType::Long), 421 | true, 422 | ))); 423 | } 424 | // add file path and position column if needed 425 | if need_file_path_and_pos { 426 | add_schema_fields.push(Arc::new(NestedField::new( 427 | highest_field_id + 2, 428 | SYS_HIDDEN_FILE_PATH, 429 | Type::Primitive(PrimitiveType::String), 430 | true, 431 | ))); 432 | add_schema_fields.push(Arc::new(NestedField::new( 433 | highest_field_id + 3, 434 | SYS_HIDDEN_POS, 435 | Type::Primitive(PrimitiveType::Long), 436 | true, 437 | ))); 438 | } 439 | // data file schema is old schema + seq_num + file_path + pos. used for data file table provider 440 | let data_file_schema = self 441 | .schema 442 | .as_ref() 443 | .clone() 444 | .into_builder() 445 | .with_fields(add_schema_fields) 446 | .build()?; 447 | // input schema is old schema. used for data file writer 448 | let input_schema = self.schema.as_ref().clone(); 449 | 450 | let sql_builder = SqlBuilder::new( 451 | &project_names, 452 | Some(POSITION_DELETE_TABLE.to_owned()), 453 | Some(DATA_FILE_TABLE.to_owned()), 454 | &equality_delete_metadatas, 455 | need_file_path_and_pos, 456 | ); 457 | let exec_sql = sql_builder.build_merge_on_read_sql()?; 458 | 459 | Ok(DataFusionTaskContext { 460 | data_file_schema: Some(data_file_schema), 461 | input_schema: Some(input_schema), 462 | data_files: Some(self.data_files), 463 | position_delete_files: Some(self.position_delete_files), 464 | equality_delete_files: Some(self.equality_delete_files), 465 | position_delete_schema: if need_file_path_and_pos { 466 | Some(position_delete_schema) 467 | } else { 468 | None 469 | }, 470 | equality_delete_metadatas: if need_seq_num { 471 | Some(equality_delete_metadatas) 472 | } else { 473 | None 474 | }, 475 | exec_sql, 476 | }) 477 | } 478 | 479 | /// Builds an equality delete schema based on the given equality_ids 480 | fn build_equality_delete_schema( 481 | &self, 482 | equality_ids: &[i32], 483 | highest_field_id: &mut i32, 484 | ) -> Result { 485 | let mut equality_delete_fields = Vec::with_capacity(equality_ids.len()); 486 | for id in equality_ids { 487 | let field = self 488 | .schema 489 | .field_by_id(*id) 490 | .ok_or_else(|| CompactionError::Config("equality_ids not found".to_owned()))?; 491 | equality_delete_fields.push(field.clone()); 492 | } 493 | *highest_field_id += 1; 494 | equality_delete_fields.push(Arc::new(NestedField::new( 495 | *highest_field_id, 496 | SYS_HIDDEN_SEQ_NUM, 497 | Type::Primitive(PrimitiveType::Long), 498 | true, 499 | ))); 500 | 501 | Schema::builder() 502 | .with_fields(equality_delete_fields) 503 | .build() 504 | .map_err(CompactionError::Iceberg) 505 | } 506 | } 507 | 508 | impl DataFusionTaskContext { 509 | pub fn builder() -> Result { 510 | Ok(DataFusionTaskContextBuilder { 511 | schema: Arc::new(Schema::builder().build()?), 512 | data_files: vec![], 513 | position_delete_files: vec![], 514 | equality_delete_files: vec![], 515 | }) 516 | } 517 | 518 | pub fn need_file_path_and_pos(&self) -> bool { 519 | self.position_delete_files 520 | .as_ref() 521 | .is_some_and(|v| !v.is_empty()) 522 | } 523 | 524 | pub fn need_seq_num(&self) -> bool { 525 | self.equality_delete_files 526 | .as_ref() 527 | .is_some_and(|v| !v.is_empty()) 528 | } 529 | } 530 | 531 | /// Metadata for equality delete files 532 | #[derive(Debug, Clone)] 533 | pub(crate) struct EqualityDeleteMetadata { 534 | pub(crate) equality_delete_schema: Schema, 535 | pub(crate) equality_delete_table_name: String, 536 | pub(crate) file_scan_tasks: Vec, 537 | } 538 | 539 | impl EqualityDeleteMetadata { 540 | pub fn new(equality_delete_schema: Schema, equality_delete_table_name: String) -> Self { 541 | Self { 542 | equality_delete_schema, 543 | equality_delete_table_name, 544 | file_scan_tasks: Vec::new(), 545 | } 546 | } 547 | 548 | pub fn equality_delete_join_names(&self) -> Vec<&str> { 549 | self.equality_delete_schema 550 | .as_struct() 551 | .fields() 552 | .iter() 553 | .map(|i| i.name.as_str()) 554 | .filter(|name| !SYS_HIDDEN_COLS.contains(name)) 555 | .collect() 556 | } 557 | 558 | pub fn add_file_scan_task(&mut self, file_scan_task: FileScanTask) { 559 | self.file_scan_tasks.push(file_scan_task); 560 | } 561 | } 562 | 563 | #[cfg(test)] 564 | mod tests { 565 | use super::*; 566 | use iceberg::spec::{NestedField, PrimitiveType, Schema, Type}; 567 | use std::sync::Arc; 568 | 569 | /// Test building SQL with no delete files 570 | #[test] 571 | fn test_build_merge_on_read_sql_no_deletes() { 572 | let project_names = vec!["id".to_owned(), "name".to_owned()]; 573 | let equality_join_names = Vec::new(); 574 | 575 | let builder = SqlBuilder::new( 576 | &project_names, 577 | Some(POSITION_DELETE_TABLE.to_owned()), 578 | Some(DATA_FILE_TABLE.to_owned()), 579 | &equality_join_names, 580 | false, 581 | ); 582 | assert_eq!( 583 | builder.build_merge_on_read_sql().unwrap(), 584 | format!( 585 | "SELECT {} FROM {}", 586 | project_names.join(","), 587 | DATA_FILE_TABLE 588 | ) 589 | ); 590 | } 591 | 592 | /// Test building SQL with position delete files 593 | #[test] 594 | fn test_build_merge_on_read_sql_with_position_deletes() { 595 | let project_names = vec!["id".to_owned(), "name".to_owned()]; 596 | let equality_join_names = Vec::new(); 597 | 598 | let builder = SqlBuilder::new( 599 | &project_names, 600 | Some(POSITION_DELETE_TABLE.to_owned()), 601 | Some(DATA_FILE_TABLE.to_owned()), 602 | &equality_join_names, 603 | true, 604 | ); 605 | let sql = builder.build_merge_on_read_sql().unwrap(); 606 | 607 | assert!(sql.contains(&format!( 608 | "LEFT ANTI JOIN {POSITION_DELETE_TABLE} ON {DATA_FILE_TABLE}", 609 | ))); 610 | assert!(sql.contains(&format!( 611 | "{POSITION_DELETE_TABLE} ON {DATA_FILE_TABLE}.{SYS_HIDDEN_FILE_PATH} = {POSITION_DELETE_TABLE}.{SYS_HIDDEN_FILE_PATH} AND {DATA_FILE_TABLE}.{SYS_HIDDEN_POS} = {POSITION_DELETE_TABLE}.{SYS_HIDDEN_POS}", 612 | ))); 613 | } 614 | 615 | /// Test building SQL with equality delete files 616 | #[test] 617 | fn test_build_merge_on_read_sql_with_equality_deletes() { 618 | let project_names = vec!["id".to_owned(), "name".to_owned()]; 619 | let equality_delete_table_name = "test".to_owned(); 620 | let equality_delete_metadatas = vec![EqualityDeleteMetadata::new( 621 | Schema::builder() 622 | .with_fields(vec![Arc::new(NestedField::new( 623 | 1, 624 | "id", 625 | Type::Primitive(PrimitiveType::Int), 626 | true, 627 | ))]) 628 | .build() 629 | .unwrap(), 630 | equality_delete_table_name.clone(), 631 | )]; 632 | 633 | let builder = SqlBuilder::new( 634 | &project_names, 635 | Some(POSITION_DELETE_TABLE.to_owned()), 636 | Some(DATA_FILE_TABLE.to_owned()), 637 | &equality_delete_metadatas, 638 | false, 639 | ); 640 | let sql = builder.build_merge_on_read_sql().unwrap(); 641 | assert!(sql.contains(&format!( 642 | "LEFT ANTI JOIN {equality_delete_table_name} ON {DATA_FILE_TABLE}", 643 | ))); 644 | assert!(sql.contains(&format!( 645 | "{equality_delete_table_name} ON {DATA_FILE_TABLE}.id = {equality_delete_table_name}.id", 646 | ))); 647 | } 648 | 649 | /// Test building SQL with equality delete files AND sequence number comparison 650 | #[test] 651 | fn test_build_merge_on_read_sql_with_equality_deletes_and_seq_num() { 652 | let project_names = vec!["id".to_owned(), "name".to_owned()]; 653 | 654 | let equality_delete_table_name = "test".to_owned(); 655 | let equality_delete_metadatas = vec![EqualityDeleteMetadata::new( 656 | Schema::builder() 657 | .with_fields(vec![Arc::new(NestedField::new( 658 | 1, 659 | "id", 660 | Type::Primitive(PrimitiveType::Int), 661 | true, 662 | ))]) 663 | .build() 664 | .unwrap(), 665 | equality_delete_table_name.clone(), 666 | )]; 667 | 668 | let builder = SqlBuilder::new( 669 | &project_names, 670 | Some(POSITION_DELETE_TABLE.to_owned()), 671 | Some(DATA_FILE_TABLE.to_owned()), 672 | &equality_delete_metadatas, 673 | false, 674 | ); 675 | let sql = builder.build_merge_on_read_sql().unwrap(); 676 | assert!(sql.contains(&format!( 677 | "{DATA_FILE_TABLE}.{SYS_HIDDEN_SEQ_NUM} < {equality_delete_table_name}.{SYS_HIDDEN_SEQ_NUM}", 678 | ))); 679 | } 680 | 681 | /// Test building SQL with both position AND equality delete files 682 | #[test] 683 | fn test_build_merge_on_read_sql_with_both_deletes() { 684 | let project_names = vec!["id".to_owned(), "name".to_owned()]; 685 | let equality_delete_table_name = "test".to_owned(); 686 | let equality_delete_metadatas = vec![EqualityDeleteMetadata::new( 687 | Schema::builder() 688 | .with_fields(vec![Arc::new(NestedField::new( 689 | 1, 690 | "id", 691 | Type::Primitive(PrimitiveType::Int), 692 | true, 693 | ))]) 694 | .build() 695 | .unwrap(), 696 | equality_delete_table_name.clone(), 697 | )]; 698 | 699 | let builder = SqlBuilder::new( 700 | &project_names, 701 | Some(POSITION_DELETE_TABLE.to_owned()), 702 | Some(DATA_FILE_TABLE.to_owned()), 703 | &equality_delete_metadatas, 704 | true, 705 | ); 706 | let sql = builder.build_merge_on_read_sql().unwrap(); 707 | assert!(sql.contains(&format!( 708 | "LEFT ANTI JOIN {POSITION_DELETE_TABLE} ON {DATA_FILE_TABLE}" 709 | ))); 710 | assert!(sql.contains(&format!( 711 | "LEFT ANTI JOIN {equality_delete_table_name} ON {DATA_FILE_TABLE}", 712 | ))); 713 | assert!(sql.contains(&format!( 714 | "{POSITION_DELETE_TABLE} ON {DATA_FILE_TABLE}.{SYS_HIDDEN_FILE_PATH} = {POSITION_DELETE_TABLE}.{SYS_HIDDEN_FILE_PATH} AND {DATA_FILE_TABLE}.{SYS_HIDDEN_POS} = {POSITION_DELETE_TABLE}.{SYS_HIDDEN_POS}", 715 | ))); 716 | assert!(sql.contains(&format!( 717 | "{equality_delete_table_name} ON {DATA_FILE_TABLE}.id = {equality_delete_table_name}.id", 718 | ))); 719 | assert!(sql.contains(&format!( 720 | "{DATA_FILE_TABLE}.{SYS_HIDDEN_SEQ_NUM} < {equality_delete_table_name}.{SYS_HIDDEN_SEQ_NUM}", 721 | ))); 722 | } 723 | 724 | /// Test building SQL with multiple equality delete files 725 | #[test] 726 | fn test_build_merge_on_read_sql_with_multiple_equality_deletes_schema() { 727 | let project_names = vec!["id".to_owned(), "name".to_owned()]; 728 | 729 | let equality_delete_table_name_1 = "test_1".to_owned(); 730 | let equality_delete_table_name_2 = "test_2".to_owned(); 731 | let equality_delete_metadatas = vec![ 732 | EqualityDeleteMetadata::new( 733 | Schema::builder() 734 | .with_fields(vec![Arc::new(NestedField::new( 735 | 1, 736 | "id", 737 | Type::Primitive(PrimitiveType::Int), 738 | true, 739 | ))]) 740 | .build() 741 | .unwrap(), 742 | equality_delete_table_name_1.clone(), 743 | ), 744 | EqualityDeleteMetadata::new( 745 | Schema::builder() 746 | .with_fields(vec![Arc::new(NestedField::new( 747 | 1, 748 | "id", 749 | Type::Primitive(PrimitiveType::Int), 750 | true, 751 | ))]) 752 | .build() 753 | .unwrap(), 754 | equality_delete_table_name_2.clone(), 755 | ), 756 | ]; 757 | 758 | let builder = SqlBuilder::new( 759 | &project_names, 760 | Some(POSITION_DELETE_TABLE.to_owned()), 761 | Some(DATA_FILE_TABLE.to_owned()), 762 | &equality_delete_metadatas, 763 | false, 764 | ); 765 | let sql = builder.build_merge_on_read_sql().unwrap(); 766 | 767 | assert!(sql.contains( 768 | &("LEFT ANTI JOIN ".to_owned() 769 | + &equality_delete_table_name_1 770 | + " ON " 771 | + DATA_FILE_TABLE) 772 | )); 773 | assert!(sql.contains( 774 | &("LEFT ANTI JOIN ".to_owned() 775 | + &equality_delete_table_name_2 776 | + " ON " 777 | + DATA_FILE_TABLE) 778 | )); 779 | assert!(sql.contains( 780 | &(equality_delete_table_name_1.clone() 781 | + " ON " 782 | + DATA_FILE_TABLE 783 | + ".id = " 784 | + &equality_delete_table_name_1 785 | + ".id") 786 | )); 787 | assert!(sql.contains( 788 | &(equality_delete_table_name_2.clone() 789 | + " ON " 790 | + DATA_FILE_TABLE 791 | + ".id = " 792 | + &equality_delete_table_name_2 793 | + ".id") 794 | )); 795 | 796 | // Check that the sequence number comparison is present for both equality delete tables 797 | assert!(sql.contains( 798 | &(DATA_FILE_TABLE.to_owned() 799 | + "." 800 | + SYS_HIDDEN_SEQ_NUM 801 | + " < " 802 | + &equality_delete_table_name_1 803 | + "." 804 | + SYS_HIDDEN_SEQ_NUM) 805 | )); 806 | assert!(sql.contains( 807 | &(DATA_FILE_TABLE.to_owned() 808 | + "." 809 | + SYS_HIDDEN_SEQ_NUM 810 | + " < " 811 | + &equality_delete_table_name_2 812 | + "." 813 | + SYS_HIDDEN_SEQ_NUM) 814 | )); 815 | } 816 | 817 | #[test] 818 | fn test_build_equality_delete_schema() { 819 | let schema = Schema::builder() 820 | .with_fields(vec![ 821 | Arc::new(NestedField::new( 822 | 1, 823 | "id", 824 | iceberg::spec::Type::Primitive(PrimitiveType::Int), 825 | true, 826 | )), 827 | Arc::new(NestedField::new( 828 | 2, 829 | "name", 830 | iceberg::spec::Type::Primitive(PrimitiveType::String), 831 | true, 832 | )), 833 | ]) 834 | .build() 835 | .unwrap(); 836 | 837 | let mut highest_field_id = schema.highest_field_id(); 838 | 839 | let builder = DataFusionTaskContextBuilder { 840 | schema: Arc::new(schema), 841 | data_files: vec![], 842 | position_delete_files: vec![], 843 | equality_delete_files: vec![], 844 | }; 845 | 846 | let equality_ids = vec![1, 2]; 847 | let equality_delete_schema = builder 848 | .build_equality_delete_schema(&equality_ids, &mut highest_field_id) 849 | .unwrap(); 850 | 851 | assert_eq!(equality_delete_schema.as_struct().fields().len(), 3); 852 | assert_eq!(equality_delete_schema.as_struct().fields()[0].name, "id"); 853 | assert_eq!(equality_delete_schema.as_struct().fields()[1].name, "name"); 854 | assert_eq!( 855 | equality_delete_schema.as_struct().fields()[2].name, 856 | "sys_hidden_seq_num" 857 | ); 858 | assert_eq!(highest_field_id, 3); 859 | } 860 | 861 | #[test] 862 | fn test_equality_delete_join_names() { 863 | use iceberg::spec::{NestedField, PrimitiveType, Schema, Type}; 864 | use std::sync::Arc; 865 | 866 | // schema 867 | let fields = vec![ 868 | Arc::new(NestedField::new( 869 | 1, 870 | "id", 871 | Type::Primitive(PrimitiveType::Int), 872 | true, 873 | )), 874 | Arc::new(NestedField::new( 875 | 2, 876 | "name", 877 | Type::Primitive(PrimitiveType::String), 878 | true, 879 | )), 880 | Arc::new(NestedField::new( 881 | 3, 882 | "sys_hidden_seq_num", 883 | Type::Primitive(PrimitiveType::Long), 884 | true, 885 | )), 886 | Arc::new(NestedField::new( 887 | 4, 888 | "sys_hidden_file_path", 889 | Type::Primitive(PrimitiveType::String), 890 | true, 891 | )), 892 | ]; 893 | let schema = Schema::builder().with_fields(fields).build().unwrap(); 894 | 895 | let meta = EqualityDeleteMetadata { 896 | equality_delete_schema: schema, 897 | equality_delete_table_name: "test_table".to_string(), 898 | file_scan_tasks: vec![], 899 | }; 900 | 901 | let join_names = meta.equality_delete_join_names(); 902 | assert_eq!(join_names, vec!["id", "name"]); 903 | } 904 | } 905 | -------------------------------------------------------------------------------- /core/src/executor/datafusion/file_scan_task_table_provider.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use std::any::Any; 18 | use std::sync::Arc; 19 | 20 | use async_trait::async_trait; 21 | use datafusion::arrow::datatypes::SchemaRef as ArrowSchemaRef; 22 | use datafusion::catalog::Session; 23 | use datafusion::datasource::{TableProvider, TableType}; 24 | use datafusion::error::Result as DFResult; 25 | use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; 26 | use datafusion::physical_plan::ExecutionPlan; 27 | use iceberg::io::FileIO; 28 | use iceberg::scan::FileScanTask; 29 | 30 | use super::iceberg_file_task_scan::IcebergFileTaskScan; 31 | 32 | /// A table provider for iceberg file scan tasks 33 | #[derive(Debug, Clone)] 34 | pub struct IcebergFileScanTaskTableProvider { 35 | file_scan_tasks: Vec, 36 | schema: ArrowSchemaRef, 37 | file_io: FileIO, 38 | need_seq_num: bool, 39 | need_file_path_and_pos: bool, 40 | batch_parallelism: usize, 41 | } 42 | impl IcebergFileScanTaskTableProvider { 43 | pub fn new( 44 | file_scan_tasks: Vec, 45 | schema: ArrowSchemaRef, 46 | file_io: FileIO, 47 | need_seq_num: bool, 48 | need_file_path_and_pos: bool, 49 | batch_parallelism: usize, 50 | ) -> Self { 51 | Self { 52 | file_scan_tasks, 53 | schema, 54 | file_io, 55 | need_seq_num, 56 | need_file_path_and_pos, 57 | batch_parallelism, 58 | } 59 | } 60 | } 61 | #[async_trait] 62 | impl TableProvider for IcebergFileScanTaskTableProvider { 63 | fn as_any(&self) -> &dyn Any { 64 | self 65 | } 66 | 67 | fn schema(&self) -> ArrowSchemaRef { 68 | self.schema.clone() 69 | } 70 | 71 | fn table_type(&self) -> TableType { 72 | TableType::Base 73 | } 74 | 75 | /// Scans the iceberg file scan tasks 76 | /// 77 | /// This function creates an execution plan for scanning the iceberg file scan tasks. 78 | /// It uses the IcebergFileTaskScan struct to create the execution plan. 79 | async fn scan( 80 | &self, 81 | _state: &dyn Session, 82 | projection: Option<&Vec>, 83 | filters: &[Expr], 84 | _limit: Option, 85 | ) -> DFResult> { 86 | Ok(Arc::new(IcebergFileTaskScan::new( 87 | self.file_scan_tasks.clone(), 88 | self.schema.clone(), 89 | projection, 90 | filters, 91 | &self.file_io, 92 | self.need_seq_num, 93 | self.need_file_path_and_pos, 94 | self.batch_parallelism, 95 | ))) 96 | } 97 | 98 | fn supports_filters_pushdown( 99 | &self, 100 | filters: &[&Expr], 101 | ) -> std::result::Result, datafusion::error::DataFusionError> 102 | { 103 | // Push down all filters, as a single source of truth, the scanner will drop the filters which couldn't be push down 104 | Ok(vec![TableProviderFilterPushDown::Inexact; filters.len()]) 105 | } 106 | } 107 | -------------------------------------------------------------------------------- /core/src/executor/datafusion/iceberg_file_task_scan.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use std::any::Any; 18 | use std::collections::BinaryHeap; 19 | use std::pin::Pin; 20 | use std::sync::Arc; 21 | use std::vec; 22 | 23 | use async_stream::try_stream; 24 | use datafusion::arrow::array::{Int64Array, RecordBatch, StringArray}; 25 | use datafusion::arrow::datatypes::{Field, Schema, SchemaRef as ArrowSchemaRef}; 26 | use datafusion::error::Result as DFResult; 27 | use datafusion::execution::{SendableRecordBatchStream, TaskContext}; 28 | use datafusion::physical_expr::EquivalenceProperties; 29 | use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; 30 | use datafusion::physical_plan::stream::RecordBatchStreamAdapter; 31 | use datafusion::physical_plan::{DisplayAs, ExecutionPlan, Partitioning, PlanProperties}; 32 | use datafusion::prelude::Expr; 33 | use futures::{Stream, StreamExt, TryStreamExt}; 34 | use iceberg::arrow::ArrowReaderBuilder; 35 | use iceberg::expr::Predicate; 36 | use iceberg::io::FileIO; 37 | use iceberg::scan::FileScanTask; 38 | use iceberg_datafusion::physical_plan::expr_to_predicate::convert_filters_to_predicate; 39 | use iceberg_datafusion::to_datafusion_error; 40 | 41 | use super::datafusion_processor::SYS_HIDDEN_SEQ_NUM; 42 | 43 | /// An execution plan for scanning iceberg file scan tasks 44 | #[derive(Debug)] 45 | pub(crate) struct IcebergFileTaskScan { 46 | file_scan_tasks_group: Vec>, 47 | plan_properties: PlanProperties, 48 | projection: Option>, 49 | predicates: Option, 50 | file_io: FileIO, 51 | need_seq_num: bool, 52 | need_file_path_and_pos: bool, 53 | } 54 | 55 | impl IcebergFileTaskScan { 56 | #[allow(clippy::too_many_arguments)] 57 | pub(crate) fn new( 58 | file_scan_tasks: Vec, 59 | schema: ArrowSchemaRef, 60 | projection: Option<&Vec>, 61 | filters: &[Expr], 62 | file_io: &FileIO, 63 | need_seq_num: bool, 64 | need_file_path_and_pos: bool, 65 | batch_parallelism: usize, 66 | ) -> Self { 67 | let output_schema = match projection { 68 | None => schema.clone(), 69 | Some(projection) => Arc::new(schema.project(projection).unwrap()), 70 | }; 71 | let file_scan_tasks_group = split_n_vecs(file_scan_tasks, batch_parallelism); 72 | let plan_properties = 73 | Self::compute_properties(output_schema.clone(), file_scan_tasks_group.len()); 74 | let projection = get_column_names(schema.clone(), projection); 75 | let predicates = convert_filters_to_predicate(filters); 76 | 77 | Self { 78 | file_scan_tasks_group, 79 | plan_properties, 80 | projection, 81 | predicates, 82 | file_io: file_io.clone(), 83 | need_seq_num, 84 | need_file_path_and_pos, 85 | } 86 | } 87 | 88 | /// Computes [`PlanProperties`] used in query optimization. 89 | fn compute_properties(schema: ArrowSchemaRef, partitioning_size: usize) -> PlanProperties { 90 | // TODO: 91 | // This is more or less a placeholder, to be replaced 92 | // once we support output-partitioning 93 | PlanProperties::new( 94 | EquivalenceProperties::new(schema), 95 | Partitioning::UnknownPartitioning(partitioning_size), 96 | EmissionType::Incremental, 97 | Boundedness::Bounded, 98 | ) 99 | } 100 | } 101 | 102 | /// Uniformly distribute scan tasks to compute nodes. 103 | /// It's deterministic so that it can best utilize the data locality. 104 | /// 105 | /// # Arguments 106 | /// * `file_scan_tasks`: The file scan tasks to be split. 107 | /// * `split_num`: The number of splits to be created. 108 | /// 109 | /// This algorithm is based on a min-heap. It will push all groups into the heap, and then pop the smallest group and add the file scan task to it. 110 | /// Ensure that the total length of each group is as balanced as possible. 111 | /// The time complexity is O(n log k), where n is the number of file scan tasks and k is the number of splits. 112 | /// The space complexity is O(k), where k is the number of splits. 113 | /// The algorithm is stable, so the order of the file scan tasks will be preserved. 114 | fn split_n_vecs(file_scan_tasks: Vec, split_num: usize) -> Vec> { 115 | use std::cmp::{Ordering, Reverse}; 116 | 117 | #[derive(Default)] 118 | struct FileScanTaskGroup { 119 | idx: usize, 120 | tasks: Vec, 121 | total_length: u64, 122 | } 123 | 124 | impl Ord for FileScanTaskGroup { 125 | fn cmp(&self, other: &Self) -> Ordering { 126 | // when total_length is the same, we will sort by index 127 | if self.total_length == other.total_length { 128 | self.idx.cmp(&other.idx) 129 | } else { 130 | self.total_length.cmp(&other.total_length) 131 | } 132 | } 133 | } 134 | 135 | impl PartialOrd for FileScanTaskGroup { 136 | fn partial_cmp(&self, other: &Self) -> Option { 137 | Some(self.cmp(other)) 138 | } 139 | } 140 | 141 | impl Eq for FileScanTaskGroup {} 142 | 143 | impl PartialEq for FileScanTaskGroup { 144 | fn eq(&self, other: &Self) -> bool { 145 | self.total_length == other.total_length 146 | } 147 | } 148 | 149 | let mut heap = BinaryHeap::new(); 150 | // push all groups into heap 151 | for idx in 0..split_num { 152 | heap.push(Reverse(FileScanTaskGroup { 153 | idx, 154 | tasks: vec![], 155 | total_length: 0, 156 | })); 157 | } 158 | 159 | for file_task in file_scan_tasks { 160 | let mut group = heap.peek_mut().unwrap(); 161 | group.0.total_length += file_task.length; 162 | group.0.tasks.push(file_task); 163 | } 164 | 165 | // convert heap into vec and extract tasks 166 | heap.into_vec() 167 | .into_iter() 168 | .map(|reverse_group| reverse_group.0.tasks) 169 | .collect() 170 | } 171 | 172 | impl ExecutionPlan for IcebergFileTaskScan { 173 | fn name(&self) -> &str { 174 | "IcebergFileTaskScan" 175 | } 176 | 177 | fn as_any(&self) -> &dyn Any { 178 | self 179 | } 180 | 181 | fn children(&self) -> Vec<&Arc<(dyn ExecutionPlan + 'static)>> { 182 | vec![] 183 | } 184 | 185 | fn with_new_children( 186 | self: Arc, 187 | _children: Vec>, 188 | ) -> DFResult> { 189 | Ok(self) 190 | } 191 | 192 | fn properties(&self) -> &PlanProperties { 193 | &self.plan_properties 194 | } 195 | 196 | fn execute( 197 | &self, 198 | partition: usize, 199 | _context: Arc, 200 | ) -> DFResult { 201 | let fut = get_batch_stream( 202 | self.file_io.clone(), 203 | self.file_scan_tasks_group[partition].clone(), 204 | self.need_seq_num, 205 | self.need_file_path_and_pos, 206 | ); 207 | let stream = futures::stream::once(fut).try_flatten(); 208 | 209 | Ok(Box::pin(RecordBatchStreamAdapter::new( 210 | self.schema(), 211 | stream, 212 | ))) 213 | } 214 | } 215 | 216 | /// Gets a stream of record batches from a list of file scan tasks 217 | async fn get_batch_stream( 218 | file_io: FileIO, 219 | file_scan_tasks: Vec, 220 | need_seq_num: bool, 221 | need_file_path_and_pos: bool, 222 | ) -> DFResult> + Send>>> { 223 | let stream = try_stream! { 224 | for task in file_scan_tasks { 225 | let file_path = task.data_file_path.clone(); 226 | let data_file_content = task.data_file_content; 227 | let sequence_number = task.sequence_number; 228 | let task_stream = futures::stream::iter(vec![Ok(task)]).boxed(); 229 | let arrow_reader_builder = ArrowReaderBuilder::new(file_io.clone()); 230 | let mut batch_stream = arrow_reader_builder.build() 231 | .read(task_stream) 232 | .await 233 | .map_err(to_datafusion_error)?; 234 | let mut index_start = 0; 235 | while let Some(batch) = batch_stream.next().await { 236 | let mut batch = batch.map_err(to_datafusion_error)?; 237 | let batch = match data_file_content { 238 | iceberg::spec::DataContentType::Data => { 239 | // add sequence number if needed 240 | if need_seq_num { 241 | batch = add_seq_num_into_batch(batch, sequence_number)?; 242 | } 243 | // add file path and position if needed 244 | if need_file_path_and_pos { 245 | batch = add_file_path_pos_into_batch(batch, &file_path, index_start)?; 246 | index_start += batch.num_rows() as i64; 247 | } 248 | batch 249 | } 250 | iceberg::spec::DataContentType::PositionDeletes => { 251 | batch 252 | }, 253 | iceberg::spec::DataContentType::EqualityDeletes => { 254 | add_seq_num_into_batch(batch, sequence_number)? 255 | }, 256 | }; 257 | yield batch; 258 | } 259 | } 260 | }; 261 | Ok(Box::pin(stream)) 262 | } 263 | 264 | /// Adds a sequence number column to a record batch 265 | fn add_seq_num_into_batch(batch: RecordBatch, seq_num: i64) -> DFResult { 266 | let schema = batch.schema(); 267 | let seq_num_field = Arc::new(Field::new( 268 | SYS_HIDDEN_SEQ_NUM, 269 | datafusion::arrow::datatypes::DataType::Int64, 270 | false, 271 | )); 272 | let mut new_fields = schema.fields().to_vec(); 273 | new_fields.push(seq_num_field); 274 | let new_schema = Arc::new(Schema::new(new_fields)); 275 | 276 | let mut columns = batch.columns().to_vec(); 277 | columns.push(Arc::new(Int64Array::from(vec![seq_num; batch.num_rows()]))); 278 | RecordBatch::try_new(new_schema, columns) 279 | .map_err(|e| datafusion::error::DataFusionError::ArrowError(e, None)) 280 | } 281 | 282 | /// Adds a file path and position column to a record batch 283 | fn add_file_path_pos_into_batch( 284 | batch: RecordBatch, 285 | file_path: &str, 286 | index_start: i64, 287 | ) -> DFResult { 288 | let schema = batch.schema(); 289 | let file_path_field = Arc::new(Field::new( 290 | "file_path", 291 | datafusion::arrow::datatypes::DataType::Utf8, 292 | false, 293 | )); 294 | let pos_field = Arc::new(Field::new( 295 | "pos", 296 | datafusion::arrow::datatypes::DataType::Int64, 297 | false, 298 | )); 299 | let mut new_fields = schema.fields().to_vec(); 300 | new_fields.push(file_path_field); 301 | new_fields.push(pos_field); 302 | let new_schema = Arc::new(Schema::new(new_fields)); 303 | 304 | let mut columns = batch.columns().to_vec(); 305 | columns.push(Arc::new(StringArray::from(vec![ 306 | file_path; 307 | batch.num_rows() 308 | ]))); 309 | columns.push(Arc::new(Int64Array::from_iter( 310 | (index_start..(index_start + batch.num_rows() as i64)).collect::>(), 311 | ))); 312 | RecordBatch::try_new(new_schema, columns) 313 | .map_err(|e| datafusion::error::DataFusionError::ArrowError(e, None)) 314 | } 315 | 316 | impl DisplayAs for IcebergFileTaskScan { 317 | fn fmt_as( 318 | &self, 319 | _t: datafusion::physical_plan::DisplayFormatType, 320 | f: &mut std::fmt::Formatter, 321 | ) -> std::fmt::Result { 322 | write!( 323 | f, 324 | "IcebergTableScan projection:[{}] predicate:[{}]", 325 | self.projection 326 | .clone() 327 | .map_or(String::new(), |v| v.join(",")), 328 | self.predicates 329 | .clone() 330 | .map_or(String::from(""), |p| format!("{}", p)) 331 | ) 332 | } 333 | } 334 | 335 | pub fn get_column_names( 336 | schema: ArrowSchemaRef, 337 | projection: Option<&Vec>, 338 | ) -> Option> { 339 | projection.map(|v| { 340 | v.iter() 341 | .map(|p| schema.field(*p).name().clone()) 342 | .collect::>() 343 | }) 344 | } 345 | 346 | #[cfg(test)] 347 | mod tests { 348 | use super::*; 349 | use iceberg::scan::FileScanTask; 350 | use iceberg::spec::{DataContentType, Schema}; 351 | use std::sync::Arc; 352 | 353 | fn create_file_scan_task(length: u64, file_id: u64) -> FileScanTask { 354 | FileScanTask { 355 | length, 356 | start: 0, 357 | record_count: Some(0), 358 | data_file_path: format!("test_{}.parquet", file_id), 359 | data_file_content: DataContentType::Data, 360 | data_file_format: iceberg::spec::DataFileFormat::Parquet, 361 | schema: Arc::new(Schema::builder().build().unwrap()), 362 | project_field_ids: vec![], 363 | predicate: None, 364 | deletes: vec![], 365 | sequence_number: 0, 366 | equality_ids: vec![], 367 | file_size_in_bytes: 0, 368 | } 369 | } 370 | 371 | #[test] 372 | fn test_split_n_vecs_basic() { 373 | let file_scan_tasks = (1..=12) 374 | .map(|i| create_file_scan_task(i + 100, i)) 375 | .collect::>(); 376 | 377 | let groups = split_n_vecs(file_scan_tasks, 3); 378 | 379 | assert_eq!(groups.len(), 3); 380 | 381 | let group_lengths: Vec = groups 382 | .iter() 383 | .map(|group| group.iter().map(|task| task.length).sum()) 384 | .collect(); 385 | 386 | let max_length = *group_lengths.iter().max().unwrap(); 387 | let min_length = *group_lengths.iter().min().unwrap(); 388 | assert!(max_length - min_length <= 10, "Groups should be balanced"); 389 | 390 | let total_tasks: usize = groups.iter().map(|group| group.len()).sum(); 391 | assert_eq!(total_tasks, 12); 392 | } 393 | 394 | #[test] 395 | fn test_split_n_vecs_empty() { 396 | let file_scan_tasks = Vec::new(); 397 | let groups = split_n_vecs(file_scan_tasks, 3); 398 | assert_eq!(groups.len(), 3); 399 | assert!(groups.iter().all(|group| group.is_empty())); 400 | } 401 | 402 | #[test] 403 | fn test_split_n_vecs_single_task() { 404 | let file_scan_tasks = vec![create_file_scan_task(100, 1)]; 405 | let groups = split_n_vecs(file_scan_tasks, 3); 406 | assert_eq!(groups.len(), 3); 407 | assert_eq!(groups.iter().filter(|group| !group.is_empty()).count(), 1); 408 | } 409 | 410 | #[test] 411 | fn test_split_n_vecs_uneven_distribution() { 412 | let file_scan_tasks = vec![ 413 | create_file_scan_task(1000, 1), 414 | create_file_scan_task(100, 2), 415 | create_file_scan_task(100, 3), 416 | create_file_scan_task(100, 4), 417 | create_file_scan_task(100, 5), 418 | ]; 419 | 420 | let groups = split_n_vecs(file_scan_tasks, 2); 421 | assert_eq!(groups.len(), 2); 422 | 423 | let group_with_large_task = groups 424 | .iter() 425 | .find(|group| group.iter().any(|task| task.length == 1000)) 426 | .unwrap(); 427 | assert_eq!(group_with_large_task.len(), 1); 428 | } 429 | 430 | #[test] 431 | fn test_split_n_vecs_same_files_distribution() { 432 | let file_scan_tasks = vec![ 433 | create_file_scan_task(100, 1), 434 | create_file_scan_task(100, 2), 435 | create_file_scan_task(100, 3), 436 | create_file_scan_task(100, 4), 437 | create_file_scan_task(100, 5), 438 | create_file_scan_task(100, 6), 439 | create_file_scan_task(100, 7), 440 | create_file_scan_task(100, 8), 441 | ]; 442 | 443 | let groups = split_n_vecs(file_scan_tasks.clone(), 4) 444 | .iter() 445 | .map(|g| { 446 | g.iter() 447 | .map(|task| task.data_file_path.clone()) 448 | .collect::>() 449 | }) 450 | .collect::>(); 451 | 452 | for _ in 0..10000 { 453 | let groups_2 = split_n_vecs(file_scan_tasks.clone(), 4) 454 | .iter() 455 | .map(|g| { 456 | g.iter() 457 | .map(|task| task.data_file_path.clone()) 458 | .collect::>() 459 | }) 460 | .collect::>(); 461 | 462 | assert_eq!(groups, groups_2); 463 | } 464 | } 465 | } 466 | -------------------------------------------------------------------------------- /core/src/executor/datafusion/mod.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use crate::{error::Result, executor::iceberg_writer::rolling_iceberg_writer}; 18 | use ::datafusion::{ 19 | parquet::file::properties::WriterProperties, 20 | prelude::{SessionConfig, SessionContext}, 21 | }; 22 | use async_trait::async_trait; 23 | use datafusion_processor::{DataFusionTaskContext, DatafusionProcessor}; 24 | use futures::{StreamExt, future::try_join_all}; 25 | use iceberg::{ 26 | io::FileIO, 27 | spec::{DataFile, PartitionSpec, Schema}, 28 | writer::{ 29 | IcebergWriter, IcebergWriterBuilder, 30 | base_writer::data_file_writer::DataFileWriterBuilder, 31 | file_writer::{ 32 | ParquetWriterBuilder, 33 | location_generator::{DefaultFileNameGenerator, DefaultLocationGenerator}, 34 | }, 35 | function_writer::fanout_partition_writer::FanoutPartitionWriterBuilder, 36 | }, 37 | }; 38 | use sqlx::types::Uuid; 39 | use std::sync::Arc; 40 | use tokio::task::JoinHandle; 41 | 42 | use crate::CompactionError; 43 | 44 | use super::{CompactionExecutor, InputFileScanTasks, RewriteFilesStat}; 45 | pub mod datafusion_processor; 46 | use super::{RewriteFilesRequest, RewriteFilesResponse}; 47 | pub mod file_scan_task_table_provider; 48 | pub mod iceberg_file_task_scan; 49 | 50 | #[derive(Default)] 51 | pub struct DataFusionExecutor {} 52 | 53 | #[async_trait] 54 | impl CompactionExecutor for DataFusionExecutor { 55 | async fn rewrite_files(&self, request: RewriteFilesRequest) -> Result { 56 | let RewriteFilesRequest { 57 | file_io, 58 | schema, 59 | input_file_scan_tasks, 60 | config, 61 | dir_path, 62 | partition_spec, 63 | } = request; 64 | let mut session_config = SessionConfig::new(); 65 | session_config = session_config.with_target_partitions(config.target_partitions); 66 | let ctx = Arc::new(SessionContext::new_with_config(session_config)); 67 | 68 | let mut stat = RewriteFilesStat::default(); 69 | let rewritten_files_count = input_file_scan_tasks.input_files_count(); 70 | 71 | let InputFileScanTasks { 72 | data_files, 73 | position_delete_files, 74 | equality_delete_files, 75 | } = input_file_scan_tasks; 76 | 77 | let datafusion_task_ctx = DataFusionTaskContext::builder()? 78 | .with_schema(schema) 79 | .with_datafile(data_files) 80 | .with_position_delete_files(position_delete_files) 81 | .with_equality_delete_files(equality_delete_files) 82 | .build_merge_on_read()?; 83 | let (batchs, input_schema) = DatafusionProcessor::new( 84 | ctx, 85 | datafusion_task_ctx, 86 | config.batch_parallelism, 87 | config.target_partitions, 88 | file_io.clone(), 89 | ) 90 | .execute() 91 | .await?; 92 | let arc_input_schema = Arc::new(input_schema); 93 | let mut futures = Vec::with_capacity(config.batch_parallelism); 94 | // build iceberg writer for each partition 95 | for mut batch in batchs { 96 | let dir_path = dir_path.clone(); 97 | let schema = arc_input_schema.clone(); 98 | let data_file_prefix = (&config.data_file_prefix).clone(); 99 | let target_file_size = config.target_file_size; 100 | let file_io = file_io.clone(); 101 | let partition_spec = partition_spec.clone(); 102 | let future: JoinHandle< 103 | std::result::Result, CompactionError>, 104 | > = tokio::spawn(async move { 105 | let mut data_file_writer = Self::build_iceberg_writer( 106 | data_file_prefix, 107 | dir_path, 108 | schema, 109 | file_io, 110 | partition_spec, 111 | target_file_size, 112 | ) 113 | .await?; 114 | while let Some(b) = batch.as_mut().next().await { 115 | data_file_writer.write(b?).await?; 116 | } 117 | let data_files = data_file_writer.close().await?; 118 | Ok(data_files) 119 | }); 120 | futures.push(future); 121 | } 122 | // collect all data files from all partitions 123 | let output_data_files: Vec = try_join_all(futures) 124 | .await 125 | .map_err(|e| CompactionError::Execution(e.to_string()))? 126 | .into_iter() 127 | .map(|res| res.map(|v| v.into_iter())) 128 | .collect::>>() 129 | .map(|iters| iters.into_iter().flatten().collect())?; 130 | 131 | stat.added_files_count = output_data_files.len() as u32; 132 | stat.rewritten_bytes = output_data_files 133 | .iter() 134 | .map(|f| f.file_size_in_bytes()) 135 | .sum(); 136 | stat.rewritten_files_count = rewritten_files_count; 137 | 138 | Ok(RewriteFilesResponse { 139 | data_files: output_data_files, 140 | stat, 141 | }) 142 | } 143 | } 144 | 145 | impl DataFusionExecutor { 146 | async fn build_iceberg_writer( 147 | data_file_prefix: String, 148 | dir_path: String, 149 | schema: Arc, 150 | file_io: FileIO, 151 | partition_spec: Arc, 152 | target_file_size: usize, 153 | ) -> Result> { 154 | let location_generator = DefaultLocationGenerator { dir_path }; 155 | let unique_uuid_suffix = Uuid::now_v7(); 156 | let file_name_generator = DefaultFileNameGenerator::new( 157 | data_file_prefix, 158 | Some(unique_uuid_suffix.to_string()), 159 | iceberg::spec::DataFileFormat::Parquet, 160 | ); 161 | 162 | let parquet_writer_builder = ParquetWriterBuilder::new( 163 | WriterProperties::default(), 164 | schema.clone(), 165 | file_io, 166 | location_generator, 167 | file_name_generator, 168 | ); 169 | let data_file_builder = 170 | DataFileWriterBuilder::new(parquet_writer_builder, None, partition_spec.spec_id()); 171 | let data_file_size_writer = rolling_iceberg_writer::RollingIcebergWriterBuilder::new( 172 | data_file_builder, 173 | target_file_size, 174 | ); 175 | let iceberg_output_writer = if partition_spec.fields().is_empty() { 176 | Box::new(data_file_size_writer.build().await?) as Box 177 | } else { 178 | Box::new( 179 | FanoutPartitionWriterBuilder::new( 180 | data_file_size_writer, 181 | partition_spec.clone(), 182 | schema, 183 | )? 184 | .build() 185 | .await?, 186 | ) as Box 187 | }; 188 | Ok(iceberg_output_writer) 189 | } 190 | } 191 | -------------------------------------------------------------------------------- /core/src/executor/iceberg_writer/mod.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | pub mod rolling_iceberg_writer; 18 | -------------------------------------------------------------------------------- /core/src/executor/iceberg_writer/rolling_iceberg_writer.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use datafusion::arrow::array::RecordBatch; 18 | use iceberg::Result; 19 | use iceberg::{ 20 | spec::DataFile, 21 | writer::{CurrentFileStatus, IcebergWriter, IcebergWriterBuilder}, 22 | }; 23 | 24 | #[derive(Clone)] 25 | /// RollingIcebergWriter wraps an IcebergWriter and splits output files by target size. 26 | pub struct RollingIcebergWriter { 27 | /// Builder for creating new inner writers. 28 | inner_writer_builder: B, 29 | /// The current active writer. 30 | inner_writer: D, 31 | /// Target file size in bytes. When exceeded, a new file is started. 32 | target_file_size: usize, 33 | /// Collected data files that have been closed. 34 | data_files: Vec, 35 | /// Current written size of the active file. 36 | current_written_size: usize, 37 | } 38 | 39 | #[async_trait::async_trait] 40 | impl IcebergWriter for RollingIcebergWriter 41 | where 42 | B: IcebergWriterBuilder, 43 | D: IcebergWriter + CurrentFileStatus, 44 | { 45 | /// Write a RecordBatch. If the current file size plus the new batch size 46 | /// exceeds the target, close the current file and start a new one. 47 | async fn write(&mut self, input: RecordBatch) -> Result<()> { 48 | let input_size = input.get_array_memory_size(); 49 | // If adding this batch would exceed the target file size, close current file and start a new one. 50 | if need_build_new_file(self.current_written_size, input_size, self.target_file_size) { 51 | let data_files = self.inner_writer.close().await?; 52 | self.data_files.extend(data_files); 53 | self.inner_writer = self.inner_writer_builder.clone().build().await?; 54 | self.current_written_size = 0; 55 | } 56 | // Write the batch to the current writer. 57 | self.inner_writer.write(input).await?; 58 | self.current_written_size += input_size; 59 | Ok(()) 60 | } 61 | 62 | /// Close the writer, ensuring all data files are finalized and returned. 63 | async fn close(&mut self) -> Result> { 64 | let mut data_files = std::mem::take(&mut self.data_files); 65 | data_files.extend(self.inner_writer.close().await?); 66 | Ok(data_files) 67 | } 68 | } 69 | 70 | pub fn need_build_new_file( 71 | current_written_size: usize, 72 | input_size: usize, 73 | target_file_size: usize, 74 | ) -> bool { 75 | // If the current file size is less than 10% of the target size, don't build a new file. 76 | if current_written_size < target_file_size * 1 / 10 { 77 | return false; 78 | } 79 | // If the total size of the current file and the new batch would exceed 1.5x the target size, build a new file. 80 | if current_written_size + input_size > target_file_size * 3 / 2 { 81 | return true; 82 | } 83 | // If the total size of the current file and the new batch would exceed the target size, build a new file. 84 | if current_written_size + input_size > target_file_size 85 | && current_written_size > target_file_size * 7 / 10 86 | { 87 | return true; 88 | } 89 | false 90 | } 91 | 92 | #[derive(Clone)] 93 | /// Builder for RollingIcebergWriter. 94 | pub struct RollingIcebergWriterBuilder { 95 | inner_builder: B, 96 | target_file_size: usize, 97 | } 98 | 99 | impl RollingIcebergWriterBuilder { 100 | /// Create a new RollingIcebergWriterBuilder. 101 | pub fn new(inner_builder: B, target_file_size: usize) -> Self { 102 | Self { 103 | inner_builder, 104 | target_file_size, 105 | } 106 | } 107 | } 108 | 109 | #[async_trait::async_trait] 110 | impl IcebergWriterBuilder for RollingIcebergWriterBuilder 111 | where 112 | B: IcebergWriterBuilder, 113 | B::R: IcebergWriter + CurrentFileStatus, 114 | { 115 | type R = RollingIcebergWriter; 116 | 117 | /// Build a new RollingIcebergWriter. 118 | async fn build(self) -> Result { 119 | Ok(RollingIcebergWriter { 120 | inner_writer_builder: self.inner_builder.clone(), 121 | inner_writer: self.inner_builder.build().await?, 122 | target_file_size: self.target_file_size, 123 | data_files: Vec::new(), 124 | current_written_size: 0, 125 | }) 126 | } 127 | } 128 | 129 | #[cfg(test)] 130 | mod tests { 131 | use super::*; 132 | 133 | #[test] 134 | fn test_need_build_new_file_total_size_exceeds_threshold() { 135 | let target_size = 1000; 136 | 137 | // Test when total size exceeds 1.5x target size 138 | assert!(need_build_new_file(800, 800, target_size)); // 1600 > 1500 139 | assert!(need_build_new_file(1000, 600, target_size)); // 1600 > 1500 140 | } 141 | 142 | #[test] 143 | fn test_need_build_new_file_normal_cases() { 144 | let target_size = 1000; 145 | 146 | // Case 1: Current file size > 70% and total size would exceed target 147 | assert!(need_build_new_file(800, 300, target_size)); 148 | 149 | // Case 2: Current file size > 70% but total size would not exceed target 150 | assert!(!need_build_new_file(800, 100, target_size)); 151 | 152 | // Case 3: Current file size < 70% even though total size would exceed target 153 | assert!(!need_build_new_file(600, 500, target_size)); 154 | } 155 | 156 | #[test] 157 | fn test_need_build_new_file_edge_cases() { 158 | let target_size = 1000; 159 | 160 | // Empty file case 161 | assert!(!need_build_new_file(0, 2000, target_size)); 162 | 163 | // Exactly at 70% threshold 164 | assert!(!need_build_new_file(700, 400, target_size)); 165 | 166 | // Just over 70% threshold 167 | assert!(need_build_new_file(701, 400, target_size)); 168 | 169 | // Exactly at 1.5x threshold 170 | assert!(!need_build_new_file(0, 1500, target_size)); 171 | 172 | // Just over 1.5x threshold 173 | assert!(!need_build_new_file(1, 1501, target_size)); 174 | } 175 | } 176 | -------------------------------------------------------------------------------- /core/src/executor/mock.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use async_trait::async_trait; 18 | 19 | use super::{CompactionExecutor, RewriteFilesRequest, RewriteFilesResponse}; 20 | use crate::error::Result; 21 | 22 | pub struct MockExecutor; 23 | 24 | #[async_trait] 25 | impl CompactionExecutor for MockExecutor { 26 | async fn rewrite_files(&self, _request: RewriteFilesRequest) -> Result { 27 | Ok(RewriteFilesResponse::default()) 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /core/src/executor/mod.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use std::sync::Arc; 18 | 19 | use async_trait::async_trait; 20 | use iceberg::scan::FileScanTask; 21 | use iceberg::{io::FileIO, spec::PartitionSpec}; 22 | 23 | use crate::parser::proto::RewriteFilesResponseProtoEncoder; 24 | use crate::{config::CompactionConfig, parser::proto::PbRewriteFilesRequestDecoder}; 25 | use iceberg::spec::{DataFile, Schema}; 26 | 27 | pub mod mock; 28 | pub use mock::MockExecutor; 29 | pub mod datafusion; 30 | pub mod iceberg_writer; 31 | use crate::error::Result; 32 | use bergloom_codegen::compactor::RewriteFilesRequest as PbRewriteFilesRequest; 33 | use bergloom_codegen::compactor::RewriteFilesResponse as PbRewriteFilesResponse; 34 | pub use datafusion::DataFusionExecutor; 35 | 36 | #[async_trait] 37 | pub trait CompactionExecutor: Send + Sync + 'static { 38 | async fn rewrite_files(&self, request: RewriteFilesRequest) -> Result; 39 | 40 | async fn rewrite_file_proto( 41 | &self, 42 | request: PbRewriteFilesRequest, 43 | ) -> Result { 44 | let request = PbRewriteFilesRequestDecoder::new(request).decode()?; 45 | let response = self.rewrite_files(request).await?; 46 | let response = RewriteFilesResponseProtoEncoder::new(response).encode(); 47 | Ok(response) 48 | } 49 | } 50 | 51 | pub struct RewriteFilesRequest { 52 | pub file_io: FileIO, 53 | pub schema: Arc, 54 | pub input_file_scan_tasks: InputFileScanTasks, 55 | pub config: Arc, 56 | pub dir_path: String, 57 | pub partition_spec: Arc, 58 | } 59 | 60 | #[derive(Debug, Clone)] 61 | /// InputFileScanTasks contains the file scan tasks for data files, position delete files, and equality delete files. 62 | pub struct InputFileScanTasks { 63 | pub data_files: Vec, 64 | pub position_delete_files: Vec, 65 | pub equality_delete_files: Vec, 66 | } 67 | 68 | impl InputFileScanTasks { 69 | pub fn input_files_count(&self) -> u32 { 70 | self.data_files.len() as u32 71 | + self.position_delete_files.len() as u32 72 | + self.equality_delete_files.len() as u32 73 | } 74 | } 75 | 76 | #[derive(Debug, Clone, Default)] 77 | pub struct RewriteFilesResponse { 78 | pub data_files: Vec, 79 | pub stat: RewriteFilesStat, 80 | } 81 | 82 | #[derive(Debug, Clone, Default)] 83 | pub struct RewriteFilesStat { 84 | pub rewritten_files_count: u32, 85 | pub added_files_count: u32, 86 | pub rewritten_bytes: u64, 87 | pub failed_data_files_count: u32, 88 | } 89 | 90 | pub enum ExecutorType { 91 | DataFusion, 92 | Mock, 93 | } 94 | 95 | pub fn create_compaction_executor(executor_type: ExecutorType) -> Box { 96 | match executor_type { 97 | ExecutorType::DataFusion => Box::new(DataFusionExecutor::default()), 98 | ExecutorType::Mock => Box::new(MockExecutor), 99 | } 100 | } 101 | -------------------------------------------------------------------------------- /core/src/lib.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | #![feature(proc_macro_hygiene, stmt_expr_attributes)] 18 | #![feature(coroutines)] 19 | 20 | pub mod common; 21 | pub mod compaction; 22 | pub mod config; 23 | pub mod error; 24 | pub mod executor; 25 | pub mod parser; 26 | 27 | pub use config::CompactionConfig; 28 | pub use error::{CompactionError, Result}; 29 | pub use executor::CompactionExecutor; 30 | -------------------------------------------------------------------------------- /core/src/parser/mod.rs: -------------------------------------------------------------------------------- 1 | pub mod proto; 2 | -------------------------------------------------------------------------------- /core/src/parser/proto.rs: -------------------------------------------------------------------------------- 1 | use std::sync::Arc; 2 | 3 | use crate::error::Result; 4 | use bergloom_codegen::compactor::DataFile; 5 | use bergloom_codegen::compactor::FileIoBuilder; 6 | use bergloom_codegen::compactor::FileScanTaskDescriptor; 7 | use bergloom_codegen::compactor::Literal; 8 | use bergloom_codegen::compactor::MapLiteral; 9 | use bergloom_codegen::compactor::NestedFieldDescriptor; 10 | use bergloom_codegen::compactor::OptionalLiteral; 11 | use bergloom_codegen::compactor::PartitionSpec; 12 | use bergloom_codegen::compactor::PrimitiveLiteral; 13 | use bergloom_codegen::compactor::RewriteFilesRequest as PbRewriteFilesRequest; 14 | use bergloom_codegen::compactor::RewriteFilesResponse as PbRewriteFilesResponse; 15 | use bergloom_codegen::compactor::RewriteFilesStat as PbRewriteFilesStat; 16 | use bergloom_codegen::compactor::SchemaDescriptor; 17 | use bergloom_codegen::compactor::StructLiteralDescriptor; 18 | use bergloom_codegen::compactor::Transform; 19 | use bergloom_codegen::compactor::literal; 20 | use bergloom_codegen::compactor::nested_field_descriptor::FieldType; 21 | use bergloom_codegen::compactor::primitive_literal::KindLiteral; 22 | use bergloom_codegen::compactor::primitive_literal::KindWithoutInnerLiteral; 23 | use bergloom_codegen::compactor::primitive_type::Kind; 24 | use bergloom_codegen::compactor::primitive_type::KindWithoutInner; 25 | use iceberg::spec::DataContentType; 26 | use iceberg::spec::NestedField; 27 | use iceberg::spec::Type; 28 | use iceberg::{scan::FileScanTask, spec::Schema}; 29 | 30 | use crate::CompactionConfig; 31 | use crate::CompactionError; 32 | use crate::executor::InputFileScanTasks; 33 | use crate::executor::RewriteFilesRequest; 34 | use crate::executor::RewriteFilesResponse; 35 | 36 | pub struct PbRewriteFilesRequestDecoder { 37 | rewrite_file_request_proto: PbRewriteFilesRequest, 38 | } 39 | 40 | impl PbRewriteFilesRequestDecoder { 41 | pub fn new(rewrite_file_request_proto: PbRewriteFilesRequest) -> Self { 42 | Self { 43 | rewrite_file_request_proto, 44 | } 45 | } 46 | 47 | pub fn decode(self) -> Result { 48 | let PbRewriteFilesRequest { 49 | file_io_builder, 50 | file_scan_task_descriptor, 51 | schema, 52 | dir_path, 53 | rewrite_file_config, 54 | partition_spec, 55 | } = self.rewrite_file_request_proto; 56 | let file_io = Self::decode_file_io( 57 | file_io_builder 58 | .ok_or_else(|| CompactionError::Config("file_io is required".to_owned()))?, 59 | )?; 60 | let (input_file_scan_tasks, schema) = Self::decode_file_scan_tasks_and_schema( 61 | file_scan_task_descriptor, 62 | schema.ok_or_else(|| CompactionError::Config("schema is required".to_owned()))?, 63 | ) 64 | .map_err(|e| { 65 | CompactionError::Config(format!("Failed to decode file scan tasks schema: {}", e)) 66 | })?; 67 | let config = serde_json::from_value::( 68 | serde_json::to_value(rewrite_file_config).map_err(|e| { 69 | CompactionError::Config(format!( 70 | "Failed to convert rewrite_file_config to JSON value: {}", 71 | e 72 | )) 73 | })?, 74 | ) 75 | .map_err(|e| { 76 | CompactionError::Config(format!("Failed to decode CompactionConfig: {}", e)) 77 | })?; 78 | 79 | let partition_spec = Self::decode_partition_spec(partition_spec, schema.clone())? 80 | .unwrap_or_else(iceberg::spec::PartitionSpec::unpartition_spec); 81 | 82 | Ok(RewriteFilesRequest { 83 | file_io, 84 | schema, 85 | input_file_scan_tasks, 86 | config: Arc::new(config), 87 | dir_path, 88 | partition_spec: Arc::new(partition_spec), 89 | }) 90 | } 91 | 92 | /// Decode an Iceberg FileIO from a protobuf FileIOBuilder 93 | fn decode_file_io(file_io_builder_pb: FileIoBuilder) -> Result { 94 | let file_io_builder = iceberg::io::FileIO::from_path(file_io_builder_pb.scheme_str)?; 95 | Ok(file_io_builder 96 | .with_props(file_io_builder_pb.props) 97 | .build()?) 98 | } 99 | 100 | /// Decode file scan tasks and schema from protobuf descriptors 101 | /// 102 | /// This function converts protobuf descriptors into Iceberg file scan tasks and schema. 103 | /// It handles different types of files: data files, position delete files, and equality delete files. 104 | pub fn decode_file_scan_tasks_and_schema( 105 | file_scan_task_descriptors: Vec, 106 | schema: SchemaDescriptor, 107 | ) -> Result<(InputFileScanTasks, Arc)> { 108 | let mut data_files = vec![]; 109 | let mut position_delete_files = vec![]; 110 | let mut equality_delete_files = vec![]; 111 | let schema = Arc::new(Self::decode_schema(schema)?); 112 | for file_scan_task_descriptor in file_scan_task_descriptors { 113 | let mut file_scan_task = FileScanTask { 114 | start: 0, 115 | length: file_scan_task_descriptor.length, 116 | record_count: Some(file_scan_task_descriptor.record_count), 117 | data_file_path: file_scan_task_descriptor.data_file_path, 118 | data_file_content: DataContentType::try_from( 119 | file_scan_task_descriptor.data_file_content, 120 | )?, 121 | data_file_format: Self::decode_data_file_format( 122 | file_scan_task_descriptor.data_file_format, 123 | ), 124 | schema: schema.clone(), 125 | project_field_ids: file_scan_task_descriptor.project_field_ids, 126 | predicate: None, 127 | deletes: vec![], 128 | sequence_number: file_scan_task_descriptor.sequence_number, 129 | equality_ids: file_scan_task_descriptor.equality_ids, 130 | file_size_in_bytes: 0, 131 | }; 132 | match file_scan_task.data_file_content { 133 | iceberg::spec::DataContentType::Data => { 134 | data_files.push(file_scan_task); 135 | } 136 | iceberg::spec::DataContentType::PositionDeletes => { 137 | file_scan_task.project_field_ids = vec![]; 138 | position_delete_files.push(file_scan_task); 139 | } 140 | iceberg::spec::DataContentType::EqualityDeletes => { 141 | file_scan_task.project_field_ids = file_scan_task.equality_ids.clone(); 142 | equality_delete_files.push(file_scan_task); 143 | } 144 | } 145 | } 146 | Ok(( 147 | InputFileScanTasks { 148 | data_files, 149 | position_delete_files, 150 | equality_delete_files, 151 | }, 152 | schema, 153 | )) 154 | } 155 | 156 | /// decode an Iceberg schema from a protobuf schema descriptor 157 | fn decode_schema(schema: SchemaDescriptor) -> Result { 158 | let iceberg_schema_builder = Schema::builder(); 159 | let fields = schema 160 | .fields 161 | .into_iter() 162 | .map(|field| { 163 | let iceberg_field = Self::decode_field(&field)?; 164 | Ok::, CompactionError>(Arc::new(iceberg_field)) 165 | }) 166 | .collect::>>()?; 167 | Ok(iceberg_schema_builder.with_fields(fields).build()?) 168 | } 169 | 170 | /// Builds an Iceberg nested field from a protobuf field descriptor 171 | fn decode_field(field: &NestedFieldDescriptor) -> Result { 172 | let field_type = match field 173 | .field_type 174 | .as_ref() 175 | .ok_or_else(|| CompactionError::Config("field_type is null".to_owned()))? 176 | { 177 | FieldType::Primitive(primitive_type) => { 178 | match primitive_type 179 | .kind 180 | .as_ref() 181 | .ok_or_else(|| CompactionError::Config("kind is null".to_owned()))? 182 | { 183 | Kind::KindWithoutInner(inner) => { 184 | match KindWithoutInner::try_from(*inner).map_err(|e| { 185 | CompactionError::Config(format!("failed to parse kind: {}", e)) 186 | })? { 187 | KindWithoutInner::Boolean => { 188 | Type::Primitive(iceberg::spec::PrimitiveType::Boolean) 189 | } 190 | KindWithoutInner::Int => { 191 | Type::Primitive(iceberg::spec::PrimitiveType::Int) 192 | } 193 | KindWithoutInner::Long => { 194 | Type::Primitive(iceberg::spec::PrimitiveType::Long) 195 | } 196 | KindWithoutInner::Float => { 197 | Type::Primitive(iceberg::spec::PrimitiveType::Float) 198 | } 199 | KindWithoutInner::Double => { 200 | Type::Primitive(iceberg::spec::PrimitiveType::Double) 201 | } 202 | KindWithoutInner::Date => { 203 | Type::Primitive(iceberg::spec::PrimitiveType::Date) 204 | } 205 | KindWithoutInner::Time => { 206 | Type::Primitive(iceberg::spec::PrimitiveType::Time) 207 | } 208 | KindWithoutInner::Timestamp => { 209 | Type::Primitive(iceberg::spec::PrimitiveType::Timestamp) 210 | } 211 | KindWithoutInner::Timestamptz => { 212 | Type::Primitive(iceberg::spec::PrimitiveType::Timestamptz) 213 | } 214 | KindWithoutInner::TimestampNs => { 215 | Type::Primitive(iceberg::spec::PrimitiveType::TimestampNs) 216 | } 217 | KindWithoutInner::TimestamptzNs => { 218 | Type::Primitive(iceberg::spec::PrimitiveType::TimestamptzNs) 219 | } 220 | KindWithoutInner::String => { 221 | Type::Primitive(iceberg::spec::PrimitiveType::String) 222 | } 223 | KindWithoutInner::Uuid => { 224 | Type::Primitive(iceberg::spec::PrimitiveType::Uuid) 225 | } 226 | KindWithoutInner::Binary => { 227 | Type::Primitive(iceberg::spec::PrimitiveType::Binary) 228 | } 229 | } 230 | } 231 | Kind::Decimal(decimal) => { 232 | Type::Primitive(iceberg::spec::PrimitiveType::Decimal { 233 | precision: decimal.precision, 234 | scale: decimal.scale, 235 | }) 236 | } 237 | Kind::Fixed(size) => { 238 | Type::Primitive(iceberg::spec::PrimitiveType::Fixed(*size)) 239 | } 240 | } 241 | } 242 | FieldType::Struct(struct_type) => { 243 | let fields = struct_type 244 | .fields 245 | .iter() 246 | .map(|field| { 247 | Ok::, CompactionError>(Arc::new(Self::decode_field( 248 | field, 249 | )?)) 250 | }) 251 | .collect::>>()?; 252 | Type::Struct(iceberg::spec::StructType::new(fields)) 253 | } 254 | FieldType::List(nested_field_descriptor) => { 255 | let element_field = Self::decode_field(nested_field_descriptor)?; 256 | Type::List(iceberg::spec::ListType::new(Arc::new(element_field))) 257 | } 258 | FieldType::Map(map_type) => { 259 | let key_field = 260 | Self::decode_field(map_type.key_field.as_ref().ok_or_else(|| { 261 | CompactionError::Config("can't find key_field in map".to_owned()) 262 | })?)?; 263 | let value_field = 264 | Self::decode_field(map_type.value_field.as_ref().ok_or_else(|| { 265 | CompactionError::Config("can't find value_field in map".to_owned()) 266 | })?)?; 267 | Type::Map(iceberg::spec::MapType::new( 268 | Arc::new(key_field), 269 | Arc::new(value_field), 270 | )) 271 | } 272 | }; 273 | Ok(NestedField::new( 274 | field.id, 275 | field.name.clone(), 276 | field_type, 277 | field.required, 278 | )) 279 | } 280 | 281 | /// Converts a protobuf data file format to an Iceberg data file format 282 | fn decode_data_file_format(data_file_format: i32) -> iceberg::spec::DataFileFormat { 283 | match data_file_format { 284 | 0 => iceberg::spec::DataFileFormat::Avro, 285 | 1 => iceberg::spec::DataFileFormat::Orc, 286 | 2 => iceberg::spec::DataFileFormat::Parquet, 287 | _ => unreachable!(), 288 | } 289 | } 290 | 291 | /// Builds an Iceberg PartitionSpec from a protobuf PartitionSpec 292 | /// 293 | /// This function converts a protobuf PartitionSpec into an Iceberg PartitionSpec. 294 | /// It handles the conversion of partition fields and their transforms. 295 | pub fn decode_partition_spec( 296 | partition_spec: Option, 297 | schema: Arc, 298 | ) -> Result> { 299 | match partition_spec { 300 | None => Ok(None), 301 | Some(partition_spec) => { 302 | let mut builder = iceberg::spec::PartitionSpec::builder(schema); 303 | builder = builder.with_spec_id(partition_spec.spec_id); 304 | let fields = partition_spec 305 | .partition_fields 306 | .into_iter() 307 | .map(|field| { 308 | Ok::( 309 | iceberg::spec::UnboundPartitionField { 310 | source_id: field.source_id, 311 | field_id: field.field_id, 312 | name: field.name, 313 | transform: Self::decode_transform(&field.transform.ok_or_else( 314 | || { 315 | CompactionError::Config( 316 | "cannot find transform from partition_field".to_owned(), 317 | ) 318 | }, 319 | )?)?, 320 | }, 321 | ) 322 | }) 323 | .collect::>>()?; 324 | builder = builder.add_unbound_fields(fields)?; 325 | Ok(Some(builder.build()?)) 326 | } 327 | } 328 | } 329 | 330 | /// Builds an Iceberg Transform from a protobuf Transform 331 | /// 332 | /// This function converts a protobuf Transform into an Iceberg Transform. 333 | /// It handles different transform types and their parameters. 334 | fn decode_transform(transform: &Transform) -> Result { 335 | match transform.params { 336 | Some(bergloom_codegen::compactor::transform::Params::TransformWithoutInner( 337 | transform_type, 338 | )) => { 339 | match bergloom_codegen::compactor::transform::TransformWithoutInner::try_from( 340 | transform_type, 341 | ) 342 | .map_err(|e| CompactionError::Config(format!("failed to parse kind: {}", e)))? 343 | { 344 | bergloom_codegen::compactor::transform::TransformWithoutInner::Identity => { 345 | Ok(iceberg::spec::Transform::Identity) 346 | } 347 | bergloom_codegen::compactor::transform::TransformWithoutInner::Year => { 348 | Ok(iceberg::spec::Transform::Year) 349 | } 350 | bergloom_codegen::compactor::transform::TransformWithoutInner::Month => { 351 | Ok(iceberg::spec::Transform::Month) 352 | } 353 | bergloom_codegen::compactor::transform::TransformWithoutInner::Day => { 354 | Ok(iceberg::spec::Transform::Day) 355 | } 356 | bergloom_codegen::compactor::transform::TransformWithoutInner::Hour => { 357 | Ok(iceberg::spec::Transform::Hour) 358 | } 359 | bergloom_codegen::compactor::transform::TransformWithoutInner::Void => { 360 | Ok(iceberg::spec::Transform::Void) 361 | } 362 | bergloom_codegen::compactor::transform::TransformWithoutInner::Unknown => { 363 | Ok(iceberg::spec::Transform::Unknown) 364 | } 365 | } 366 | } 367 | Some(bergloom_codegen::compactor::transform::Params::Bucket(bucket_num)) => { 368 | Ok(iceberg::spec::Transform::Bucket(bucket_num)) 369 | } 370 | Some(bergloom_codegen::compactor::transform::Params::Truncate(truncate_width)) => { 371 | Ok(iceberg::spec::Transform::Truncate(truncate_width)) 372 | } 373 | None => Err(CompactionError::Config( 374 | "Transform params is None".to_owned(), 375 | )), 376 | } 377 | } 378 | } 379 | 380 | pub struct RewriteFilesResponseProtoEncoder { 381 | rewrite_files_response: RewriteFilesResponse, 382 | } 383 | 384 | impl RewriteFilesResponseProtoEncoder { 385 | pub fn new(rewrite_files_response: RewriteFilesResponse) -> Self { 386 | Self { 387 | rewrite_files_response, 388 | } 389 | } 390 | 391 | pub fn encode(self) -> PbRewriteFilesResponse { 392 | let RewriteFilesResponse { data_files, stat } = self.rewrite_files_response; 393 | let data_files = data_files.into_iter().map(Self::encode_data_file).collect(); 394 | let stat = Some(PbRewriteFilesStat { 395 | rewritten_files_count: stat.rewritten_files_count, 396 | added_files_count: stat.added_files_count, 397 | rewritten_bytes: stat.rewritten_bytes, 398 | failed_data_files_count: stat.failed_data_files_count, 399 | }); 400 | PbRewriteFilesResponse { data_files, stat } 401 | } 402 | 403 | /// Converts an Iceberg data file to a protobuf DataFile 404 | pub fn encode_data_file(data_file: iceberg::spec::DataFile) -> DataFile { 405 | DataFile { 406 | content: data_file.content_type() as i32, 407 | file_path: data_file.file_path().to_owned(), 408 | file_format: Self::encode_data_file_format(data_file.file_format()), 409 | partition: Some(Self::encode_struct(data_file.partition().clone())), 410 | record_count: data_file.record_count(), 411 | file_size_in_bytes: data_file.file_size_in_bytes(), 412 | column_sizes: data_file.column_sizes().clone(), 413 | value_counts: data_file.value_counts().clone(), 414 | null_value_counts: data_file.null_value_counts().clone(), 415 | nan_value_counts: data_file.nan_value_counts().clone(), 416 | lower_bounds: data_file 417 | .lower_bounds() 418 | .clone() 419 | .into_iter() 420 | .map(|(k, v)| (k, v.to_bytes().unwrap().into_vec())) 421 | .collect(), 422 | upper_bounds: data_file 423 | .lower_bounds() 424 | .clone() 425 | .into_iter() 426 | .map(|(k, v)| (k, v.to_bytes().unwrap().into_vec())) 427 | .collect(), 428 | key_metadata: data_file.key_metadata().map(|k| k.to_vec()), 429 | split_offsets: data_file.split_offsets().to_vec(), 430 | equality_ids: data_file.equality_ids().to_vec(), 431 | sort_order_id: data_file.sort_order_id(), 432 | partition_spec_id: 0, 433 | } 434 | } 435 | 436 | /// Converts an Iceberg data file format to a protobuf data file format 437 | fn encode_data_file_format(data_file_format: iceberg::spec::DataFileFormat) -> i32 { 438 | match data_file_format { 439 | iceberg::spec::DataFileFormat::Avro => 0, 440 | iceberg::spec::DataFileFormat::Orc => 1, 441 | iceberg::spec::DataFileFormat::Parquet => 2, 442 | } 443 | } 444 | 445 | /// Converts an Iceberg primitive literal to a protobuf PrimitiveLiteral 446 | fn encode_primitive_literal( 447 | primitive_literal: iceberg::spec::PrimitiveLiteral, 448 | ) -> PrimitiveLiteral { 449 | match primitive_literal { 450 | iceberg::spec::PrimitiveLiteral::Boolean(b) => PrimitiveLiteral { 451 | kind_literal: Some(KindLiteral::Boolean(b)), 452 | }, 453 | iceberg::spec::PrimitiveLiteral::Int(i) => PrimitiveLiteral { 454 | kind_literal: Some(KindLiteral::Int(i)), 455 | }, 456 | iceberg::spec::PrimitiveLiteral::Long(l) => PrimitiveLiteral { 457 | kind_literal: Some(KindLiteral::Long(l)), 458 | }, 459 | iceberg::spec::PrimitiveLiteral::Float(f) => PrimitiveLiteral { 460 | kind_literal: Some(KindLiteral::Float(f.0)), 461 | }, 462 | iceberg::spec::PrimitiveLiteral::Double(f) => PrimitiveLiteral { 463 | kind_literal: Some(KindLiteral::Double(f.0)), 464 | }, 465 | iceberg::spec::PrimitiveLiteral::String(s) => PrimitiveLiteral { 466 | kind_literal: Some(KindLiteral::String(s)), 467 | }, 468 | iceberg::spec::PrimitiveLiteral::Binary(b) => PrimitiveLiteral { 469 | kind_literal: Some(KindLiteral::Binary(b)), 470 | }, 471 | iceberg::spec::PrimitiveLiteral::Int128(i) => PrimitiveLiteral { 472 | kind_literal: Some(KindLiteral::Int128(i.to_be_bytes().to_vec())), 473 | }, 474 | iceberg::spec::PrimitiveLiteral::UInt128(i) => PrimitiveLiteral { 475 | kind_literal: Some(KindLiteral::Uint128(i.to_be_bytes().to_vec())), 476 | }, 477 | iceberg::spec::PrimitiveLiteral::AboveMax => PrimitiveLiteral { 478 | kind_literal: Some(KindLiteral::KindWithoutInnerLiteral( 479 | KindWithoutInnerLiteral::AboveMax as i32, 480 | )), 481 | }, 482 | iceberg::spec::PrimitiveLiteral::BelowMin => PrimitiveLiteral { 483 | kind_literal: Some(KindLiteral::KindWithoutInnerLiteral( 484 | KindWithoutInnerLiteral::BelowMin as i32, 485 | )), 486 | }, 487 | } 488 | } 489 | 490 | /// Converts an Iceberg struct to a protobuf StructLiteralDescriptor 491 | fn encode_struct(structs: iceberg::spec::Struct) -> StructLiteralDescriptor { 492 | let literals = structs 493 | .into_iter() 494 | .map(|literal| { 495 | let literal = literal.map(Self::encode_literal); 496 | OptionalLiteral { value: literal } 497 | }) 498 | .collect(); 499 | StructLiteralDescriptor { inner: literals } 500 | } 501 | 502 | /// Converts an Iceberg literal to a protobuf Literal 503 | fn encode_literal(literal: iceberg::spec::Literal) -> Literal { 504 | match literal { 505 | iceberg::spec::Literal::Primitive(primitive_literal) => { 506 | let primitive_literal = Self::encode_primitive_literal(primitive_literal); 507 | Literal { 508 | literal: Some(literal::Literal::Primitive(primitive_literal)), 509 | } 510 | } 511 | iceberg::spec::Literal::Struct(literals) => { 512 | let literals = Self::encode_struct(literals); 513 | Literal { 514 | literal: Some(literal::Literal::Struct(literals)), 515 | } 516 | } 517 | iceberg::spec::Literal::List(literals) => { 518 | let literals = literals 519 | .into_iter() 520 | .map(|literal| { 521 | let literal = literal.map(Self::encode_literal); 522 | OptionalLiteral { value: literal } 523 | }) 524 | .collect(); 525 | Literal { 526 | literal: Some(literal::Literal::List(StructLiteralDescriptor { 527 | inner: literals, 528 | })), 529 | } 530 | } 531 | iceberg::spec::Literal::Map(map) => { 532 | let mut keys = Vec::with_capacity(map.len()); 533 | let mut values = Vec::with_capacity(map.len()); 534 | for (k, v) in map.into_iter() { 535 | keys.push(Self::encode_literal(k)); 536 | let value = OptionalLiteral { 537 | value: v.map(Self::encode_literal), 538 | }; 539 | values.push(value); 540 | } 541 | Literal { 542 | literal: Some(literal::Literal::Map(MapLiteral { keys, values })), 543 | } 544 | } 545 | } 546 | } 547 | } 548 | 549 | #[cfg(test)] 550 | mod test { 551 | use super::*; 552 | use bergloom_codegen::compactor::{MapType, NestedFieldDescriptor, PrimitiveType, StructType}; 553 | 554 | /// Test building a struct field from protobuf 555 | #[test] 556 | fn test_decode_field_from_pb_struct() { 557 | let nested_field = NestedFieldDescriptor { 558 | id: 1, 559 | name: "nested".to_owned(), 560 | field_type: Some(FieldType::Primitive(PrimitiveType { 561 | kind: Some(Kind::KindWithoutInner(KindWithoutInner::Int.into())), 562 | })), 563 | required: true, 564 | }; 565 | 566 | let struct_field = NestedFieldDescriptor { 567 | id: 2, 568 | name: "struct_field".to_owned(), 569 | field_type: Some(FieldType::Struct(StructType { 570 | fields: vec![nested_field], 571 | })), 572 | required: true, 573 | }; 574 | 575 | let result = PbRewriteFilesRequestDecoder::decode_field(&struct_field); 576 | assert!(result.is_ok()); 577 | let field = result.unwrap(); 578 | assert_eq!(field.id, 2); 579 | assert_eq!(field.name, "struct_field"); 580 | assert!(field.required); 581 | 582 | match *field.field_type { 583 | Type::Struct(struct_type) => { 584 | assert_eq!(struct_type.fields().len(), 1); 585 | let nested = *struct_type.fields()[0].field_type.clone(); 586 | assert!(matches!( 587 | nested, 588 | Type::Primitive(iceberg::spec::PrimitiveType::Int) 589 | )); 590 | } 591 | _ => panic!("Expected Struct type"), 592 | } 593 | } 594 | 595 | /// Test building a list field from protobuf 596 | #[test] 597 | fn test_decode_field_from_pb_list() { 598 | let element_field = NestedFieldDescriptor { 599 | id: 1, 600 | name: "element".to_owned(), 601 | field_type: Some(FieldType::Primitive(PrimitiveType { 602 | kind: Some(Kind::KindWithoutInner(KindWithoutInner::String.into())), 603 | })), 604 | required: true, 605 | }; 606 | 607 | let list_field = NestedFieldDescriptor { 608 | id: 2, 609 | name: "list_field".to_owned(), 610 | field_type: Some(FieldType::List(Box::new(element_field))), 611 | required: true, 612 | }; 613 | 614 | let result = PbRewriteFilesRequestDecoder::decode_field(&list_field); 615 | assert!(result.is_ok()); 616 | let field = result.unwrap(); 617 | assert_eq!(field.id, 2); 618 | assert_eq!(field.name, "list_field"); 619 | assert!(field.required); 620 | 621 | match *field.field_type { 622 | Type::List(list_type) => { 623 | let element_type = *list_type.element_field.field_type.clone(); 624 | assert!(matches!( 625 | element_type, 626 | Type::Primitive(iceberg::spec::PrimitiveType::String) 627 | )); 628 | } 629 | _ => panic!("Expected List type"), 630 | } 631 | } 632 | 633 | /// Test building a map field from protobuf 634 | #[test] 635 | fn test_build_field_from_pb_map() { 636 | let key_field = NestedFieldDescriptor { 637 | id: 1, 638 | name: "key".to_owned(), 639 | field_type: Some(FieldType::Primitive(PrimitiveType { 640 | kind: Some(Kind::KindWithoutInner(KindWithoutInner::String.into())), 641 | })), 642 | required: true, 643 | }; 644 | 645 | let value_field = NestedFieldDescriptor { 646 | id: 2, 647 | name: "value".to_owned(), 648 | field_type: Some(FieldType::Primitive(PrimitiveType { 649 | kind: Some(Kind::KindWithoutInner(KindWithoutInner::Int.into())), 650 | })), 651 | required: true, 652 | }; 653 | 654 | let map_field = NestedFieldDescriptor { 655 | id: 3, 656 | name: "map_field".to_owned(), 657 | field_type: Some(FieldType::Map(Box::new(MapType { 658 | key_field: Some(Box::new(key_field)), 659 | value_field: Some(Box::new(value_field)), 660 | }))), 661 | required: true, 662 | }; 663 | 664 | let result = PbRewriteFilesRequestDecoder::decode_field(&map_field); 665 | assert!(result.is_ok()); 666 | let field = result.unwrap(); 667 | assert_eq!(field.id, 3); 668 | assert_eq!(field.name, "map_field"); 669 | assert!(field.required); 670 | 671 | match *field.field_type { 672 | Type::Map(map_type) => { 673 | let key_type = *map_type.key_field.field_type.clone(); 674 | let value_type = *map_type.value_field.field_type.clone(); 675 | assert!(matches!( 676 | key_type, 677 | Type::Primitive(iceberg::spec::PrimitiveType::String) 678 | )); 679 | assert!(matches!( 680 | value_type, 681 | Type::Primitive(iceberg::spec::PrimitiveType::Int) 682 | )); 683 | } 684 | _ => panic!("Expected Map type"), 685 | } 686 | } 687 | 688 | /// Test building a deeply nested field from protobuf 689 | #[test] 690 | fn test_build_field_from_pb_deeply_nested() { 691 | let inner_struct_field1 = NestedFieldDescriptor { 692 | id: 1, 693 | name: "int_field".to_owned(), 694 | field_type: Some(FieldType::Primitive(PrimitiveType { 695 | kind: Some(Kind::KindWithoutInner(KindWithoutInner::Int.into())), 696 | })), 697 | required: true, 698 | }; 699 | 700 | let inner_struct_field2 = NestedFieldDescriptor { 701 | id: 2, 702 | name: "string_field".to_owned(), 703 | field_type: Some(FieldType::Primitive(PrimitiveType { 704 | kind: Some(Kind::KindWithoutInner(KindWithoutInner::String.into())), 705 | })), 706 | required: true, 707 | }; 708 | 709 | let inner_struct = NestedFieldDescriptor { 710 | id: 3, 711 | name: "inner_struct".to_owned(), 712 | field_type: Some(FieldType::Struct(StructType { 713 | fields: vec![inner_struct_field1, inner_struct_field2], 714 | })), 715 | required: true, 716 | }; 717 | 718 | let list_field = NestedFieldDescriptor { 719 | id: 4, 720 | name: "list_field".to_owned(), 721 | field_type: Some(FieldType::List(Box::new(inner_struct))), 722 | required: true, 723 | }; 724 | 725 | let key_field = NestedFieldDescriptor { 726 | id: 5, 727 | name: "key".to_owned(), 728 | field_type: Some(FieldType::Primitive(PrimitiveType { 729 | kind: Some(Kind::KindWithoutInner(KindWithoutInner::String.into())), 730 | })), 731 | required: true, 732 | }; 733 | 734 | let map_field = NestedFieldDescriptor { 735 | id: 6, 736 | name: "map_field".to_owned(), 737 | field_type: Some(FieldType::Map(Box::new(MapType { 738 | key_field: Some(Box::new(key_field)), 739 | value_field: Some(Box::new(list_field)), 740 | }))), 741 | required: true, 742 | }; 743 | 744 | let result = PbRewriteFilesRequestDecoder::decode_field(&map_field); 745 | assert!(result.is_ok()); 746 | let field = result.unwrap(); 747 | assert_eq!(field.id, 6); 748 | assert_eq!(field.name, "map_field"); 749 | assert!(field.required); 750 | 751 | match *field.field_type { 752 | Type::Map(map_type) => { 753 | let key_type = *map_type.key_field.field_type.clone(); 754 | assert!(matches!( 755 | key_type, 756 | Type::Primitive(iceberg::spec::PrimitiveType::String) 757 | )); 758 | 759 | let value_type = *map_type.value_field.field_type.clone(); 760 | match value_type { 761 | Type::List(list_type) => { 762 | let element_type = *list_type.element_field.field_type.clone(); 763 | match element_type { 764 | Type::Struct(struct_type) => { 765 | assert_eq!(struct_type.fields().len(), 2); 766 | let field1_type = *struct_type.fields()[0].field_type.clone(); 767 | let field2_type = *struct_type.fields()[1].field_type.clone(); 768 | assert!(matches!( 769 | field1_type, 770 | Type::Primitive(iceberg::spec::PrimitiveType::Int) 771 | )); 772 | assert!(matches!( 773 | field2_type, 774 | Type::Primitive(iceberg::spec::PrimitiveType::String) 775 | )); 776 | } 777 | _ => panic!("Expected Struct type in List"), 778 | } 779 | } 780 | _ => panic!("Expected List type in Map value"), 781 | } 782 | } 783 | _ => panic!("Expected Map type"), 784 | } 785 | } 786 | } 787 | -------------------------------------------------------------------------------- /docker/compactor/Dockerfile: -------------------------------------------------------------------------------- 1 | # Build stage 2 | FROM rustlang/rust:nightly as builder 3 | 4 | # Install protobuf compiler 5 | RUN apt-get update && apt-get install -y \ 6 | protobuf-compiler \ 7 | && rm -rf /var/lib/apt/lists/* 8 | 9 | # Set working directory 10 | WORKDIR /usr/src/ic 11 | 12 | # Copy workspace files 13 | COPY . . 14 | 15 | # Set working directory to compactor service 16 | WORKDIR /usr/src/ic/services/compactor 17 | 18 | # Build project 19 | RUN cargo build --release 20 | 21 | # Runtime stage 22 | FROM ubuntu:24.04 AS base 23 | 24 | # Install runtime dependencies 25 | RUN apt-get update && apt-get install -y \ 26 | ca-certificates \ 27 | && rm -rf /var/lib/apt/lists/* 28 | 29 | # Copy binary from builder stage 30 | COPY --from=builder /usr/src/ic/target/release/compactor-server /usr/local/bin/ 31 | 32 | # Set working directory 33 | WORKDIR /app 34 | 35 | # Copy configuration file 36 | COPY docker/compactor/config.yaml /app/ 37 | 38 | # Expose service port 39 | EXPOSE 7777 40 | 41 | # Set environment variables 42 | ENV RUST_LOG=info 43 | 44 | # Run service 45 | CMD ["compactor-server"] -------------------------------------------------------------------------------- /docker/compactor/config.yaml: -------------------------------------------------------------------------------- 1 | # Compactor service configuration for local development 2 | 3 | # Server configuration 4 | server: 5 | host: "127.0.0.1" 6 | port: 7777 7 | 8 | # Logging configuration 9 | logging: 10 | level: "info" 11 | format: "text" -------------------------------------------------------------------------------- /docker/compactor/docker-compose.yml: -------------------------------------------------------------------------------- 1 | services: 2 | compactor: 3 | build: 4 | context: ../.. 5 | dockerfile: docker/compactor/Dockerfile 6 | ports: 7 | - "7777:7777" 8 | volumes: 9 | - ./config.yaml:/app/config.yaml 10 | environment: 11 | - RUST_LOG=info 12 | restart: unless-stopped -------------------------------------------------------------------------------- /rust-toolchain: -------------------------------------------------------------------------------- 1 | [toolchain] 2 | channel = "nightly-2025-03-05" -------------------------------------------------------------------------------- /services/compactor/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "bergloom-service-compactor" 3 | version = "0.1.0" 4 | edition = "2024" 5 | 6 | [[bin]] 7 | name = "compactor-server" 8 | path = "src/bin/main.rs" 9 | 10 | [dependencies] 11 | async-trait = "0.1.86" 12 | bergloom-codegen = { workspace = true } 13 | bergloom-core = { workspace = true } 14 | iceberg = { workspace = true } 15 | serde = { version = "1.0", features = ["derive"] } 16 | serde_yaml = "0.9" 17 | tokio = { workspace = true } 18 | tonic = { workspace = true } 19 | tracing = "0.1" 20 | tracing-subscriber = "0.3" 21 | -------------------------------------------------------------------------------- /services/compactor/config.yaml: -------------------------------------------------------------------------------- 1 | # Compactor service configuration for local development 2 | 3 | # Server configuration 4 | server: 5 | host: "127.0.0.1" 6 | port: 7777 7 | 8 | # Logging configuration 9 | logging: 10 | level: "info" 11 | format: "text" -------------------------------------------------------------------------------- /services/compactor/src/bin/main.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use bergloom_service_compactor::{config::Config, server::grpc_compactor_serve}; 18 | use tracing_subscriber::fmt; 19 | use tracing_subscriber::layer::SubscriberExt; 20 | use tracing_subscriber::util::SubscriberInitExt; 21 | 22 | use std::{env, net::SocketAddr, path::PathBuf}; 23 | 24 | #[tokio::main] 25 | async fn main() { 26 | let config_path = find_config_file(); 27 | 28 | let config = Config::from_file(config_path).unwrap(); 29 | unsafe { 30 | env::set_var("RUST_LOG", &config.logging.level); 31 | } 32 | tracing_subscriber::registry().with(fmt::layer()).init(); 33 | 34 | // read ip and port from env 35 | let listen_addr = SocketAddr::new(config.server.host, config.server.port); 36 | let join_handle = grpc_compactor_serve(listen_addr).await; 37 | tracing::info!("Start server successful {:?}", listen_addr); 38 | 39 | // join_handle 40 | match join_handle.await { 41 | Ok(_) => { 42 | tracing::info!("Server stopped gracefully"); 43 | } 44 | Err(e) => { 45 | tracing::error!("Server stopped with error: {}", e); 46 | } 47 | } 48 | } 49 | 50 | fn find_config_file() -> PathBuf { 51 | let current_dir = env::current_dir().unwrap_or_else(|_| PathBuf::from(".")); 52 | let possible_paths = [ 53 | current_dir.join("config.yaml"), 54 | current_dir.join("services/compactor/config.yaml"), 55 | PathBuf::from("/app/config.yaml"), 56 | ]; 57 | 58 | for path in possible_paths.iter() { 59 | if path.exists() { 60 | return path.clone(); 61 | } 62 | } 63 | 64 | current_dir.join("config.yaml") 65 | } 66 | -------------------------------------------------------------------------------- /services/compactor/src/config.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use serde::Deserialize; 18 | use std::fs; 19 | use std::net::IpAddr; 20 | use std::path::Path; 21 | 22 | #[derive(Debug, Deserialize)] 23 | pub struct ServerConfig { 24 | #[serde(deserialize_with = "deserialize_ip_addr")] 25 | pub host: IpAddr, 26 | pub port: u16, 27 | } 28 | 29 | #[derive(Debug, Deserialize)] 30 | pub struct LoggingConfig { 31 | pub level: String, 32 | pub format: String, 33 | } 34 | 35 | #[derive(Debug, Deserialize)] 36 | pub struct Config { 37 | pub server: ServerConfig, 38 | pub logging: LoggingConfig, 39 | } 40 | 41 | fn deserialize_ip_addr<'de, D>(deserializer: D) -> Result 42 | where 43 | D: serde::Deserializer<'de>, 44 | { 45 | let s = String::deserialize(deserializer)?; 46 | s.parse().map_err(serde::de::Error::custom) 47 | } 48 | 49 | impl Config { 50 | pub fn from_file>(path: P) -> Result> { 51 | let contents = fs::read_to_string(path)?; 52 | let config: Config = serde_yaml::from_str(&contents)?; 53 | Ok(config) 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /services/compactor/src/lib.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | pub mod config; 18 | pub mod rpc; 19 | pub mod server; 20 | -------------------------------------------------------------------------------- /services/compactor/src/rpc.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use bergloom_codegen::compactor::compactor_service_server::CompactorService; 18 | use bergloom_codegen::compactor::{EchoRequest, EchoResponse}; 19 | use bergloom_core::CompactionExecutor; 20 | use bergloom_core::executor::DataFusionExecutor; 21 | 22 | use bergloom_codegen::compactor::{ 23 | RewriteFilesRequest as PbRewriteFilesRequest, RewriteFilesResponse as PbRewriteFilesResponse, 24 | }; 25 | 26 | #[derive(Default)] 27 | pub struct CompactorServiceImpl; 28 | 29 | #[async_trait::async_trait] 30 | impl CompactorService for CompactorServiceImpl { 31 | async fn rewrite_files( 32 | &self, 33 | request: tonic::Request, 34 | ) -> std::result::Result, tonic::Status> { 35 | let request = request.into_inner(); 36 | let response = DataFusionExecutor::default() 37 | .rewrite_file_proto(request) 38 | .await 39 | .map_err(|e| { 40 | tracing::error!("Error processing request: {:?}", e); 41 | tonic::Status::internal(format!("Internal error: {}", e)) 42 | })?; 43 | Ok(tonic::Response::new(response)) 44 | } 45 | 46 | async fn echo( 47 | &self, 48 | request: tonic::Request, 49 | ) -> std::result::Result, tonic::Status> { 50 | tracing::info!("Echo request: {:?}", request); 51 | Ok(tonic::Response::new(EchoResponse { 52 | message: format!("Echo: {}", request.into_inner().message), 53 | })) 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /services/compactor/src/server.rs: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2025 BergLoom 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | use crate::rpc::CompactorServiceImpl; 18 | use bergloom_codegen::compactor::compactor_service_server::CompactorServiceServer; 19 | use std::net::SocketAddr; 20 | use tokio::task::JoinHandle; 21 | use tonic::transport::Server; 22 | 23 | pub async fn grpc_compactor_serve( 24 | listen_addr: SocketAddr, 25 | ) -> JoinHandle> { 26 | let compactor_srv = CompactorServiceImpl {}; 27 | 28 | let server = Server::builder() 29 | .add_service(CompactorServiceServer::new(compactor_srv)) 30 | .serve(listen_addr); 31 | 32 | tokio::spawn(server) 33 | } 34 | --------------------------------------------------------------------------------