@@ -13,7 +13,7 @@ use std::{
1313use apache_avro:: {
1414 types:: Value as AvroValue , Reader as AvroReader , Schema as AvroSchema , Writer as AvroWriter ,
1515} ;
16- use futures:: { future:: join_all, TryStreamExt } ;
16+ use futures:: { future:: join_all, TryFutureExt , TryStreamExt } ;
1717use iceberg_rust_spec:: {
1818 manifest:: { partition_value_schema, DataFile , ManifestEntry , Status } ,
1919 manifest_list:: {
@@ -24,7 +24,7 @@ use iceberg_rust_spec::{
2424 table_metadata:: { FormatVersion , TableMetadata } ,
2525 util:: strip_prefix,
2626} ;
27- use object_store:: { ObjectStore , PutResult } ;
27+ use object_store:: ObjectStore ;
2828use smallvec:: SmallVec ;
2929
3030use crate :: {
@@ -708,7 +708,7 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
708708 snapshot_id : i64 ,
709709 object_store : Arc < dyn ObjectStore > ,
710710 content : Content ,
711- ) -> Result < impl Future < Output = Result < PutResult , Error > > , Error > {
711+ ) -> Result < impl Future < Output = Result < ( ) , Error > > , Error > {
712712 self . append_filtered_concurrently (
713713 data_files,
714714 snapshot_id,
@@ -795,7 +795,7 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
795795 filter : Option < impl Fn ( & Result < ManifestEntry , Error > ) -> bool > ,
796796 object_store : Arc < dyn ObjectStore > ,
797797 content : Content ,
798- ) -> Result < impl Future < Output = Result < PutResult , Error > > , Error > {
798+ ) -> Result < impl Future < Output = Result < ( ) , Error > > , Error > {
799799 let selected_manifest = match content {
800800 Content :: Data => self . selected_data_manifest . take ( ) ,
801801 Content :: Deletes => self . selected_delete_manifest . take ( ) ,
@@ -935,7 +935,7 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
935935 n_splits : u32 ,
936936 object_store : Arc < dyn ObjectStore > ,
937937 content : Content ,
938- ) -> Result < impl Future < Output = Result < Vec < PutResult > , Error > > , Error > {
938+ ) -> Result < impl Future < Output = Result < ( ) , Error > > , Error > {
939939 self . append_multiple_filtered_concurrently (
940940 data_files,
941941 snapshot_id,
@@ -1039,7 +1039,7 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
10391039 filter : Option < impl Fn ( & Result < ManifestEntry , Error > ) -> bool > ,
10401040 object_store : Arc < dyn ObjectStore > ,
10411041 content : Content ,
1042- ) -> Result < impl Future < Output = Result < Vec < PutResult > , Error > > , Error > {
1042+ ) -> Result < impl Future < Output = Result < ( ) , Error > > , Error > {
10431043 let partition_fields = self
10441044 . table_metadata
10451045 . current_partition_fields ( self . branch . as_deref ( ) ) ?;
@@ -1134,7 +1134,7 @@ impl<'schema, 'metadata> ManifestListWriter<'schema, 'metadata> {
11341134 self . writer . append_ser ( manifest) ?;
11351135 }
11361136
1137- let future = futures:: future:: try_join_all ( manifest_futures) ;
1137+ let future = futures:: future:: try_join_all ( manifest_futures) . map_ok ( |_| ( ) ) ;
11381138
11391139 Ok ( future)
11401140 }
0 commit comments