@@ -27,6 +27,14 @@ use uuid::Uuid;
2727
2828use crate :: error:: DataLoadingError ;
2929
30+ // Defines how to behave with existing tables
31+ #[ derive( Debug , Clone , PartialEq ) ]
32+ enum WriteMode {
33+ CreateExclusive , // Error out if the table already exists
34+ Overwrite , // Overwrite existing table data
35+ Append , // Append to existing table data
36+ }
37+
3038fn create_file_io ( target_url : String ) -> Result < FileIO , DataLoadingError > {
3139 let mut file_io_props: Vec < ( String , String ) > = vec ! [ ] ;
3240 if let Ok ( aws_endpoint) = std:: env:: var ( "AWS_ENDPOINT" ) {
@@ -96,14 +104,38 @@ fn update_metadata_snapshot(
96104 Ok ( new_metadata)
97105}
98106
107+ async fn get_manifest_files (
108+ file_io : & FileIO ,
109+ table_metadata : & TableMetadata ,
110+ ) -> Result < Option < Vec < ManifestFile > > , DataLoadingError > {
111+ let snapshot = match table_metadata. current_snapshot ( ) {
112+ None => return Ok ( None ) ,
113+ Some ( s) => s,
114+ } ;
115+ let manifest_list = snapshot. load_manifest_list ( file_io, table_metadata) . await ?;
116+ Ok ( Some ( manifest_list. consume_entries ( ) . into_iter ( ) . collect ( ) ) )
117+ }
118+
99119const DEFAULT_SCHEMA_ID : i32 = 0 ;
100120
101121pub async fn record_batches_to_iceberg (
102122 record_batch_stream : impl TryStream < Item = Result < RecordBatch , DataLoadingError > > ,
103123 arrow_schema : SchemaRef ,
104124 target_url : Url ,
105125 overwrite : bool ,
126+ append : bool ,
106127) -> Result < ( ) , DataLoadingError > {
128+ let write_mode = match ( overwrite, append) {
129+ ( false , false ) => WriteMode :: CreateExclusive ,
130+ ( true , false ) => WriteMode :: Overwrite ,
131+ ( false , true ) => WriteMode :: Append ,
132+ ( true , true ) => {
133+ return Err ( DataLoadingError :: BadInputError (
134+ "Cannot use overwrite flag with append flag" . to_string ( ) ,
135+ ) ) ;
136+ }
137+ } ;
138+
107139 pin_mut ! ( record_batch_stream) ;
108140
109141 let file_io = create_file_io ( target_url. to_string ( ) ) ?;
@@ -115,7 +147,7 @@ pub async fn record_batches_to_iceberg(
115147 let version_hint_location = format ! ( "{}/metadata/version-hint.text" , target_url) ;
116148 let version_hint_input = file_io. new_input ( & version_hint_location) ?;
117149 let old_version_hint: Option < u64 > = if version_hint_input. exists ( ) . await ? {
118- if !overwrite {
150+ if write_mode == WriteMode :: CreateExclusive {
119151 return Err ( DataLoadingError :: IoError ( std:: io:: Error :: other (
120152 "Table exists. Pass the overwrite flag to lakehouse-loader to overwrite data" ,
121153 ) ) ) ;
@@ -233,8 +265,20 @@ pub async fn record_batches_to_iceberg(
233265 } )
234266 . collect ( ) ,
235267 ) ;
236- let manifest_file: ManifestFile = manifest_writer. write ( manifest) . await ?;
237- info ! ( "Wrote manifest file: {:?}" , manifest_file. manifest_path) ;
268+ let new_manifest_file: ManifestFile = manifest_writer. write ( manifest) . await ?;
269+ info ! ( "Wrote manifest file: {:?}" , new_manifest_file. manifest_path) ;
270+
271+ let new_manifest_files_vec: Vec < ManifestFile > = match write_mode {
272+ WriteMode :: CreateExclusive | WriteMode :: Overwrite => vec ! [ new_manifest_file] , // Only include new manifest
273+ WriteMode :: Append => match get_manifest_files ( & file_io, & previous_metadata) . await ? {
274+ Some ( mut manifest_files) => {
275+ // Include new manifest and all manifests from previous snapshot
276+ manifest_files. push ( new_manifest_file) ;
277+ manifest_files
278+ }
279+ None => vec ! [ new_manifest_file] , // Only include new manifest
280+ } ,
281+ } ;
238282
239283 let manifest_list_path = format ! (
240284 "{}/metadata/manifest-list-{}.avro" ,
@@ -244,7 +288,7 @@ pub async fn record_batches_to_iceberg(
244288 let manifest_file_output = file_io. new_output ( manifest_list_path. clone ( ) ) ?;
245289 let mut manifest_list_writer: ManifestListWriter =
246290 ManifestListWriter :: v2 ( manifest_file_output, snapshot_id, None , sequence_number) ;
247- manifest_list_writer. add_manifests ( vec ! [ manifest_file ] . into_iter ( ) ) ?;
291+ manifest_list_writer. add_manifests ( new_manifest_files_vec . into_iter ( ) ) ?;
248292 manifest_list_writer. close ( ) . await ?;
249293 info ! ( "Wrote manifest list: {:?}" , manifest_list_path) ;
250294
0 commit comments