marvinlanhenke commented on code in PR #360:
URL: https://github.com/apache/iceberg-rust/pull/360#discussion_r1584100390


##########
crates/iceberg/src/scan.rs:
##########
@@ -314,6 +312,140 @@ impl TableScan {
     }
 }
 
+#[derive(Debug)]
+/// Holds the context necessary for file scanning operations
+/// in a streaming environment.
+struct FileScanStreamContext {
+    schema: SchemaRef,
+    snapshot: SnapshotRef,
+    table_metadata: TableMetadataRef,
+    file_io: FileIO,
+    filter: Option<Arc<Predicate>>,
+    case_sensitive: bool,
+}
+
+impl FileScanStreamContext {
+    /// Creates a new [`FileScanStreamContext`].
+    fn new(
+        schema: SchemaRef,
+        snapshot: SnapshotRef,
+        table_metadata: TableMetadataRef,
+        file_io: FileIO,
+        filter: Option<Arc<Predicate>>,
+        case_sensitive: bool,
+    ) -> Self {
+        Self {
+            schema,
+            snapshot,
+            table_metadata,
+            file_io,
+            filter,
+            case_sensitive,
+        }
+    }
+
+    /// Creates a [`BoundPredicate`] from row filter [`Predicate`].
+    fn bound_filter(&self) -> Result<Option<BoundPredicate>> {
+        match self.filter {
+            Some(ref filter) => Ok(Some(filter.bind(self.schema.clone(), 
self.case_sensitive)?)),
+            None => Ok(None),
+        }
+    }
+
+    /// Creates a reference-counted [`PartitionSpec`] and a
+    /// corresponding schema based on the specified partition spec id.
+    fn create_partition_spec_and_schema(
+        &self,
+        spec_id: i32,
+    ) -> Result<(&PartitionSpecRef, SchemaRef)> {

Review Comment:
   > Could we also do this up-front in `new` or just once
   
   I'm not sure we can do that, since we need the partition_spec_id from each 
`entry` inside the for-loop. However, the `FileScanStreamContext` need to be 
instantiated outside the stream. So I guess we can't "cache" the spec and the 
partition_schema?
   



-- 
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: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to