1212
1313use std:: { io:: Cursor , sync:: Arc } ;
1414
15- use futures:: future:: { self , try_join_all} ;
15+ use futures:: future:: try_join_all;
1616use itertools:: Itertools ;
1717use manifest:: ManifestReader ;
1818use manifest_list:: read_snapshot;
1919use object_store:: { path:: Path , ObjectStore } ;
2020
21- use futures:: { stream, Stream , StreamExt , TryFutureExt , TryStreamExt } ;
21+ use futures:: { stream, StreamExt , TryFutureExt , TryStreamExt } ;
2222use iceberg_rust_spec:: util:: { self } ;
2323use iceberg_rust_spec:: {
2424 spec:: {
@@ -262,7 +262,8 @@ impl Table {
262262 manifests : & ' a [ ManifestListEntry ] ,
263263 filter : Option < Vec < bool > > ,
264264 sequence_number_range : ( Option < i64 > , Option < i64 > ) ,
265- ) -> Result < impl Stream < Item = Result < ( ManifestPath , ManifestEntry ) , Error > > + ' a , Error > {
265+ ) -> Result < impl Iterator < Item = Result < ( ManifestPath , ManifestEntry ) , Error > > + ' a , Error >
266+ {
266267 datafiles (
267268 self . object_store ( ) ,
268269 manifests,
@@ -279,7 +280,7 @@ impl Table {
279280 ) -> Result < bool , Error > {
280281 let manifests = self . manifests ( start, end) . await ?;
281282 let datafiles = self . datafiles ( & manifests, None , ( None , None ) ) . await ?;
282- datafiles
283+ stream :: iter ( datafiles)
283284 . try_any ( |entry| async move { !matches ! ( entry. 1 . data_file( ) . content( ) , Content :: Data ) } )
284285 . await
285286 }
@@ -311,7 +312,7 @@ async fn datafiles(
311312 manifests : & ' _ [ ManifestListEntry ] ,
312313 filter : Option < Vec < bool > > ,
313314 sequence_number_range : ( Option < i64 > , Option < i64 > ) ,
314- ) -> Result < impl Stream < Item = Result < ( ManifestPath , ManifestEntry ) , Error > > + ' _ , Error > {
315+ ) -> Result < impl Iterator < Item = Result < ( ManifestPath , ManifestEntry ) , Error > > + ' _ , Error > {
315316 // filter manifest files according to filter vector
316317 let iter: Box < dyn Iterator < Item = & ManifestListEntry > + Send + Sync > = match filter {
317318 Some ( predicate) => {
@@ -345,31 +346,34 @@ async fn datafiles(
345346
346347 let results = try_join_all ( futures) . await ?;
347348
348- Ok ( stream :: iter ( results) . flat_map ( move |result| {
349+ Ok ( results. into_iter ( ) . flat_map ( move |result| {
349350 let ( bytes, path, sequence_number) = result;
350351
351352 let reader = ManifestReader :: new ( bytes) . unwrap ( ) ;
352- stream:: iter ( reader) . try_filter_map ( move |mut x| {
353- future:: ready ( {
354- let sequence_number = if let Some ( sequence_number) = x. sequence_number ( ) {
355- * sequence_number
356- } else {
357- * x. sequence_number_mut ( ) = Some ( sequence_number) ;
358- sequence_number
359- } ;
353+ reader. filter_map ( move |x| {
354+ let mut x = match x {
355+ Ok ( entry) => entry,
356+ Err ( _) => return None ,
357+ } ;
360358
361- let filter = match sequence_number_range {
362- ( Some ( start) , Some ( end) ) => start < sequence_number && sequence_number <= end,
363- ( Some ( start) , None ) => start < sequence_number,
364- ( None , Some ( end) ) => sequence_number <= end,
365- _ => true ,
366- } ;
367- if filter {
368- Ok ( Some ( ( path. to_owned ( ) , x) ) )
369- } else {
370- Ok ( None )
371- }
372- } )
359+ let sequence_number = if let Some ( sequence_number) = x. sequence_number ( ) {
360+ * sequence_number
361+ } else {
362+ * x. sequence_number_mut ( ) = Some ( sequence_number) ;
363+ sequence_number
364+ } ;
365+
366+ let filter = match sequence_number_range {
367+ ( Some ( start) , Some ( end) ) => start < sequence_number && sequence_number <= end,
368+ ( Some ( start) , None ) => start < sequence_number,
369+ ( None , Some ( end) ) => sequence_number <= end,
370+ _ => true ,
371+ } ;
372+ if filter {
373+ Some ( Ok ( ( path. to_owned ( ) , x) ) )
374+ } else {
375+ None
376+ }
373377 } )
374378 } ) )
375379}
@@ -390,7 +394,7 @@ pub(crate) async fn delete_all_table_files(
390394 let snapshots = & metadata. snapshots ;
391395
392396 // stream::iter(datafiles.into_iter())
393- datafiles
397+ stream :: iter ( datafiles)
394398 . try_for_each_concurrent ( None , |datafile| {
395399 let object_store = object_store. clone ( ) ;
396400 async move {
0 commit comments