liurenjie1024 commented on code in PR #1511:
URL: https://github.com/apache/iceberg-rust/pull/1511#discussion_r2218651049


##########
crates/integrations/datafusion/src/physical_plan/write.rs:
##########
@@ -0,0 +1,262 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::any::Any;
+use std::fmt::{Debug, Formatter};
+use std::str::FromStr;
+use std::sync::Arc;
+
+use datafusion::arrow::array::{ArrayRef, RecordBatch, StringArray};
+use datafusion::arrow::datatypes::{
+    DataType, Field, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef,
+};
+use datafusion::common::Result as DFResult;
+use datafusion::error::DataFusionError;
+use datafusion::execution::{SendableRecordBatchStream, TaskContext};
+use datafusion::physical_expr::{EquivalenceProperties, Partitioning};
+use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
+use datafusion::physical_plan::{
+    DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, 
PlanProperties,
+    execute_input_stream,
+};
+use futures::StreamExt;
+use iceberg::arrow::schema_to_arrow_schema;
+use iceberg::spec::{
+    DataFileFormat, FormatVersion, PROPERTY_DEFAULT_FILE_FORMAT,
+    PROPERTY_DEFAULT_FILE_FORMAT_DEFAULT, serialize_data_file_to_json,
+};
+use iceberg::table::Table;
+use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder;
+use iceberg::writer::file_writer::ParquetWriterBuilder;
+use iceberg::writer::file_writer::location_generator::{
+    DefaultFileNameGenerator, DefaultLocationGenerator,
+};
+use iceberg::writer::{IcebergWriter, IcebergWriterBuilder};
+use iceberg::{Error, ErrorKind};
+use parquet::file::properties::WriterProperties;
+use uuid::Uuid;
+
+use crate::to_datafusion_error;
+
+pub(crate) struct IcebergWriteExec {
+    table: Table,
+    input: Arc<dyn ExecutionPlan>,
+    result_schema: ArrowSchemaRef,
+    plan_properties: PlanProperties,
+}
+
+impl IcebergWriteExec {
+    pub fn new(table: Table, input: Arc<dyn ExecutionPlan>, schema: 
ArrowSchemaRef) -> Self {
+        let plan_properties = Self::compute_properties(&input, schema);
+
+        Self {
+            table,
+            input,
+            result_schema: Self::make_result_schema(),
+            plan_properties,
+        }
+    }
+
+    fn compute_properties(
+        input: &Arc<dyn ExecutionPlan>,
+        schema: ArrowSchemaRef,
+    ) -> PlanProperties {
+        PlanProperties::new(
+            EquivalenceProperties::new(schema),
+            
Partitioning::UnknownPartitioning(input.output_partitioning().partition_count()),
+            input.pipeline_behavior(),
+            input.boundedness(),
+        )
+    }
+
+    // Create a record batch with serialized data files
+    fn make_result_batch(data_files: Vec<String>) -> DFResult<RecordBatch> {
+        let files_array = Arc::new(StringArray::from(data_files)) as ArrayRef;
+
+        RecordBatch::try_from_iter_with_nullable(vec![("data_files", 
files_array, false)]).map_err(
+            |e| DataFusionError::ArrowError(e, Some("Failed to make result 
batch".to_string())),
+        )
+    }
+
+    fn make_result_schema() -> ArrowSchemaRef {
+        // Define a schema.
+        Arc::new(ArrowSchema::new(vec![Field::new(
+            "data_files",
+            DataType::Utf8,
+            false,
+        )]))
+    }
+}
+
+impl Debug for IcebergWriteExec {
+    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
+        write!(f, "IcebergWriteExec")
+    }
+}
+
+impl DisplayAs for IcebergWriteExec {
+    fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> 
std::fmt::Result {
+        match t {
+            DisplayFormatType::Default => {
+                write!(f, "IcebergWriteExec: table={}", 
self.table.identifier())
+            }
+            DisplayFormatType::Verbose => {
+                write!(
+                    f,
+                    "IcebergWriteExec: table={}, result_schema={:?}",
+                    self.table.identifier(),
+                    self.result_schema
+                )
+            }
+            DisplayFormatType::TreeRender => {
+                write!(f, "IcebergWriteExec: table={}", 
self.table.identifier())
+            }
+        }
+    }
+}
+
+impl ExecutionPlan for IcebergWriteExec {
+    fn name(&self) -> &str {
+        "IcebergWriteExec"
+    }
+
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn properties(&self) -> &PlanProperties {
+        &self.plan_properties
+    }
+
+    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
+        vec![&self.input]
+    }
+
+    fn with_new_children(
+        self: Arc<Self>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> DFResult<Arc<dyn ExecutionPlan>> {
+        if children.len() != 1 {
+            return Err(DataFusionError::Internal(
+                "IcebergWriteExec expects exactly one child".to_string(),

Review Comment:
   ```suggestion
                   "IcebergWriteExec expects exactly one child, but provided {} 
".to_string(),
   ```



##########
crates/integrations/datafusion/src/physical_plan/write.rs:
##########
@@ -0,0 +1,262 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::any::Any;
+use std::fmt::{Debug, Formatter};
+use std::str::FromStr;
+use std::sync::Arc;
+
+use datafusion::arrow::array::{ArrayRef, RecordBatch, StringArray};
+use datafusion::arrow::datatypes::{
+    DataType, Field, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef,
+};
+use datafusion::common::Result as DFResult;
+use datafusion::error::DataFusionError;
+use datafusion::execution::{SendableRecordBatchStream, TaskContext};
+use datafusion::physical_expr::{EquivalenceProperties, Partitioning};
+use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
+use datafusion::physical_plan::{
+    DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, 
PlanProperties,
+    execute_input_stream,
+};
+use futures::StreamExt;
+use iceberg::arrow::schema_to_arrow_schema;
+use iceberg::spec::{
+    DataFileFormat, FormatVersion, PROPERTY_DEFAULT_FILE_FORMAT,
+    PROPERTY_DEFAULT_FILE_FORMAT_DEFAULT, serialize_data_file_to_json,
+};
+use iceberg::table::Table;
+use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder;
+use iceberg::writer::file_writer::ParquetWriterBuilder;
+use iceberg::writer::file_writer::location_generator::{
+    DefaultFileNameGenerator, DefaultLocationGenerator,
+};
+use iceberg::writer::{IcebergWriter, IcebergWriterBuilder};
+use iceberg::{Error, ErrorKind};
+use parquet::file::properties::WriterProperties;
+use uuid::Uuid;
+
+use crate::to_datafusion_error;
+
+pub(crate) struct IcebergWriteExec {
+    table: Table,
+    input: Arc<dyn ExecutionPlan>,
+    result_schema: ArrowSchemaRef,
+    plan_properties: PlanProperties,
+}
+
+impl IcebergWriteExec {
+    pub fn new(table: Table, input: Arc<dyn ExecutionPlan>, schema: 
ArrowSchemaRef) -> Self {
+        let plan_properties = Self::compute_properties(&input, schema);
+
+        Self {
+            table,
+            input,
+            result_schema: Self::make_result_schema(),
+            plan_properties,
+        }
+    }
+
+    fn compute_properties(
+        input: &Arc<dyn ExecutionPlan>,
+        schema: ArrowSchemaRef,
+    ) -> PlanProperties {
+        PlanProperties::new(
+            EquivalenceProperties::new(schema),
+            
Partitioning::UnknownPartitioning(input.output_partitioning().partition_count()),
+            input.pipeline_behavior(),
+            input.boundedness(),
+        )
+    }
+
+    // Create a record batch with serialized data files
+    fn make_result_batch(data_files: Vec<String>) -> DFResult<RecordBatch> {
+        let files_array = Arc::new(StringArray::from(data_files)) as ArrayRef;
+
+        RecordBatch::try_from_iter_with_nullable(vec![("data_files", 
files_array, false)]).map_err(

Review Comment:
   nit: Why not just `try_new` so that we could reuse the result of 
`make_result_schema`?



##########
crates/integrations/datafusion/src/physical_plan/write.rs:
##########
@@ -0,0 +1,262 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::any::Any;
+use std::fmt::{Debug, Formatter};
+use std::str::FromStr;
+use std::sync::Arc;
+
+use datafusion::arrow::array::{ArrayRef, RecordBatch, StringArray};
+use datafusion::arrow::datatypes::{
+    DataType, Field, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef,
+};
+use datafusion::common::Result as DFResult;
+use datafusion::error::DataFusionError;
+use datafusion::execution::{SendableRecordBatchStream, TaskContext};
+use datafusion::physical_expr::{EquivalenceProperties, Partitioning};
+use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
+use datafusion::physical_plan::{
+    DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, 
PlanProperties,
+    execute_input_stream,
+};
+use futures::StreamExt;
+use iceberg::arrow::schema_to_arrow_schema;
+use iceberg::spec::{
+    DataFileFormat, FormatVersion, PROPERTY_DEFAULT_FILE_FORMAT,
+    PROPERTY_DEFAULT_FILE_FORMAT_DEFAULT, serialize_data_file_to_json,
+};
+use iceberg::table::Table;
+use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder;
+use iceberg::writer::file_writer::ParquetWriterBuilder;
+use iceberg::writer::file_writer::location_generator::{
+    DefaultFileNameGenerator, DefaultLocationGenerator,
+};
+use iceberg::writer::{IcebergWriter, IcebergWriterBuilder};
+use iceberg::{Error, ErrorKind};
+use parquet::file::properties::WriterProperties;
+use uuid::Uuid;
+
+use crate::to_datafusion_error;
+
+pub(crate) struct IcebergWriteExec {
+    table: Table,
+    input: Arc<dyn ExecutionPlan>,
+    result_schema: ArrowSchemaRef,
+    plan_properties: PlanProperties,
+}
+
+impl IcebergWriteExec {
+    pub fn new(table: Table, input: Arc<dyn ExecutionPlan>, schema: 
ArrowSchemaRef) -> Self {
+        let plan_properties = Self::compute_properties(&input, schema);
+
+        Self {
+            table,
+            input,
+            result_schema: Self::make_result_schema(),
+            plan_properties,
+        }
+    }
+
+    fn compute_properties(
+        input: &Arc<dyn ExecutionPlan>,
+        schema: ArrowSchemaRef,
+    ) -> PlanProperties {
+        PlanProperties::new(
+            EquivalenceProperties::new(schema),
+            
Partitioning::UnknownPartitioning(input.output_partitioning().partition_count()),
+            input.pipeline_behavior(),
+            input.boundedness(),
+        )
+    }
+
+    // Create a record batch with serialized data files
+    fn make_result_batch(data_files: Vec<String>) -> DFResult<RecordBatch> {
+        let files_array = Arc::new(StringArray::from(data_files)) as ArrayRef;
+
+        RecordBatch::try_from_iter_with_nullable(vec![("data_files", 
files_array, false)]).map_err(
+            |e| DataFusionError::ArrowError(e, Some("Failed to make result 
batch".to_string())),
+        )
+    }
+
+    fn make_result_schema() -> ArrowSchemaRef {
+        // Define a schema.
+        Arc::new(ArrowSchema::new(vec![Field::new(
+            "data_files",
+            DataType::Utf8,
+            false,
+        )]))
+    }
+}
+
+impl Debug for IcebergWriteExec {
+    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
+        write!(f, "IcebergWriteExec")
+    }
+}
+
+impl DisplayAs for IcebergWriteExec {
+    fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> 
std::fmt::Result {
+        match t {
+            DisplayFormatType::Default => {
+                write!(f, "IcebergWriteExec: table={}", 
self.table.identifier())
+            }
+            DisplayFormatType::Verbose => {
+                write!(
+                    f,
+                    "IcebergWriteExec: table={}, result_schema={:?}",
+                    self.table.identifier(),
+                    self.result_schema
+                )
+            }
+            DisplayFormatType::TreeRender => {
+                write!(f, "IcebergWriteExec: table={}", 
self.table.identifier())
+            }
+        }
+    }
+}
+
+impl ExecutionPlan for IcebergWriteExec {
+    fn name(&self) -> &str {
+        "IcebergWriteExec"
+    }
+
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn properties(&self) -> &PlanProperties {
+        &self.plan_properties
+    }
+
+    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
+        vec![&self.input]
+    }
+
+    fn with_new_children(
+        self: Arc<Self>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> DFResult<Arc<dyn ExecutionPlan>> {
+        if children.len() != 1 {
+            return Err(DataFusionError::Internal(
+                "IcebergWriteExec expects exactly one child".to_string(),
+            ));
+        }
+
+        Ok(Arc::new(Self::new(
+            self.table.clone(),
+            Arc::clone(&children[0]),
+            self.schema(),
+        )))
+    }
+
+    fn execute(
+        &self,
+        partition: usize,
+        context: Arc<TaskContext>,
+    ) -> DFResult<SendableRecordBatchStream> {
+        // todo non-default partition spec?
+        let spec_id = self.table.metadata().default_partition_spec_id();
+        let partition_type = 
self.table.metadata().default_partition_type().clone();
+        let is_version_1 = self.table.metadata().format_version() == 
FormatVersion::V1;
+
+        // Check data file format
+        let file_format = DataFileFormat::from_str(
+            self.table
+                .metadata()
+                .properties()
+                .get(PROPERTY_DEFAULT_FILE_FORMAT)
+                .unwrap_or(&PROPERTY_DEFAULT_FILE_FORMAT_DEFAULT.to_string()),
+        )
+        .map_err(to_datafusion_error)?;
+        if file_format != DataFileFormat::Parquet {
+            return Err(to_datafusion_error(Error::new(
+                ErrorKind::FeatureUnsupported,
+                format!(
+                    "File format {} is not supported for insert_into yet!",
+                    file_format
+                ),
+            )));
+        }
+
+        // Create data file writer builder
+        let data_file_writer_builder = DataFileWriterBuilder::new(
+            ParquetWriterBuilder::new(

Review Comment:
   This should be `RollingFileWriter`



##########
crates/iceberg/src/spec/manifest/mod.rs:
##########
@@ -119,12 +119,45 @@ impl Manifest {
     }
 }
 
+/// Serialize a DataFile to a JSON string.
+pub fn serialize_data_file_to_json(
+    data_file: DataFile,
+    partition_type: &super::StructType,
+    is_version_1: bool,

Review Comment:
   We hould use `TableFormatVersion`



##########
crates/integrations/datafusion/src/physical_plan/commit.rs:
##########
@@ -0,0 +1,269 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::any::Any;
+use std::fmt::{Debug, Formatter};
+use std::sync::Arc;
+
+use datafusion::arrow::array::{Array, ArrayRef, RecordBatch, StringArray, 
UInt64Array};
+use datafusion::arrow::datatypes::{
+    DataType, Field, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef,
+};
+use datafusion::common::{DataFusionError, Result as DFResult};
+use datafusion::execution::{SendableRecordBatchStream, TaskContext};
+use datafusion::physical_expr::{EquivalenceProperties, Partitioning};
+use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
+use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
+use datafusion::physical_plan::{
+    DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, 
execute_stream_partitioned,
+};
+use futures::StreamExt;
+use iceberg::Catalog;
+use iceberg::spec::{DataFile, deserialize_data_file_from_json};
+use iceberg::table::Table;
+use iceberg::transaction::Transaction;
+
+use crate::to_datafusion_error;
+
+/// IcebergCommitExec is responsible for collecting results from multiple 
IcebergWriteExec
+/// instances and using Transaction::fast_append to commit the data files 
written.
+pub(crate) struct IcebergCommitExec {
+    table: Table,
+    catalog: Arc<dyn Catalog>,
+    write_plan: Arc<dyn ExecutionPlan>,
+    schema: ArrowSchemaRef,
+    count_schema: ArrowSchemaRef,
+    plan_properties: PlanProperties,
+}
+
+impl IcebergCommitExec {
+    pub fn new(
+        table: Table,
+        catalog: Arc<dyn Catalog>,
+        write_plan: Arc<dyn ExecutionPlan>,
+        schema: ArrowSchemaRef,
+    ) -> Self {
+        let plan_properties = Self::compute_properties(schema.clone());
+
+        Self {
+            table,
+            catalog,
+            write_plan,
+            schema,
+            count_schema: Self::make_count_schema(),
+            plan_properties,
+        }
+    }
+
+    // Compute the plan properties for this execution plan
+    fn compute_properties(schema: ArrowSchemaRef) -> PlanProperties {
+        PlanProperties::new(
+            EquivalenceProperties::new(schema),
+            Partitioning::UnknownPartitioning(1),
+            EmissionType::Incremental,
+            Boundedness::Bounded,
+        )
+    }
+
+    // Create a record batch with just the count of rows written
+    fn make_count_batch(count: u64) -> DFResult<RecordBatch> {
+        let count_array = Arc::new(UInt64Array::from(vec![count])) as ArrayRef;
+
+        RecordBatch::try_from_iter_with_nullable(vec![("count", count_array, 
false)]).map_err(|e| {
+            DataFusionError::ArrowError(e, Some("Failed to make count 
batch!".to_string()))
+        })
+    }
+
+    fn make_count_schema() -> ArrowSchemaRef {
+        // Define a schema.
+        Arc::new(ArrowSchema::new(vec![Field::new(
+            "count",
+            DataType::UInt64,
+            false,
+        )]))
+    }
+}
+
+impl Debug for IcebergCommitExec {
+    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
+        write!(f, "IcebergCommitExec")
+    }
+}
+
+impl DisplayAs for IcebergCommitExec {
+    fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> 
std::fmt::Result {
+        match t {
+            DisplayFormatType::Default => {
+                write!(f, "IcebergCommitExec: table={}", 
self.table.identifier())
+            }
+            DisplayFormatType::Verbose => {
+                write!(
+                    f,
+                    "IcebergCommitExec: table={}, schema={:?}",
+                    self.table.identifier(),
+                    self.schema
+                )
+            }
+            DisplayFormatType::TreeRender => {
+                write!(f, "IcebergCommitExec: table={}", 
self.table.identifier())
+            }
+        }
+    }
+}
+
+impl ExecutionPlan for IcebergCommitExec {
+    fn name(&self) -> &str {
+        "IcebergCommitExec"
+    }
+
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn properties(&self) -> &PlanProperties {
+        &self.plan_properties
+    }
+
+    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
+        vec![&self.write_plan]
+    }
+
+    fn with_new_children(
+        self: Arc<Self>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> DFResult<Arc<dyn ExecutionPlan>> {
+        if children.len() != 1 {
+            return Err(DataFusionError::Internal(
+                "IcebergCommitExec expects exactly one child".to_string(),
+            ));
+        }
+
+        Ok(Arc::new(IcebergCommitExec::new(
+            self.table.clone(),
+            self.catalog.clone(),
+            children[0].clone(),
+            self.schema.clone(),
+        )))
+    }
+
+    fn execute(
+        &self,
+        partition: usize,
+        context: Arc<TaskContext>,
+    ) -> DFResult<SendableRecordBatchStream> {
+        // IcebergCommitExec only has one partition (partition 0)
+        if partition != 0 {
+            return Err(DataFusionError::Internal(format!(
+                "IcebergCommitExec only has one partition, but got partition 
{}",
+                partition
+            )));
+        }
+
+        let table = self.table.clone();
+        let input_plan = self.write_plan.clone();
+        let count_schema = Arc::clone(&self.count_schema);
+
+        // todo revisit this
+        let spec_id = self.table.metadata().default_partition_spec_id();
+        let partition_type = 
self.table.metadata().default_partition_type().clone();
+        let current_schema = self.table.metadata().current_schema().clone();
+
+        let _catalog = Arc::clone(&self.catalog);
+
+        // Process the input streams from all partitions and commit the data 
files
+        let stream = futures::stream::once(async move {
+            let mut data_files: Vec<DataFile> = Vec::new();
+            let mut total_record_count: u64 = 0;
+
+            // Execute and collect results from all partitions of the input 
plan
+            let batches = execute_stream_partitioned(input_plan, context)?;
+
+            // Collect all data files from this partition's stream
+            for mut batch_stream in batches {
+                while let Some(batch_result) = 
batch_stream.as_mut().next().await {
+                    let batch = batch_result?;
+
+                    let files_array = batch
+                        .column_by_name("data_files")

Review Comment:
   We should define these as constants



##########
crates/iceberg/src/arrow/nan_val_cnt_visitor.rs:
##########
@@ -159,6 +159,8 @@ impl NanValueCountVisitor {
         let arrow_arr_partner_accessor = ArrowArrayAccessor {};
 
         let struct_arr = Arc::new(StructArray::from(batch)) as ArrayRef;
+        println!("----StructArray from record stream: {:?}", struct_arr);
+        println!("----Schema.as_struct from table: {:?}", schema.as_struct());

Review Comment:
   We should use log here.



##########
crates/iceberg/src/arrow/value.rs:
##########
@@ -440,10 +440,12 @@ impl PartnerAccessor<ArrayRef> for ArrowArrayAccessor {
         Ok(schema_partner)
     }
 
+    // todo generate field_pos in datafusion instead of passing to here

Review Comment:
   Could you help me to understand why we need to change this?



##########
crates/integrations/datafusion/src/physical_plan/commit.rs:
##########
@@ -0,0 +1,269 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::any::Any;
+use std::fmt::{Debug, Formatter};
+use std::sync::Arc;
+
+use datafusion::arrow::array::{Array, ArrayRef, RecordBatch, StringArray, 
UInt64Array};
+use datafusion::arrow::datatypes::{
+    DataType, Field, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef,
+};
+use datafusion::common::{DataFusionError, Result as DFResult};
+use datafusion::execution::{SendableRecordBatchStream, TaskContext};
+use datafusion::physical_expr::{EquivalenceProperties, Partitioning};
+use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
+use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
+use datafusion::physical_plan::{
+    DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, 
execute_stream_partitioned,
+};
+use futures::StreamExt;
+use iceberg::Catalog;
+use iceberg::spec::{DataFile, deserialize_data_file_from_json};
+use iceberg::table::Table;
+use iceberg::transaction::Transaction;
+
+use crate::to_datafusion_error;
+
+/// IcebergCommitExec is responsible for collecting results from multiple 
IcebergWriteExec
+/// instances and using Transaction::fast_append to commit the data files 
written.
+pub(crate) struct IcebergCommitExec {
+    table: Table,
+    catalog: Arc<dyn Catalog>,
+    write_plan: Arc<dyn ExecutionPlan>,
+    schema: ArrowSchemaRef,
+    count_schema: ArrowSchemaRef,
+    plan_properties: PlanProperties,
+}
+
+impl IcebergCommitExec {
+    pub fn new(
+        table: Table,
+        catalog: Arc<dyn Catalog>,
+        write_plan: Arc<dyn ExecutionPlan>,
+        schema: ArrowSchemaRef,
+    ) -> Self {
+        let plan_properties = Self::compute_properties(schema.clone());
+
+        Self {
+            table,
+            catalog,
+            write_plan,
+            schema,
+            count_schema: Self::make_count_schema(),
+            plan_properties,
+        }
+    }
+
+    // Compute the plan properties for this execution plan
+    fn compute_properties(schema: ArrowSchemaRef) -> PlanProperties {
+        PlanProperties::new(
+            EquivalenceProperties::new(schema),
+            Partitioning::UnknownPartitioning(1),
+            EmissionType::Incremental,
+            Boundedness::Bounded,
+        )
+    }
+
+    // Create a record batch with just the count of rows written
+    fn make_count_batch(count: u64) -> DFResult<RecordBatch> {
+        let count_array = Arc::new(UInt64Array::from(vec![count])) as ArrayRef;
+
+        RecordBatch::try_from_iter_with_nullable(vec![("count", count_array, 
false)]).map_err(|e| {
+            DataFusionError::ArrowError(e, Some("Failed to make count 
batch!".to_string()))
+        })
+    }
+
+    fn make_count_schema() -> ArrowSchemaRef {
+        // Define a schema.
+        Arc::new(ArrowSchema::new(vec![Field::new(
+            "count",
+            DataType::UInt64,
+            false,
+        )]))
+    }
+}
+
+impl Debug for IcebergCommitExec {
+    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
+        write!(f, "IcebergCommitExec")
+    }
+}
+
+impl DisplayAs for IcebergCommitExec {
+    fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> 
std::fmt::Result {
+        match t {
+            DisplayFormatType::Default => {
+                write!(f, "IcebergCommitExec: table={}", 
self.table.identifier())
+            }
+            DisplayFormatType::Verbose => {
+                write!(
+                    f,
+                    "IcebergCommitExec: table={}, schema={:?}",
+                    self.table.identifier(),
+                    self.schema
+                )
+            }
+            DisplayFormatType::TreeRender => {
+                write!(f, "IcebergCommitExec: table={}", 
self.table.identifier())
+            }
+        }
+    }
+}
+
+impl ExecutionPlan for IcebergCommitExec {
+    fn name(&self) -> &str {
+        "IcebergCommitExec"
+    }
+
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn properties(&self) -> &PlanProperties {
+        &self.plan_properties
+    }
+
+    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
+        vec![&self.write_plan]
+    }
+
+    fn with_new_children(
+        self: Arc<Self>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> DFResult<Arc<dyn ExecutionPlan>> {
+        if children.len() != 1 {
+            return Err(DataFusionError::Internal(
+                "IcebergCommitExec expects exactly one child".to_string(),
+            ));
+        }
+
+        Ok(Arc::new(IcebergCommitExec::new(
+            self.table.clone(),
+            self.catalog.clone(),
+            children[0].clone(),
+            self.schema.clone(),
+        )))
+    }
+
+    fn execute(
+        &self,
+        partition: usize,
+        context: Arc<TaskContext>,
+    ) -> DFResult<SendableRecordBatchStream> {
+        // IcebergCommitExec only has one partition (partition 0)
+        if partition != 0 {
+            return Err(DataFusionError::Internal(format!(
+                "IcebergCommitExec only has one partition, but got partition 
{}",
+                partition
+            )));
+        }
+
+        let table = self.table.clone();
+        let input_plan = self.write_plan.clone();
+        let count_schema = Arc::clone(&self.count_schema);
+
+        // todo revisit this
+        let spec_id = self.table.metadata().default_partition_spec_id();
+        let partition_type = 
self.table.metadata().default_partition_type().clone();
+        let current_schema = self.table.metadata().current_schema().clone();
+
+        let _catalog = Arc::clone(&self.catalog);
+
+        // Process the input streams from all partitions and commit the data 
files
+        let stream = futures::stream::once(async move {
+            let mut data_files: Vec<DataFile> = Vec::new();
+            let mut total_record_count: u64 = 0;
+
+            // Execute and collect results from all partitions of the input 
plan
+            let batches = execute_stream_partitioned(input_plan, context)?;
+
+            // Collect all data files from this partition's stream
+            for mut batch_stream in batches {

Review Comment:
   We could insert an `CoalescePartitionsExec` to parallelize the execution of 
children



##########
crates/integrations/datafusion/src/physical_plan/commit.rs:
##########
@@ -0,0 +1,269 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::any::Any;
+use std::fmt::{Debug, Formatter};
+use std::sync::Arc;
+
+use datafusion::arrow::array::{Array, ArrayRef, RecordBatch, StringArray, 
UInt64Array};
+use datafusion::arrow::datatypes::{
+    DataType, Field, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef,
+};
+use datafusion::common::{DataFusionError, Result as DFResult};
+use datafusion::execution::{SendableRecordBatchStream, TaskContext};
+use datafusion::physical_expr::{EquivalenceProperties, Partitioning};
+use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
+use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
+use datafusion::physical_plan::{
+    DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, 
execute_stream_partitioned,
+};
+use futures::StreamExt;
+use iceberg::Catalog;
+use iceberg::spec::{DataFile, deserialize_data_file_from_json};
+use iceberg::table::Table;
+use iceberg::transaction::Transaction;
+
+use crate::to_datafusion_error;
+
+/// IcebergCommitExec is responsible for collecting results from multiple 
IcebergWriteExec
+/// instances and using Transaction::fast_append to commit the data files 
written.
+pub(crate) struct IcebergCommitExec {
+    table: Table,
+    catalog: Arc<dyn Catalog>,
+    write_plan: Arc<dyn ExecutionPlan>,
+    schema: ArrowSchemaRef,
+    count_schema: ArrowSchemaRef,
+    plan_properties: PlanProperties,
+}
+
+impl IcebergCommitExec {
+    pub fn new(
+        table: Table,
+        catalog: Arc<dyn Catalog>,
+        write_plan: Arc<dyn ExecutionPlan>,
+        schema: ArrowSchemaRef,
+    ) -> Self {
+        let plan_properties = Self::compute_properties(schema.clone());
+
+        Self {
+            table,
+            catalog,
+            write_plan,
+            schema,
+            count_schema: Self::make_count_schema(),
+            plan_properties,
+        }
+    }
+
+    // Compute the plan properties for this execution plan
+    fn compute_properties(schema: ArrowSchemaRef) -> PlanProperties {
+        PlanProperties::new(
+            EquivalenceProperties::new(schema),
+            Partitioning::UnknownPartitioning(1),
+            EmissionType::Incremental,
+            Boundedness::Bounded,
+        )
+    }
+
+    // Create a record batch with just the count of rows written
+    fn make_count_batch(count: u64) -> DFResult<RecordBatch> {
+        let count_array = Arc::new(UInt64Array::from(vec![count])) as ArrayRef;
+
+        RecordBatch::try_from_iter_with_nullable(vec![("count", count_array, 
false)]).map_err(|e| {
+            DataFusionError::ArrowError(e, Some("Failed to make count 
batch!".to_string()))
+        })
+    }
+
+    fn make_count_schema() -> ArrowSchemaRef {
+        // Define a schema.
+        Arc::new(ArrowSchema::new(vec![Field::new(
+            "count",
+            DataType::UInt64,
+            false,
+        )]))
+    }
+}
+
+impl Debug for IcebergCommitExec {
+    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
+        write!(f, "IcebergCommitExec")
+    }
+}
+
+impl DisplayAs for IcebergCommitExec {
+    fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> 
std::fmt::Result {
+        match t {
+            DisplayFormatType::Default => {
+                write!(f, "IcebergCommitExec: table={}", 
self.table.identifier())
+            }
+            DisplayFormatType::Verbose => {
+                write!(
+                    f,
+                    "IcebergCommitExec: table={}, schema={:?}",
+                    self.table.identifier(),
+                    self.schema
+                )
+            }
+            DisplayFormatType::TreeRender => {
+                write!(f, "IcebergCommitExec: table={}", 
self.table.identifier())
+            }
+        }
+    }
+}
+
+impl ExecutionPlan for IcebergCommitExec {
+    fn name(&self) -> &str {
+        "IcebergCommitExec"
+    }
+
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn properties(&self) -> &PlanProperties {
+        &self.plan_properties
+    }
+
+    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
+        vec![&self.write_plan]
+    }
+
+    fn with_new_children(
+        self: Arc<Self>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> DFResult<Arc<dyn ExecutionPlan>> {
+        if children.len() != 1 {
+            return Err(DataFusionError::Internal(
+                "IcebergCommitExec expects exactly one child".to_string(),
+            ));
+        }
+
+        Ok(Arc::new(IcebergCommitExec::new(
+            self.table.clone(),
+            self.catalog.clone(),
+            children[0].clone(),
+            self.schema.clone(),
+        )))
+    }
+
+    fn execute(
+        &self,
+        partition: usize,
+        context: Arc<TaskContext>,
+    ) -> DFResult<SendableRecordBatchStream> {
+        // IcebergCommitExec only has one partition (partition 0)
+        if partition != 0 {
+            return Err(DataFusionError::Internal(format!(
+                "IcebergCommitExec only has one partition, but got partition 
{}",
+                partition
+            )));
+        }
+
+        let table = self.table.clone();
+        let input_plan = self.write_plan.clone();
+        let count_schema = Arc::clone(&self.count_schema);
+
+        // todo revisit this
+        let spec_id = self.table.metadata().default_partition_spec_id();
+        let partition_type = 
self.table.metadata().default_partition_type().clone();
+        let current_schema = self.table.metadata().current_schema().clone();
+
+        let _catalog = Arc::clone(&self.catalog);
+
+        // Process the input streams from all partitions and commit the data 
files
+        let stream = futures::stream::once(async move {
+            let mut data_files: Vec<DataFile> = Vec::new();
+            let mut total_record_count: u64 = 0;
+
+            // Execute and collect results from all partitions of the input 
plan
+            let batches = execute_stream_partitioned(input_plan, context)?;
+
+            // Collect all data files from this partition's stream
+            for mut batch_stream in batches {
+                while let Some(batch_result) = 
batch_stream.as_mut().next().await {
+                    let batch = batch_result?;
+
+                    let files_array = batch
+                        .column_by_name("data_files")
+                        .ok_or_else(|| {
+                            DataFusionError::Internal(
+                                "Expected 'data_files' column in input 
batch".to_string(),
+                            )
+                        })?
+                        .as_any()
+                        .downcast_ref::<StringArray>()
+                        .ok_or_else(|| {
+                            DataFusionError::Internal(
+                                "Expected 'data_files' column to be 
StringArray".to_string(),
+                            )
+                        })?;
+
+                    // todo remove log
+                    println!("files_array to deserialize: {:?}", files_array);
+
+                    // Deserialize all data files from the StringArray
+                    let batch_files: Vec<DataFile> = files_array
+                        .into_iter()
+                        .flatten()
+                        .map(|f| -> DFResult<DataFile> {
+                            // Parse JSON to DataFileSerde and convert to 
DataFile
+                            deserialize_data_file_from_json(
+                                f,
+                                spec_id,
+                                &partition_type,
+                                &current_schema,
+                            )
+                            .map_err(to_datafusion_error)
+                        })
+                        .collect::<datafusion::common::Result<_>>()?;
+
+                    // add record_counts from the current batch to total 
record count
+                    total_record_count += batch_files.iter().map(|f| 
f.record_count()).sum::<u64>();
+
+                    // Add all deserialized files to our collection
+                    data_files.extend(batch_files);
+                }
+            }
+
+            // If no data files were collected, return an empty result
+            if data_files.is_empty() {
+                return Ok(RecordBatch::new_empty(count_schema));
+            }
+
+            // Create a transaction and commit the data files
+            let tx = Transaction::new(&table);
+            let _action = tx.fast_append().add_data_files(data_files);
+
+            // todo uncomment this
+            // // Apply the action and commit the transaction
+            // let updated_table = action
+            //     .apply(tx)
+            //     .map_err(to_datafusion_error)?
+            //     .commit(catalog.as_ref())
+            //     .await
+            //     .map_err(to_datafusion_error)?;

Review Comment:
   Why comment out this?



##########
crates/integrations/datafusion/src/physical_plan/write.rs:
##########
@@ -0,0 +1,257 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::any::Any;
+use std::fmt::{Debug, Formatter};
+use std::str::FromStr;
+use std::sync::Arc;
+
+use datafusion::arrow::array::{ArrayRef, RecordBatch, StringArray, 
UInt64Array};
+use datafusion::arrow::datatypes::{
+    DataType, Field, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef,
+};
+use datafusion::common::Result as DFResult;
+use datafusion::error::DataFusionError;
+use datafusion::execution::{SendableRecordBatchStream, TaskContext};
+use datafusion::physical_expr::EquivalenceProperties;
+use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
+use datafusion::physical_plan::{
+    DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, 
PlanProperties,
+    execute_input_stream,
+};
+use futures::StreamExt;
+use iceberg::arrow::schema_to_arrow_schema;
+use iceberg::spec::{
+    DataFileFormat, FormatVersion, PROPERTY_DEFAULT_FILE_FORMAT,
+    PROPERTY_DEFAULT_FILE_FORMAT_DEFAULT, serialize_data_file_to_json,
+};
+use iceberg::table::Table;
+use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder;
+use iceberg::writer::file_writer::ParquetWriterBuilder;
+use iceberg::writer::file_writer::location_generator::{
+    DefaultFileNameGenerator, DefaultLocationGenerator,
+};
+use iceberg::writer::{CurrentFileStatus, IcebergWriter, IcebergWriterBuilder};
+use iceberg::{Error, ErrorKind};
+use parquet::file::properties::WriterProperties;
+use uuid::Uuid;
+
+use crate::to_datafusion_error;
+
+pub(crate) struct IcebergWriteExec {
+    table: Table,
+    input: Arc<dyn ExecutionPlan>,
+    result_schema: ArrowSchemaRef,
+    plan_properties: PlanProperties,
+}
+
+impl IcebergWriteExec {
+    pub fn new(table: Table, input: Arc<dyn ExecutionPlan>, schema: 
ArrowSchemaRef) -> Self {
+        let plan_properties = Self::compute_properties(&input, schema.clone());
+
+        Self {
+            table,
+            input,
+            result_schema: Self::make_result_schema(),
+            plan_properties,
+        }
+    }
+
+    fn compute_properties(
+        input: &Arc<dyn ExecutionPlan>,
+        schema: ArrowSchemaRef,
+    ) -> PlanProperties {
+        PlanProperties::new(
+            EquivalenceProperties::new(schema),
+            input.output_partitioning().clone(),
+            input.pipeline_behavior(),

Review Comment:
   I'm not quite familiar with datafusion's streaming mode, but my suggestion 
is that we should not assume it's executed in streaming for now. We could 
always change this when we actually add streaming support.



##########
crates/integrations/datafusion/src/physical_plan/write.rs:
##########
@@ -0,0 +1,257 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::any::Any;
+use std::fmt::{Debug, Formatter};
+use std::str::FromStr;
+use std::sync::Arc;
+
+use datafusion::arrow::array::{ArrayRef, RecordBatch, StringArray, 
UInt64Array};
+use datafusion::arrow::datatypes::{
+    DataType, Field, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef,
+};
+use datafusion::common::Result as DFResult;
+use datafusion::error::DataFusionError;
+use datafusion::execution::{SendableRecordBatchStream, TaskContext};
+use datafusion::physical_expr::EquivalenceProperties;
+use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
+use datafusion::physical_plan::{
+    DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, 
PlanProperties,
+    execute_input_stream,
+};
+use futures::StreamExt;
+use iceberg::arrow::schema_to_arrow_schema;
+use iceberg::spec::{
+    DataFileFormat, FormatVersion, PROPERTY_DEFAULT_FILE_FORMAT,
+    PROPERTY_DEFAULT_FILE_FORMAT_DEFAULT, serialize_data_file_to_json,
+};
+use iceberg::table::Table;
+use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder;
+use iceberg::writer::file_writer::ParquetWriterBuilder;
+use iceberg::writer::file_writer::location_generator::{
+    DefaultFileNameGenerator, DefaultLocationGenerator,
+};
+use iceberg::writer::{CurrentFileStatus, IcebergWriter, IcebergWriterBuilder};
+use iceberg::{Error, ErrorKind};
+use parquet::file::properties::WriterProperties;
+use uuid::Uuid;
+
+use crate::to_datafusion_error;
+
+pub(crate) struct IcebergWriteExec {
+    table: Table,
+    input: Arc<dyn ExecutionPlan>,
+    result_schema: ArrowSchemaRef,
+    plan_properties: PlanProperties,
+}
+
+impl IcebergWriteExec {
+    pub fn new(table: Table, input: Arc<dyn ExecutionPlan>, schema: 
ArrowSchemaRef) -> Self {
+        let plan_properties = Self::compute_properties(&input, schema.clone());
+
+        Self {
+            table,
+            input,
+            result_schema: Self::make_result_schema(),
+            plan_properties,
+        }
+    }
+
+    fn compute_properties(
+        input: &Arc<dyn ExecutionPlan>,
+        schema: ArrowSchemaRef,
+    ) -> PlanProperties {
+        PlanProperties::new(
+            EquivalenceProperties::new(schema),
+            input.output_partitioning().clone(),
+            input.pipeline_behavior(),
+            input.boundedness(),

Review Comment:
   Ditto.



##########
crates/integrations/datafusion/src/physical_plan/commit.rs:
##########
@@ -0,0 +1,269 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use std::any::Any;
+use std::fmt::{Debug, Formatter};
+use std::sync::Arc;
+
+use datafusion::arrow::array::{Array, ArrayRef, RecordBatch, StringArray, 
UInt64Array};
+use datafusion::arrow::datatypes::{
+    DataType, Field, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef,
+};
+use datafusion::common::{DataFusionError, Result as DFResult};
+use datafusion::execution::{SendableRecordBatchStream, TaskContext};
+use datafusion::physical_expr::{EquivalenceProperties, Partitioning};
+use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
+use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
+use datafusion::physical_plan::{
+    DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, 
execute_stream_partitioned,
+};
+use futures::StreamExt;
+use iceberg::Catalog;
+use iceberg::spec::{DataFile, deserialize_data_file_from_json};
+use iceberg::table::Table;
+use iceberg::transaction::Transaction;
+
+use crate::to_datafusion_error;
+
+/// IcebergCommitExec is responsible for collecting results from multiple 
IcebergWriteExec
+/// instances and using Transaction::fast_append to commit the data files 
written.
+pub(crate) struct IcebergCommitExec {
+    table: Table,
+    catalog: Arc<dyn Catalog>,
+    write_plan: Arc<dyn ExecutionPlan>,
+    schema: ArrowSchemaRef,
+    count_schema: ArrowSchemaRef,
+    plan_properties: PlanProperties,
+}
+
+impl IcebergCommitExec {
+    pub fn new(
+        table: Table,
+        catalog: Arc<dyn Catalog>,
+        write_plan: Arc<dyn ExecutionPlan>,
+        schema: ArrowSchemaRef,
+    ) -> Self {
+        let plan_properties = Self::compute_properties(schema.clone());
+
+        Self {
+            table,
+            catalog,
+            write_plan,
+            schema,
+            count_schema: Self::make_count_schema(),
+            plan_properties,
+        }
+    }
+
+    // Compute the plan properties for this execution plan
+    fn compute_properties(schema: ArrowSchemaRef) -> PlanProperties {
+        PlanProperties::new(
+            EquivalenceProperties::new(schema),
+            Partitioning::UnknownPartitioning(1),
+            EmissionType::Incremental,
+            Boundedness::Bounded,
+        )
+    }
+
+    // Create a record batch with just the count of rows written
+    fn make_count_batch(count: u64) -> DFResult<RecordBatch> {
+        let count_array = Arc::new(UInt64Array::from(vec![count])) as ArrayRef;
+
+        RecordBatch::try_from_iter_with_nullable(vec![("count", count_array, 
false)]).map_err(|e| {
+            DataFusionError::ArrowError(e, Some("Failed to make count 
batch!".to_string()))
+        })
+    }
+
+    fn make_count_schema() -> ArrowSchemaRef {
+        // Define a schema.
+        Arc::new(ArrowSchema::new(vec![Field::new(
+            "count",
+            DataType::UInt64,
+            false,
+        )]))
+    }
+}
+
+impl Debug for IcebergCommitExec {
+    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
+        write!(f, "IcebergCommitExec")
+    }
+}
+
+impl DisplayAs for IcebergCommitExec {
+    fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> 
std::fmt::Result {
+        match t {
+            DisplayFormatType::Default => {
+                write!(f, "IcebergCommitExec: table={}", 
self.table.identifier())
+            }
+            DisplayFormatType::Verbose => {
+                write!(
+                    f,
+                    "IcebergCommitExec: table={}, schema={:?}",
+                    self.table.identifier(),
+                    self.schema
+                )
+            }
+            DisplayFormatType::TreeRender => {
+                write!(f, "IcebergCommitExec: table={}", 
self.table.identifier())
+            }
+        }
+    }
+}
+
+impl ExecutionPlan for IcebergCommitExec {
+    fn name(&self) -> &str {
+        "IcebergCommitExec"
+    }
+
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn properties(&self) -> &PlanProperties {
+        &self.plan_properties
+    }
+
+    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
+        vec![&self.write_plan]
+    }
+
+    fn with_new_children(
+        self: Arc<Self>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> DFResult<Arc<dyn ExecutionPlan>> {
+        if children.len() != 1 {
+            return Err(DataFusionError::Internal(
+                "IcebergCommitExec expects exactly one child".to_string(),

Review Comment:
   ```suggestion
                   "IcebergCommitExec expects exactly one child, but provided 
{children.len()}".to_string(),
   ```



-- 
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