1
1
use std:: { collections:: HashMap , sync:: Arc } ;
2
2
3
- use etl:: types:: TableSchema ;
4
- use iceberg:: { Catalog , NamespaceIdent , TableCreation , TableIdent } ;
3
+ use arrow:: array:: RecordBatch ;
4
+ use etl:: {
5
+ error:: EtlResult ,
6
+ types:: { TableRow , TableSchema } ,
7
+ } ;
8
+ use iceberg:: {
9
+ Catalog , NamespaceIdent , TableCreation , TableIdent ,
10
+ table:: Table ,
11
+ transaction:: { ApplyTransactionAction , Transaction } ,
12
+ writer:: {
13
+ IcebergWriter , IcebergWriterBuilder ,
14
+ base_writer:: data_file_writer:: DataFileWriterBuilder ,
15
+ file_writer:: {
16
+ ParquetWriterBuilder ,
17
+ location_generator:: { DefaultFileNameGenerator , DefaultLocationGenerator } ,
18
+ } ,
19
+ } ,
20
+ } ;
5
21
use iceberg_catalog_rest:: { RestCatalog , RestCatalogConfig } ;
22
+ use parquet:: { basic:: Compression , file:: properties:: WriterProperties } ;
6
23
7
- use crate :: iceberg:: schema:: postgres_to_iceberg_schema;
24
+ use crate :: iceberg:: {
25
+ encoding:: rows_to_record_batch,
26
+ error:: { arrow_error_to_etl_error, iceberg_error_to_etl_error} ,
27
+ schema:: postgres_to_iceberg_schema,
28
+ } ;
8
29
9
30
/// Client for connecting to Iceberg data lakes.
10
31
#[ derive( Clone ) ]
@@ -14,10 +35,15 @@ pub struct IcebergClient {
14
35
15
36
impl IcebergClient {
16
37
/// Creates a new [IcebergClient] from a REST catalog URI and a warehouse name.
17
- pub fn new_with_rest_catalog ( catalog_uri : String , warehouse_name : String ) -> Self {
38
+ pub fn new_with_rest_catalog (
39
+ catalog_uri : String ,
40
+ warehouse_name : String ,
41
+ props : HashMap < String , String > ,
42
+ ) -> Self {
18
43
let catalog_config = RestCatalogConfig :: builder ( )
19
44
. uri ( catalog_uri)
20
45
. warehouse ( warehouse_name)
46
+ . props ( props)
21
47
. build ( ) ;
22
48
let catalog = RestCatalog :: new ( catalog_config) ;
23
49
IcebergClient {
@@ -92,4 +118,106 @@ impl IcebergClient {
92
118
let namespace_ident = NamespaceIdent :: from_strs ( namespace. split ( '.' ) ) ?;
93
119
self . catalog . drop_namespace ( & namespace_ident) . await
94
120
}
121
+
122
+ /// Load a table
123
+ pub async fn load_table (
124
+ & self ,
125
+ namespace : String ,
126
+ table_name : String ,
127
+ ) -> Result < iceberg:: table:: Table , iceberg:: Error > {
128
+ let namespace_ident = NamespaceIdent :: new ( namespace) ;
129
+ let table_ident = TableIdent :: new ( namespace_ident, table_name) ;
130
+ self . catalog . load_table ( & table_ident) . await
131
+ }
132
+
133
+ /// Insert table rows into the table in the destination
134
+ pub async fn insert_rows (
135
+ & self ,
136
+ namespace : String ,
137
+ table_name : String ,
138
+ table_rows : & [ TableRow ] ,
139
+ ) -> EtlResult < ( ) > {
140
+ let namespace_ident = NamespaceIdent :: new ( namespace) ;
141
+ let table_ident = TableIdent :: new ( namespace_ident, table_name) ;
142
+
143
+ let table = self
144
+ . catalog
145
+ . load_table ( & table_ident)
146
+ . await
147
+ . map_err ( iceberg_error_to_etl_error) ?;
148
+ let table_metadata = table. metadata ( ) ;
149
+ let iceberg_schema = table_metadata. current_schema ( ) ;
150
+
151
+ // Convert the actual Iceberg schema to Arrow schema using iceberg-rust's built-in converter
152
+ // This preserves field IDs properly for transaction-based writes
153
+ let arrow_schema = iceberg:: arrow:: schema_to_arrow_schema ( iceberg_schema)
154
+ . map_err ( iceberg_error_to_etl_error) ?;
155
+ let record_batch =
156
+ rows_to_record_batch ( table_rows, arrow_schema) . map_err ( arrow_error_to_etl_error) ?;
157
+
158
+ self . write_record_batch ( & table, record_batch)
159
+ . await
160
+ . map_err ( iceberg_error_to_etl_error) ?;
161
+
162
+ Ok ( ( ) )
163
+ }
164
+
165
+ async fn write_record_batch (
166
+ & self ,
167
+ table : & Table ,
168
+ record_batch : RecordBatch ,
169
+ ) -> Result < ( ) , iceberg:: Error > {
170
+ // Create Parquet writer properties
171
+ let writer_props = WriterProperties :: builder ( )
172
+ . set_compression ( Compression :: SNAPPY )
173
+ . build ( ) ;
174
+
175
+ // Create location and file name generators
176
+ let location_gen = DefaultLocationGenerator :: new ( table. metadata ( ) . clone ( ) ) ?;
177
+ let file_name_gen = DefaultFileNameGenerator :: new (
178
+ "data" . to_string ( ) ,
179
+ Some ( uuid:: Uuid :: new_v4 ( ) . to_string ( ) ) , // Add unique UUID for each file
180
+ iceberg:: spec:: DataFileFormat :: Parquet ,
181
+ ) ;
182
+
183
+ // Create Parquet writer builder
184
+ let parquet_writer_builder = ParquetWriterBuilder :: new (
185
+ writer_props,
186
+ table. metadata ( ) . current_schema ( ) . clone ( ) ,
187
+ table. file_io ( ) . clone ( ) ,
188
+ location_gen,
189
+ file_name_gen,
190
+ ) ;
191
+
192
+ // Create data file writer with empty partition (unpartitioned table)
193
+ let data_file_writer_builder = DataFileWriterBuilder :: new (
194
+ parquet_writer_builder,
195
+ None , // No partition value for unpartitioned tables
196
+ table. metadata ( ) . default_partition_spec_id ( ) ,
197
+ ) ;
198
+
199
+ // Build the writer
200
+ let mut data_file_writer = data_file_writer_builder. build ( ) . await ?;
201
+
202
+ // Write the record batch using Iceberg writer
203
+ data_file_writer. write ( record_batch. clone ( ) ) . await ?;
204
+
205
+ // Close writer and get data files
206
+ let data_files = data_file_writer. close ( ) . await ?;
207
+
208
+ // Create transaction and fast append action
209
+ let transaction = Transaction :: new ( table) ;
210
+ let append_action = transaction
211
+ . fast_append ( )
212
+ . with_check_duplicate ( false ) // Don't check duplicates for performance
213
+ . add_data_files ( data_files) ;
214
+
215
+ // Apply the append action to create updated transaction
216
+ let updated_transaction = append_action. apply ( transaction) ?;
217
+
218
+ // Commit the transaction to the catalog
219
+ let _updated_table = updated_transaction. commit ( & * self . catalog ) . await ?;
220
+
221
+ Ok ( ( ) )
222
+ }
95
223
}
0 commit comments