@@ -20,7 +20,7 @@ use crate::deltalake::config::DeltaTableConfig;
20
20
use crate :: deltalake:: events:: { materialize_events, materialize_events_append_only} ;
21
21
use crate :: deltalake:: maintenance:: TableMaintenanceState ;
22
22
use crate :: deltalake:: operations:: { append_to_table, delete_from_table, merge_to_table} ;
23
- use crate :: deltalake:: schema:: postgres_to_delta_schema;
23
+ use crate :: deltalake:: schema:: { postgres_to_arrow_schema , postgres_to_delta_schema} ;
24
24
25
25
/// Configuration for Delta Lake destination
26
26
#[ derive( Debug , Clone ) ]
@@ -111,10 +111,12 @@ where
111
111
} ) ?;
112
112
113
113
let table_name = & table_schema. name . name ;
114
- let table_path = parse_table_uri ( format ! ( "{}/{}" , self . config. base_uri, table_name) )
115
- . map_err ( |e| {
116
- etl_error ! ( ErrorKind :: DestinationError , "Failed to parse table path" , e)
117
- } ) ?;
114
+ let pg_table_schema = & table_schema. name . schema ;
115
+ let table_path = parse_table_uri ( format ! (
116
+ "{}/{}/{}" ,
117
+ self . config. base_uri, pg_table_schema, table_name
118
+ ) )
119
+ . map_err ( |e| etl_error ! ( ErrorKind :: DestinationError , "Failed to parse table path" , e) ) ?;
118
120
119
121
let mut table_builder = DeltaTableBuilder :: from_uri ( table_path) . map_err ( |e| {
120
122
etl_error ! (
@@ -245,14 +247,16 @@ where
245
247
. append_only ;
246
248
247
249
if is_append_only {
248
- let rows = materialize_events_append_only ( & events, & table_schema) ?;
249
- self . write_table_rows_internal ( & table_id, rows) . await ?;
250
+ let row_refs = materialize_events_append_only ( & events, & table_schema) ?;
251
+ let rows: Vec < PgTableRow > = row_refs. into_iter ( ) . cloned ( ) . collect ( ) ;
252
+ self . write_table_rows_internal ( & table_id, & rows) . await ?;
250
253
} else {
251
- let ( delete_predicates, rows) = materialize_events ( & events, & table_schema) ?;
254
+ let ( delete_predicates, row_refs) = materialize_events ( & events, & table_schema) ?;
255
+ let rows: Vec < PgTableRow > = row_refs. into_iter ( ) . cloned ( ) . collect ( ) ;
252
256
self . execute_delete_append_transaction_expr (
253
257
table_id,
254
258
& table_schema,
255
- rows,
259
+ & rows,
256
260
delete_predicates,
257
261
)
258
262
. await ?;
@@ -266,7 +270,7 @@ where
266
270
& self ,
267
271
table_id : TableId ,
268
272
table_schema : & PgTableSchema ,
269
- upsert_rows : Vec < & PgTableRow > ,
273
+ upsert_rows : & [ PgTableRow ] ,
270
274
delete_predicates : Vec < Expr > ,
271
275
) -> EtlResult < ( ) > {
272
276
let combined_predicate = delete_predicates. into_iter ( ) . reduce ( |acc, e| acc. or ( e) ) ;
@@ -347,29 +351,47 @@ where
347
351
async fn write_table_rows_internal (
348
352
& self ,
349
353
table_id : & TableId ,
350
- table_rows : Vec < & PgTableRow > ,
354
+ table_rows : & [ PgTableRow ] ,
351
355
) -> EtlResult < ( ) > {
352
356
if table_rows. is_empty ( ) {
353
357
return Ok ( ( ) ) ;
354
358
}
355
359
356
360
let table = self . table_handle ( table_id) . await ?;
357
361
362
+ let table_schema = self
363
+ . store
364
+ . get_table_schema ( table_id)
365
+ . await ?
366
+ . ok_or_else ( || {
367
+ etl_error ! (
368
+ ErrorKind :: MissingTableSchema ,
369
+ "Table schema not found" ,
370
+ format!( "Schema for table {} not found in store" , table_id. 0 )
371
+ )
372
+ } ) ?;
373
+
358
374
let row_length = table_rows. len ( ) ;
359
375
trace ! ( "Writing {} rows to Delta table" , row_length) ;
360
-
361
- let config = self . config_for_table_name ( & table_schema. name . name ) ;
376
+
362
377
let mut table_guard = table. lock ( ) . await ;
363
- let schema = table_guard. snapshot ( ) . schema ( ) ;
378
+ let arrow_schema = postgres_to_arrow_schema ( & table_schema) . map_err ( |e| {
379
+ etl_error ! (
380
+ ErrorKind :: ConversionError ,
381
+ "Failed to convert table schema to Arrow schema" ,
382
+ e
383
+ )
384
+ } ) ?;
364
385
365
- let record_batch =
366
- rows_to_record_batch ( table_rows. iter ( ) , table_schema. clone ( ) ) . map_err ( |e| {
367
- etl_error ! (
368
- ErrorKind :: ConversionError ,
369
- "Failed to encode table rows" ,
370
- format!( "Error converting to Arrow: {}" , e)
371
- )
372
- } ) ?;
386
+ let config = self . config_for_table_name ( & table_schema. name . name ) ;
387
+
388
+ let record_batch = rows_to_record_batch ( table_rows, arrow_schema. clone ( ) ) . map_err ( |e| {
389
+ etl_error ! (
390
+ ErrorKind :: ConversionError ,
391
+ "Failed to encode table rows" ,
392
+ format!( "Error converting to Arrow: {}" , e)
393
+ )
394
+ } ) ?;
373
395
append_to_table ( & mut table_guard, config. as_ref ( ) , record_batch)
374
396
. await
375
397
. map_err ( |e| {
@@ -450,8 +472,7 @@ where
450
472
table_id : TableId ,
451
473
table_rows : Vec < PgTableRow > ,
452
474
) -> EtlResult < ( ) > {
453
- self . write_table_rows_internal ( & table_id, table_rows. iter ( ) . collect ( ) )
454
- . await
475
+ self . write_table_rows_internal ( & table_id, & table_rows) . await
455
476
}
456
477
457
478
async fn write_events ( & self , events : Vec < Event > ) -> EtlResult < ( ) > {
0 commit comments