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


##########
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:
   Good point! Added `rolling_writer.rs`



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