liurenjie1024 commented on code in PR #1511: URL: https://github.com/apache/iceberg-rust/pull/1511#discussion_r2209883442
########## crates/integrations/datafusion/src/table/mod.rs: ########## @@ -46,6 +50,8 @@ pub struct IcebergTableProvider { snapshot_id: Option<i64>, /// A reference-counted arrow `Schema`. schema: ArrowSchemaRef, + /// The catalog that the table belongs to. + catalog: Option<Arc<dyn Catalog>>, Review Comment: We don't need to make this optional? ########## crates/iceberg/src/spec/manifest/mod.rs: ########## @@ -15,7 +15,9 @@ // specific language governing permissions and limitations // under the License. +// todo fix encapsulation mod _serde; +pub use _serde::*; Review Comment: I don't think we should expose this, a better approach is to expose a pub function to serializa/deserialize to json ########## crates/integrations/datafusion/src/physical_plan/write.rs: ########## @@ -0,0 +1,371 @@ +// 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::{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, Partitioning}; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, execute_input_stream, +}; +use futures::StreamExt; +use iceberg::arrow::schema_to_arrow_schema; +use iceberg::spec::{DataFileFormat, DataFileSerde, FormatVersion}; +use iceberg::table::Table; +use iceberg::writer::CurrentFileStatus; +use iceberg::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, +}; +use iceberg::writer::file_writer::{FileWriter, FileWriterBuilder, ParquetWriterBuilder}; +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(schema.clone()); + + Self { + table, + input, + result_schema: Self::make_result_schema(), + plan_properties, + } + } + + /// todo: Copied from scan.rs + fn compute_properties(schema: ArrowSchemaRef) -> PlanProperties { + // TODO: + // This is more or less a placeholder, to be replaced + // once we support output-partitioning + PlanProperties::new( + EquivalenceProperties::new(schema), + Partitioning::UnknownPartitioning(1), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } + + // Create a record batch with count and serialized data files + fn make_result_batch(count: u64, data_files: Vec<String>) -> DFResult<RecordBatch> { + let count_array = Arc::new(UInt64Array::from(vec![count])) as ArrayRef; + let files_array = Arc::new(StringArray::from(data_files)) as ArrayRef; + + RecordBatch::try_from_iter_with_nullable(vec![ + ("count", count_array, false), + ("data_files", files_array, false), + ]) + .map_err(|e| { + DataFusionError::ArrowError(e, Some("Failed to make result batch".to_string())) + }) + } + + fn make_result_schema() -> ArrowSchemaRef { Review Comment: This could be a lazy static field? ########## crates/integrations/datafusion/src/physical_plan/write.rs: ########## @@ -0,0 +1,371 @@ +// 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::{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, Partitioning}; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, execute_input_stream, +}; +use futures::StreamExt; +use iceberg::arrow::schema_to_arrow_schema; +use iceberg::spec::{DataFileFormat, DataFileSerde, FormatVersion}; +use iceberg::table::Table; +use iceberg::writer::CurrentFileStatus; +use iceberg::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, +}; +use iceberg::writer::file_writer::{FileWriter, FileWriterBuilder, ParquetWriterBuilder}; +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(schema.clone()); + + Self { + table, + input, + result_schema: Self::make_result_schema(), + plan_properties, + } + } + + /// todo: Copied from scan.rs + fn compute_properties(schema: ArrowSchemaRef) -> PlanProperties { + // TODO: + // This is more or less a placeholder, to be replaced + // once we support output-partitioning + PlanProperties::new( + EquivalenceProperties::new(schema), + Partitioning::UnknownPartitioning(1), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } + + // Create a record batch with count and serialized data files + fn make_result_batch(count: u64, data_files: Vec<String>) -> DFResult<RecordBatch> { + let count_array = Arc::new(UInt64Array::from(vec![count])) as ArrayRef; + let files_array = Arc::new(StringArray::from(data_files)) as ArrayRef; + + RecordBatch::try_from_iter_with_nullable(vec![ + ("count", count_array, false), + ("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), + Field::new("count", DataType::UInt64, 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![] + } + + fn with_new_children( + self: Arc<Self>, + _children: Vec<Arc<dyn ExecutionPlan>>, + ) -> DFResult<Arc<dyn ExecutionPlan>> { + Ok(self) + } + + fn execute( + &self, + partition: usize, + context: Arc<TaskContext>, + ) -> DFResult<SendableRecordBatchStream> { + let parquet_writer_fut = ParquetWriterBuilder::new( + WriterProperties::default(), + self.table.metadata().current_schema().clone(), + self.table.file_io().clone(), + DefaultLocationGenerator::new(self.table.metadata().clone()) + .map_err(to_datafusion_error)?, + // todo filename prefix/suffix should be configurable + DefaultFileNameGenerator::new( + "datafusion".to_string(), + Some(Uuid::now_v7().to_string()), + DataFileFormat::Parquet, + ), + ) + .build(); + + let data = execute_input_stream( + Arc::clone(&self.input), + Arc::new( + schema_to_arrow_schema(self.table.metadata().current_schema()) + .map_err(to_datafusion_error)?, + ), + partition, + Arc::clone(&context), + )?; + + // 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; + + let stream = futures::stream::once(async move { + let mut writer = parquet_writer_fut.await.map_err(to_datafusion_error)?; Review Comment: We should use `DataFileWriter` rather using `ParquetWriter` directly. ########## crates/integrations/datafusion/tests/integration_datafusion_test.rs: ########## @@ -432,3 +433,69 @@ async fn test_metadata_table() -> Result<()> { Ok(()) } + +#[tokio::test] +async fn test_insert_into() -> Result<()> { Review Comment: I'm not a big fan of adding this kind of integration tests. How about adding sqllogictests? ########## crates/integrations/datafusion/src/physical_plan/write.rs: ########## @@ -0,0 +1,371 @@ +// 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::{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, Partitioning}; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, execute_input_stream, +}; +use futures::StreamExt; +use iceberg::arrow::schema_to_arrow_schema; +use iceberg::spec::{DataFileFormat, DataFileSerde, FormatVersion}; +use iceberg::table::Table; +use iceberg::writer::CurrentFileStatus; +use iceberg::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, +}; +use iceberg::writer::file_writer::{FileWriter, FileWriterBuilder, ParquetWriterBuilder}; +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(schema.clone()); + + Self { + table, + input, + result_schema: Self::make_result_schema(), + plan_properties, + } + } + + /// todo: Copied from scan.rs + fn compute_properties(schema: ArrowSchemaRef) -> PlanProperties { + // TODO: + // This is more or less a placeholder, to be replaced + // once we support output-partitioning + PlanProperties::new( + EquivalenceProperties::new(schema), + Partitioning::UnknownPartitioning(1), Review Comment: The partition number should be same as input ########## crates/integrations/datafusion/src/physical_plan/write.rs: ########## @@ -0,0 +1,371 @@ +// 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::{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, Partitioning}; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, execute_input_stream, +}; +use futures::StreamExt; +use iceberg::arrow::schema_to_arrow_schema; +use iceberg::spec::{DataFileFormat, DataFileSerde, FormatVersion}; +use iceberg::table::Table; +use iceberg::writer::CurrentFileStatus; +use iceberg::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, +}; +use iceberg::writer::file_writer::{FileWriter, FileWriterBuilder, ParquetWriterBuilder}; +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(schema.clone()); + + Self { + table, + input, + result_schema: Self::make_result_schema(), + plan_properties, + } + } + + /// todo: Copied from scan.rs + fn compute_properties(schema: ArrowSchemaRef) -> PlanProperties { + // TODO: + // This is more or less a placeholder, to be replaced + // once we support output-partitioning + PlanProperties::new( + EquivalenceProperties::new(schema), + Partitioning::UnknownPartitioning(1), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } + + // Create a record batch with count and serialized data files + fn make_result_batch(count: u64, data_files: Vec<String>) -> DFResult<RecordBatch> { + let count_array = Arc::new(UInt64Array::from(vec![count])) as ArrayRef; + let files_array = Arc::new(StringArray::from(data_files)) as ArrayRef; + + RecordBatch::try_from_iter_with_nullable(vec![ + ("count", count_array, false), + ("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), + Field::new("count", DataType::UInt64, false), Review Comment: What's the meaning of `count`? ########## crates/integrations/datafusion/src/physical_plan/write.rs: ########## @@ -0,0 +1,371 @@ +// 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::{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, Partitioning}; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, execute_input_stream, +}; +use futures::StreamExt; +use iceberg::arrow::schema_to_arrow_schema; +use iceberg::spec::{DataFileFormat, DataFileSerde, FormatVersion}; +use iceberg::table::Table; +use iceberg::writer::CurrentFileStatus; +use iceberg::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, +}; +use iceberg::writer::file_writer::{FileWriter, FileWriterBuilder, ParquetWriterBuilder}; +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(schema.clone()); + + Self { + table, + input, + result_schema: Self::make_result_schema(), + plan_properties, + } + } + + /// todo: Copied from scan.rs + fn compute_properties(schema: ArrowSchemaRef) -> PlanProperties { + // TODO: + // This is more or less a placeholder, to be replaced + // once we support output-partitioning + PlanProperties::new( + EquivalenceProperties::new(schema), + Partitioning::UnknownPartitioning(1), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } + + // Create a record batch with count and serialized data files + fn make_result_batch(count: u64, data_files: Vec<String>) -> DFResult<RecordBatch> { + let count_array = Arc::new(UInt64Array::from(vec![count])) as ArrayRef; + let files_array = Arc::new(StringArray::from(data_files)) as ArrayRef; + + RecordBatch::try_from_iter_with_nullable(vec![ + ("count", count_array, false), + ("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), + Field::new("count", DataType::UInt64, 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![] + } + + fn with_new_children( + self: Arc<Self>, + _children: Vec<Arc<dyn ExecutionPlan>>, + ) -> DFResult<Arc<dyn ExecutionPlan>> { + Ok(self) + } + + fn execute( + &self, + partition: usize, + context: Arc<TaskContext>, + ) -> DFResult<SendableRecordBatchStream> { + let parquet_writer_fut = ParquetWriterBuilder::new( Review Comment: We should check table property that its data format is parquet, and throw error if not. -- 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