44
55use std:: {
66 collections:: { HashMap , HashSet } ,
7+ future:: Future ,
78 io:: { Cursor , Read } ,
89 iter:: { repeat, Map , Repeat , Zip } ,
910 sync:: Arc ,
@@ -12,7 +13,7 @@ use std::{
1213use apache_avro:: {
1314 types:: Value as AvroValue , Reader as AvroReader , Schema as AvroSchema , Writer as AvroWriter ,
1415} ;
15- use futures:: { future:: join_all, TryStreamExt } ;
16+ use futures:: { future:: join_all, TryFutureExt , TryStreamExt } ;
1617use iceberg_rust_spec:: {
1718 manifest:: { partition_value_schema, DataFile , ManifestEntry , Status } ,
1819 manifest_list:: {
@@ -700,6 +701,24 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
700701 . await
701702 }
702703
704+ #[ inline]
705+ pub ( crate ) async fn append_concurrently (
706+ & mut self ,
707+ data_files : impl Iterator < Item = Result < ManifestEntry , Error > > ,
708+ snapshot_id : i64 ,
709+ object_store : Arc < dyn ObjectStore > ,
710+ content : Content ,
711+ ) -> Result < impl Future < Output = Result < ( ) , Error > > , Error > {
712+ self . append_filtered_concurrently (
713+ data_files,
714+ snapshot_id,
715+ None :: < fn ( & Result < ManifestEntry , Error > ) -> bool > ,
716+ object_store,
717+ content,
718+ )
719+ . await
720+ }
721+
703722 /// Appends data files to a single manifest with optional filtering and finalizes the manifest list.
704723 ///
705724 /// This method extends the basic `append` functionality by providing the ability to
@@ -754,6 +773,7 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
754773 /// object_store,
755774 /// ).await?;
756775 /// ```
776+ #[ inline]
757777 pub ( crate ) async fn append_filtered (
758778 & mut self ,
759779 data_files : impl Iterator < Item = Result < ManifestEntry , Error > > ,
@@ -762,6 +782,20 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
762782 object_store : Arc < dyn ObjectStore > ,
763783 content : Content ,
764784 ) -> Result < ( ) , Error > {
785+ self . append_filtered_concurrently ( data_files, snapshot_id, filter, object_store, content)
786+ . await ?
787+ . await ?;
788+ Ok ( ( ) )
789+ }
790+
791+ pub ( crate ) async fn append_filtered_concurrently (
792+ & mut self ,
793+ data_files : impl Iterator < Item = Result < ManifestEntry , Error > > ,
794+ snapshot_id : i64 ,
795+ filter : Option < impl Fn ( & Result < ManifestEntry , Error > ) -> bool > ,
796+ object_store : Arc < dyn ObjectStore > ,
797+ content : Content ,
798+ ) -> Result < impl Future < Output = Result < ( ) , Error > > , Error > {
765799 let selected_manifest = match content {
766800 Content :: Data => self . selected_data_manifest . take ( ) ,
767801 Content :: Deletes => self . selected_delete_manifest . take ( ) ,
@@ -820,11 +854,11 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
820854 manifest_writer. append ( manifest_entry?) ?;
821855 }
822856
823- let manifest = manifest_writer. finish ( object_store. clone ( ) ) . await ?;
857+ let ( manifest, future ) = manifest_writer. finish_concurrently ( object_store. clone ( ) ) ?;
824858
825859 self . writer . append_ser ( manifest) ?;
826860
827- Ok ( ( ) )
861+ Ok ( future )
828862 }
829863
830864 /// Appends data files by splitting them across multiple manifests and finalizes the manifest list.
@@ -875,15 +909,15 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
875909 /// object_store,
876910 /// ).await?;
877911 /// ```
878- pub ( crate ) async fn append_multiple (
912+ pub ( crate ) async fn append_multiple_concurrently (
879913 & mut self ,
880914 data_files : impl Iterator < Item = Result < ManifestEntry , Error > > ,
881915 snapshot_id : i64 ,
882916 n_splits : u32 ,
883917 object_store : Arc < dyn ObjectStore > ,
884918 content : Content ,
885- ) -> Result < ( ) , Error > {
886- self . append_multiple_filtered (
919+ ) -> Result < impl Future < Output = Result < ( ) , Error > > , Error > {
920+ self . append_multiple_filtered_concurrently (
887921 data_files,
888922 snapshot_id,
889923 n_splits,
@@ -955,6 +989,7 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
955989 /// object_store,
956990 /// ).await?;
957991 /// ```
992+ #[ inline]
958993 pub ( crate ) async fn append_multiple_filtered (
959994 & mut self ,
960995 data_files : impl Iterator < Item = Result < ManifestEntry , Error > > ,
@@ -964,6 +999,28 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
964999 object_store : Arc < dyn ObjectStore > ,
9651000 content : Content ,
9661001 ) -> Result < ( ) , Error > {
1002+ self . append_multiple_filtered_concurrently (
1003+ data_files,
1004+ snapshot_id,
1005+ n_splits,
1006+ filter,
1007+ object_store,
1008+ content,
1009+ )
1010+ . await ?
1011+ . await ?;
1012+ Ok ( ( ) )
1013+ }
1014+
1015+ pub ( crate ) async fn append_multiple_filtered_concurrently (
1016+ & mut self ,
1017+ data_files : impl Iterator < Item = Result < ManifestEntry , Error > > ,
1018+ snapshot_id : i64 ,
1019+ n_splits : u32 ,
1020+ filter : Option < impl Fn ( & Result < ManifestEntry , Error > ) -> bool > ,
1021+ object_store : Arc < dyn ObjectStore > ,
1022+ content : Content ,
1023+ ) -> Result < impl Future < Output = Result < ( ) , Error > > , Error > {
9671024 let partition_fields = self
9681025 . table_metadata
9691026 . current_partition_fields ( self . branch . as_deref ( ) ) ?;
@@ -1032,7 +1089,7 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
10321089 split_datafiles ( data_files, bounds, & partition_column_names, n_splits) ?
10331090 } ;
10341091
1035- let manifest_futures = splits
1092+ let ( manifests , manifest_futures) = splits
10361093 . into_iter ( )
10371094 . map ( |entries| {
10381095 let manifest_location = self . next_manifest_location ( ) ;
@@ -1050,17 +1107,17 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
10501107 manifest_writer. append ( manifest_entry) ?;
10511108 }
10521109
1053- Ok :: < _ , Error > ( manifest_writer. finish ( object_store. clone ( ) ) )
1110+ manifest_writer. finish_concurrently ( object_store. clone ( ) )
10541111 } )
1055- . collect :: < Result < Vec < _ > , _ > > ( ) ?;
1056-
1057- let manifests = futures:: future:: try_join_all ( manifest_futures) . await ?;
1112+ . collect :: < Result < ( Vec < _ > , Vec < _ > ) , _ > > ( ) ?;
10581113
10591114 for manifest in manifests {
10601115 self . writer . append_ser ( manifest) ?;
10611116 }
10621117
1063- Ok ( ( ) )
1118+ let future = futures:: future:: try_join_all ( manifest_futures) . map_ok ( |_| ( ) ) ;
1119+
1120+ Ok ( future)
10641121 }
10651122
10661123 pub ( crate ) async fn finish (
0 commit comments