blackmwk commented on code in PR #2296:
URL: https://github.com/apache/iceberg-rust/pull/2296#discussion_r3278384895


##########
crates/iceberg/src/transaction/snapshot.rs:
##########
@@ -170,23 +172,43 @@ impl<'a> SnapshotProducer<'a> {
             .map(|df| df.file_path.as_str())
             .collect();
 
-        let mut referenced_files = Vec::new();
-        if let Some(current_snapshot) = 
self.table.metadata().current_snapshot() {
-            let manifest_list = current_snapshot
-                .load_manifest_list(self.table.file_io(), 
&self.table.metadata_ref())
-                .await?;
-            for manifest_list_entry in manifest_list.entries() {
-                let manifest = manifest_list_entry
-                    .load_manifest(self.table.file_io())
-                    .await?;
-                for entry in manifest.entries() {
-                    let file_path = entry.file_path();
-                    if new_files.contains(file_path) && entry.is_alive() {
-                        referenced_files.push(file_path.to_string());
-                    }
-                }
-            }
-        }
+        let runtime = self.table.runtime().clone();
+        let file_io = self.table.file_io();
+        let metadata_ref = self.table.metadata_ref();
+
+        let referenced_files: Vec<String> =
+            
OptionFuture::from(self.table.metadata().current_snapshot().map(|snapshot| {
+                snapshot
+                    .load_manifest_list(file_io, &metadata_ref)
+                    .and_then(|manifest_list| {
+                        futures::stream::iter(
+                            manifest_list
+                                .consume_entries()
+                                .into_iter()
+                                .map(|entry| {
+                                    let file_io = file_io.clone();
+                                    runtime
+                                        .io()
+                                        .spawn(async move { 
entry.load_manifest(&file_io).await })
+                                })
+                                .collect::<Vec<_>>(),
+                        )
+                        .then(|handle| async move { handle.await? })

Review Comment:
   +1, this actually limited the concurrency.



##########
crates/iceberg/src/transaction/snapshot.rs:
##########
@@ -170,23 +172,43 @@ impl<'a> SnapshotProducer<'a> {
             .map(|df| df.file_path.as_str())
             .collect();
 
-        let mut referenced_files = Vec::new();
-        if let Some(current_snapshot) = 
self.table.metadata().current_snapshot() {
-            let manifest_list = current_snapshot
-                .load_manifest_list(self.table.file_io(), 
&self.table.metadata_ref())
-                .await?;
-            for manifest_list_entry in manifest_list.entries() {
-                let manifest = manifest_list_entry
-                    .load_manifest(self.table.file_io())
-                    .await?;
-                for entry in manifest.entries() {
-                    let file_path = entry.file_path();
-                    if new_files.contains(file_path) && entry.is_alive() {
-                        referenced_files.push(file_path.to_string());
-                    }
-                }
-            }
-        }
+        let runtime = self.table.runtime().clone();
+        let file_io = self.table.file_io();
+        let metadata_ref = self.table.metadata_ref();
+
+        let referenced_files: Vec<String> =
+            
OptionFuture::from(self.table.metadata().current_snapshot().map(|snapshot| {
+                snapshot
+                    .load_manifest_list(file_io, &metadata_ref)
+                    .and_then(|manifest_list| {
+                        futures::stream::iter(
+                            manifest_list
+                                .consume_entries()
+                                .into_iter()
+                                .map(|entry| {
+                                    let file_io = file_io.clone();
+                                    runtime
+                                        .io()
+                                        .spawn(async move { 
entry.load_manifest(&file_io).await })
+                                })
+                                .collect::<Vec<_>>(),
+                        )
+                        .then(|handle| async move { handle.await? })
+                        .try_fold(Vec::new(), |mut acc, manifest| {

Review Comment:
   This is unnecessarily complicated. The whole thing could be simplified as:
   ```
   stream.flat_map_unordered().filter().collect()
   ```



##########
crates/iceberg/src/transaction/snapshot.rs:
##########
@@ -170,23 +172,43 @@ impl<'a> SnapshotProducer<'a> {
             .map(|df| df.file_path.as_str())
             .collect();
 
-        let mut referenced_files = Vec::new();
-        if let Some(current_snapshot) = 
self.table.metadata().current_snapshot() {
-            let manifest_list = current_snapshot
-                .load_manifest_list(self.table.file_io(), 
&self.table.metadata_ref())
-                .await?;
-            for manifest_list_entry in manifest_list.entries() {
-                let manifest = manifest_list_entry
-                    .load_manifest(self.table.file_io())
-                    .await?;
-                for entry in manifest.entries() {
-                    let file_path = entry.file_path();
-                    if new_files.contains(file_path) && entry.is_alive() {
-                        referenced_files.push(file_path.to_string());
-                    }
-                }
-            }
-        }
+        let runtime = self.table.runtime().clone();
+        let file_io = self.table.file_io();
+        let metadata_ref = self.table.metadata_ref();
+
+        let referenced_files: Vec<String> =
+            
OptionFuture::from(self.table.metadata().current_snapshot().map(|snapshot| {
+                snapshot
+                    .load_manifest_list(file_io, &metadata_ref)
+                    .and_then(|manifest_list| {
+                        futures::stream::iter(
+                            manifest_list
+                                .consume_entries()
+                                .into_iter()
+                                .map(|entry| {
+                                    let file_io = file_io.clone();
+                                    runtime
+                                        .io()
+                                        .spawn(async move { 
entry.load_manifest(&file_io).await })
+                                })
+                                .collect::<Vec<_>>(),

Review Comment:
   Do we really need to collect to vec here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to