From 565f43d5f93ddf12a5c2bd68bca351d66fada370 Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Thu, 7 Mar 2024 21:15:51 +0100 Subject: [PATCH 01/18] close to compiling --- crates/core/src/operations/mod.rs | 4 +- crates/core/src/operations/write.rs | 294 ++++++++++++++++------------ python/deltalake/_internal.pyi | 1 + python/deltalake/writer.py | 1 + python/src/lib.rs | 5 +- 5 files changed, 177 insertions(+), 128 deletions(-) diff --git a/crates/core/src/operations/mod.rs b/crates/core/src/operations/mod.rs index 666b2dc66a..b5276ff5d3 100644 --- a/crates/core/src/operations/mod.rs +++ b/crates/core/src/operations/mod.rs @@ -137,8 +137,8 @@ impl DeltaOps { /// Write data to Delta table #[cfg(feature = "datafusion")] #[must_use] - pub fn write(self, batches: impl IntoIterator) -> WriteBuilder { - WriteBuilder::new(self.0.log_store, self.0.state).with_input_batches(batches) + pub fn write(self, batches: impl Iterator + Send + Sync, schema: arrow_schema::SchemaRef) -> WriteBuilder { + WriteBuilder::new(self.0.log_store, self.0.state).with_input_batches(batches).with_input_schema(schema) } /// Vacuum stale files from delta table diff --git a/crates/core/src/operations/write.rs b/crates/core/src/operations/write.rs index 03f219cb6f..b0bb14407e 100644 --- a/crates/core/src/operations/write.rs +++ b/crates/core/src/operations/write.rs @@ -21,9 +21,10 @@ //! let ids = arrow::array::Int32Array::from(vec![1, 2, 3, 4, 5]); //! let batch = RecordBatch::try_new(schema, vec![Arc::new(ids)])?; //! let ops = DeltaOps::try_from_uri("../path/to/empty/dir").await?; -//! let table = ops.write(vec![batch]).await?; +//! let table = ops.write(vec![batch], batch.schema()).await?; //! ```` +use std::borrow::BorrowMut; use std::collections::HashMap; use std::str::FromStr; use std::sync::Arc; @@ -41,6 +42,7 @@ use datafusion_common::DFSchema; use datafusion_expr::Expr; use futures::future::BoxFuture; use futures::StreamExt; +use itertools::Itertools; use parquet::file::properties::WriterProperties; use super::datafusion_utils::Expression; @@ -135,7 +137,9 @@ pub struct WriteBuilder { /// Number of records to be written in single batch to underlying writer write_batch_size: Option, /// RecordBatches to be written into the table - batches: Option>, + batches: Option + Send + Sync>>, + /// The schema of the batches to be written + input_schema: Option, /// whether to overwrite the schema or to merge it. None means to fail on schmema drift schema_mode: Option, /// how to handle cast failures, either return NULL (safe=true) or return ERR (safe=false) @@ -172,6 +176,7 @@ impl WriteBuilder { app_metadata: None, name: None, description: None, + input_schema: None, configuration: Default::default(), } } @@ -217,11 +222,18 @@ impl WriteBuilder { } /// Execution plan that produces the data to be written to the delta table - pub fn with_input_batches(mut self, batches: impl IntoIterator) -> Self { - self.batches = Some(batches.into_iter().collect()); + pub fn with_input_batches(mut self, batches: impl Iterator + Send + Sync) -> Self { + self.batches = Some(Box::new(batches)); self } + /// Schema of the + pub fn with_input_schema(mut self, schema: ArrowSchemaRef) -> Self { + self.input_schema = Some(schema); + self + } + + /// Specify the target file size for data files written to the delta table. pub fn with_target_file_size(mut self, target_file_size: usize) -> Self { self.target_file_size = Some(target_file_size); @@ -289,8 +301,8 @@ impl WriteBuilder { if let Some(plan) = &self.input { let schema: StructType = (plan.schema()).try_into()?; PROTOCOL.check_can_write_timestamp_ntz(snapshot, &schema)?; - } else if let Some(batches) = &self.batches { - let schema: StructType = (batches[0].schema()).try_into()?; + } else if let Some(schema) = &self.input_schema { + let schema: StructType = schema.as_ref().try_into()?; PROTOCOL.check_can_write_timestamp_ntz(snapshot, &schema)?; } @@ -304,11 +316,8 @@ impl WriteBuilder { None => { let schema: StructType = if let Some(plan) = &self.input { Ok(plan.schema().try_into()?) - } else if let Some(batches) = &self.batches { - if batches.is_empty() { - return Err(WriteError::MissingData.into()); - } - Ok(batches[0].schema().try_into()?) + } else if let Some(schema) = &self.input_schema { + Ok(schema.as_ref().try_into()?) } else { Err(WriteError::MissingData) }?; @@ -595,44 +604,82 @@ impl std::future::IntoFuture for WriteBuilder { Ok(this.partition_columns.unwrap_or_default()) }?; let mut schema_drift = false; - let plan = if let Some(plan) = this.input { - if this.schema_mode == Some(SchemaMode::Merge) { - return Err(DeltaTableError::Generic( - "Schema merge not supported yet for Datafusion".to_string(), - )); + let input_schema = if let Some(plan) = this.input { + plan.schema() + } else if let Some(schema) = this.input_schema { + schema + } else { + return Err(WriteError::MissingData.into()); + }; + + let mut new_schema = None; + if let Some(snapshot) = &this.snapshot { + let table_schema = snapshot + .physical_arrow_schema(this.log_store.object_store().clone()) + .await + .or_else(|_| snapshot.arrow_schema()) + .unwrap_or(input_schema.clone()); + + if let Err(schema_err) = + try_cast_batch(input_schema.fields(), table_schema.fields()) + { + schema_drift = true; + if this.mode == SaveMode::Overwrite && this.schema_mode.is_some() { + new_schema = None // we overwrite anyway, so no need to cast + } else if this.schema_mode == Some(SchemaMode::Merge) { + new_schema = Some(Arc::new(merge_schema( + table_schema.as_ref().clone(), + input_schema.as_ref().clone(), + )?)); + } else { + return Err(schema_err.into()); + } } - Ok(plan) - } else if let Some(batches) = this.batches { - if batches.is_empty() { - Err(WriteError::MissingData) - } else { - let schema = batches[0].schema(); - - let mut new_schema = None; - if let Some(snapshot) = &this.snapshot { - let table_schema = snapshot - .physical_arrow_schema(this.log_store.object_store().clone()) - .await - .or_else(|_| snapshot.arrow_schema()) - .unwrap_or(schema.clone()); - - if let Err(schema_err) = - try_cast_batch(schema.fields(), table_schema.fields()) - { - schema_drift = true; - if this.mode == SaveMode::Overwrite && this.schema_mode.is_some() { - new_schema = None // we overwrite anyway, so no need to cast - } else if this.schema_mode == Some(SchemaMode::Merge) { - new_schema = Some(Arc::new(merge_schema( - table_schema.as_ref().clone(), - schema.as_ref().clone(), - )?)); - } else { - return Err(schema_err.into()); - } + } + let target_schema = new_schema.clone().unwrap_or(input_schema.clone()); + + let state = match this.state { + Some(state) => state, + None => { + let ctx = SessionContext::new(); + register_store(this.log_store.clone(), ctx.runtime_env()); + ctx.state() + } + }; + + let (predicate_str, predicate) = match this.predicate { + Some(predicate) => { + let pred = match predicate { + Expression::DataFusion(expr) => expr, + Expression::String(s) => { + let df_schema = DFSchema::try_from(target_schema.as_ref().to_owned())?; + parse_predicate_expression(&df_schema, s, &state)? + // this.snapshot.unwrap().parse_predicate_expression(s, &state)? } - } + }; + (Some(fmt_expr_to_sql(&pred)?), Some(pred)) + } + _ => (None, None), + }; + if let Some(plan) = this.input { + let add_actions = write_execution_plan_with_predicate( + predicate.clone(), + this.snapshot.as_ref(), + state.clone(), + plan, + partition_columns.clone(), + this.log_store.object_store().clone(), + this.target_file_size, + this.write_batch_size, + this.writer_properties.clone(), + this.safe_cast, + this.schema_mode, + ) + .await?; + actions.extend(add_actions); + } else if let Some(input_batches) = this.batches { + for batches in chunks(input_batches, 10) { let data = if !partition_columns.is_empty() { // TODO partitioning should probably happen in its own plan ... let mut partitions: HashMap> = HashMap::new(); @@ -645,7 +692,7 @@ impl std::future::IntoFuture for WriteBuilder { }; let divided = divide_by_partition_values( - new_schema.clone().unwrap_or(schema.clone()), + target_schema.clone(), partition_columns.clone(), &real_batch, )?; @@ -680,19 +727,38 @@ impl std::future::IntoFuture for WriteBuilder { } }; - Ok(Arc::new(MemoryExec::try_new( + let plan = Arc::new(MemoryExec::try_new( &data, - new_schema.unwrap_or(schema).clone(), + target_schema.clone(), None, - )?) as Arc) + )?); + + let add_actions = write_execution_plan_with_predicate( + predicate.clone(), + this.snapshot.as_ref(), + state.clone(), + plan, + partition_columns.clone(), + this.log_store.object_store().clone(), + this.target_file_size, + this.write_batch_size, + this.writer_properties.clone(), + this.safe_cast, + this.schema_mode, + ) + .await?; + actions.extend(add_actions); } + } else { - Err(WriteError::MissingData) - }?; - let schema = plan.schema(); + return Err(WriteError::MissingData.into()) + }; + // Here we need to validate if the new data conforms to a predicate if one is provided + + if this.schema_mode == Some(SchemaMode::Merge) && schema_drift { if let Some(snapshot) = &this.snapshot { - let schema_struct: StructType = schema.clone().try_into()?; + let schema_struct: StructType = target_schema.clone().try_into()?; let schema_action = Action::Metadata(Metadata::try_new( schema_struct, partition_columns.clone(), @@ -701,47 +767,6 @@ impl std::future::IntoFuture for WriteBuilder { actions.push(schema_action); } } - let state = match this.state { - Some(state) => state, - None => { - let ctx = SessionContext::new(); - register_store(this.log_store.clone(), ctx.runtime_env()); - ctx.state() - } - }; - - let (predicate_str, predicate) = match this.predicate { - Some(predicate) => { - let pred = match predicate { - Expression::DataFusion(expr) => expr, - Expression::String(s) => { - let df_schema = DFSchema::try_from(schema.as_ref().to_owned())?; - parse_predicate_expression(&df_schema, s, &state)? - // this.snapshot.unwrap().parse_predicate_expression(s, &state)? - } - }; - (Some(fmt_expr_to_sql(&pred)?), Some(pred)) - } - _ => (None, None), - }; - - // Here we need to validate if the new data conforms to a predicate if one is provided - let add_actions = write_execution_plan_with_predicate( - predicate.clone(), - this.snapshot.as_ref(), - state.clone(), - plan, - partition_columns.clone(), - this.log_store.object_store().clone(), - this.target_file_size, - this.write_batch_size, - this.writer_properties.clone(), - this.safe_cast, - this.schema_mode, - ) - .await?; - actions.extend(add_actions); - // Collect remove actions if we are overwriting the table if let Some(snapshot) = &this.snapshot { if matches!(this.mode, SaveMode::Overwrite) { @@ -750,11 +775,11 @@ impl std::future::IntoFuture for WriteBuilder { .physical_arrow_schema(this.log_store.object_store().clone()) .await .or_else(|_| snapshot.arrow_schema()) - .unwrap_or(schema.clone()); + .unwrap_or(target_schema.clone()); - if schema != table_schema { + if target_schema != table_schema { let mut metadata = snapshot.metadata().clone(); - let delta_schema: StructType = schema.as_ref().try_into()?; + let delta_schema: StructType = target_schema.as_ref().try_into()?; metadata.schema_string = serde_json::to_string(&delta_schema)?; actions.push(Action::Metadata(metadata)); } @@ -825,6 +850,27 @@ impl std::future::IntoFuture for WriteBuilder { } } +fn chunks( + mut iter: Box + Send + Sync>, + chunk_size: usize, +) -> Box> + Send> { + Box::new(std::iter::from_fn(move || { + let mut chunk = Vec::with_capacity(chunk_size); + for _ in 0..chunk_size { + if let Some(batch) = iter.next() { + chunk.push(batch); + } else { + if chunk.is_empty() { + return None; + } else { + return Some(chunk); + } + } + } + Some(chunk) + })) +} + fn try_cast_batch(from_fields: &Fields, to_fields: &Fields) -> Result<(), ArrowError> { if from_fields.len() != to_fields.len() { return Err(ArrowError::SchemaError(format!( @@ -891,7 +937,7 @@ mod tests { let table = write_batch(table, batch.clone()).await; // Overwrite let _err = DeltaOps(table) - .write(vec![batch]) + .write(vec![batch].into_iter(), batch.schema()) .with_save_mode(SaveMode::Overwrite) .await .expect_err("Remove action is included when Delta table is append-only. Should error"); @@ -913,7 +959,7 @@ mod tests { // write some data let metadata = HashMap::from_iter(vec![("k1".to_string(), json!("v1.1"))]); let mut table = DeltaOps(table) - .write(vec![batch.clone()]) + .write(vec![batch].clone(), batch.schema()) .with_save_mode(SaveMode::Append) .with_metadata(metadata.clone()) .await @@ -936,7 +982,7 @@ mod tests { let metadata: HashMap = HashMap::from_iter(vec![("k1".to_string(), json!("v1.2"))]); let mut table = DeltaOps(table) - .write(vec![batch.clone()]) + .write(vec![batch].clone(), batch.schema()) .with_save_mode(SaveMode::Append) .with_metadata(metadata.clone()) .await @@ -959,7 +1005,7 @@ mod tests { let metadata: HashMap = HashMap::from_iter(vec![("k2".to_string(), json!("v2.1"))]); let mut table = DeltaOps(table) - .write(vec![batch]) + .write(vec![batch], batch.schema()) .with_save_mode(SaveMode::Overwrite) .with_metadata(metadata.clone()) .await @@ -995,7 +1041,7 @@ mod tests { vec![Arc::new(Int32Array::from(vec![Some(0), None]))], ) .unwrap(); - let table = DeltaOps::new_in_memory().write(vec![batch]).await.unwrap(); + let table = DeltaOps::new_in_memory().write(vec![batch], batch.schema()).await.unwrap(); let schema = Arc::new(ArrowSchema::new(vec![Field::new( "value", @@ -1015,7 +1061,7 @@ mod tests { // Test cast options let table = DeltaOps::from(table) - .write(vec![batch.clone()]) + .write(vec![batch].clone(), batch.schema()) .with_cast_safety(true) .await .unwrap(); @@ -1034,7 +1080,7 @@ mod tests { let actual = get_data(&table).await; assert_batches_sorted_eq!(&expected, &actual); - let res = DeltaOps::from(table).write(vec![batch]).await; + let res = DeltaOps::from(table).write(vec![batch], batch.schema()).await; assert!(res.is_err()); // Validate the datetime -> string behavior @@ -1051,7 +1097,7 @@ mod tests { )]))], ) .unwrap(); - let table = DeltaOps::new_in_memory().write(vec![batch]).await.unwrap(); + let table = DeltaOps::new_in_memory().write(vec![batch], batch.schema()).await.unwrap(); let schema = Arc::new(ArrowSchema::new(vec![Field::new( "value", @@ -1066,7 +1112,7 @@ mod tests { ) .unwrap(); - let _res = DeltaOps::from(table).write(vec![batch]).await.unwrap(); + let _res = DeltaOps::from(table).write(vec![batch], batch.schema()).await.unwrap(); let expected = [ "+--------------------------+", "| value |", @@ -1083,7 +1129,7 @@ mod tests { async fn test_write_nonexistent() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch]) + .write(vec![batch], batch.schema()) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1095,7 +1141,7 @@ mod tests { async fn test_write_partitioned() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch.clone()]) + .write(vec![batch].clone(), batch.schema()) .with_save_mode(SaveMode::ErrorIfExists) .with_partition_columns(["modified"]) .await @@ -1104,7 +1150,7 @@ mod tests { assert_eq!(table.get_files_count(), 2); let table = DeltaOps::new_in_memory() - .write(vec![batch]) + .write(vec![batch], batch.schema()) .with_save_mode(SaveMode::ErrorIfExists) .with_partition_columns(["modified", "id"]) .await @@ -1117,7 +1163,7 @@ mod tests { async fn test_merge_schema() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch.clone()]) + .write(vec![batch].clone(), batch.schema()) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1158,7 +1204,7 @@ mod tests { .unwrap(); let mut table = DeltaOps(table) - .write(vec![new_batch]) + .write(vec![new_batch], new_batch.schema()) .with_save_mode(SaveMode::Append) .with_schema_mode(SchemaMode::Merge) .await @@ -1175,7 +1221,7 @@ mod tests { async fn test_merge_schema_with_partitions() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch.clone()]) + .write(vec![batch].clone(), batch.schema()) .with_partition_columns(vec!["id", "value"]) .with_save_mode(SaveMode::ErrorIfExists) .await @@ -1217,7 +1263,7 @@ mod tests { .unwrap(); println!("new_batch: {:?}", new_batch.schema()); let table = DeltaOps(table) - .write(vec![new_batch]) + .write(vec![new_batch], new_batch.schema()) .with_save_mode(SaveMode::Append) .with_schema_mode(SchemaMode::Merge) .await @@ -1237,7 +1283,7 @@ mod tests { async fn test_overwrite_schema() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch.clone()]) + .write(vec![batch].clone(), batch.schema()) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1278,7 +1324,7 @@ mod tests { .unwrap(); let table = DeltaOps(table) - .write(vec![new_batch]) + .write(vec![new_batch], new_batch.schema()) .with_save_mode(SaveMode::Append) .with_schema_mode(SchemaMode::Overwrite) .await; @@ -1290,7 +1336,7 @@ mod tests { // If you do not pass a schema mode, we want to check the schema let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch.clone()]) + .write(vec![batch].clone(), batch.schema()) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1320,7 +1366,7 @@ mod tests { RecordBatch::try_new(Arc::new(new_schema), vec![Arc::new(inserted_by)]).unwrap(); let table = DeltaOps(table) - .write(vec![new_batch]) + .write(vec![new_batch], new_batch.schema()) .with_save_mode(SaveMode::Append) .await; assert!(table.is_err()); @@ -1348,7 +1394,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 0); - let table = DeltaOps(table).write(vec![batch.clone()]).await.unwrap(); + let table = DeltaOps(table).write(vec![batch].clone(), batch.schema()).await.unwrap(); assert_eq!(table.version(), 1); let schema: StructType = serde_json::from_value(json!({ @@ -1370,7 +1416,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 0); - let table = DeltaOps(table).write(vec![batch.clone()]).await; + let table = DeltaOps(table).write(vec![batch].clone(), batch.schema()).await; assert!(table.is_err()) } @@ -1387,7 +1433,7 @@ mod tests { assert_eq!(table.version(), 0); let table = DeltaOps(table) - .write(vec![batch.clone()]) + .write(vec![batch].clone(), batch.schema()) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -1468,7 +1514,7 @@ mod tests { .unwrap(); let table = DeltaOps::new_in_memory() - .write(vec![batch]) + .write(vec![batch], batch.schema()) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -1524,7 +1570,7 @@ mod tests { .unwrap(); let table = DeltaOps::new_in_memory() - .write(vec![batch]) + .write(vec![batch], batch.schema()) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -1565,7 +1611,7 @@ mod tests { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch]) + .write(vec![batch], batch.schema()) .with_partition_columns(["id", "value"]) .with_save_mode(SaveMode::Append) .await diff --git a/python/deltalake/_internal.pyi b/python/deltalake/_internal.pyi index b16d468571..709abd4ed0 100644 --- a/python/deltalake/_internal.pyi +++ b/python/deltalake/_internal.pyi @@ -171,6 +171,7 @@ def write_new_deltalake( def write_to_deltalake( table_uri: str, data: pyarrow.RecordBatchReader, + data_schema: pyarrow.Schema, partition_by: Optional[List[str]], mode: str, max_rows_per_group: int, diff --git a/python/deltalake/writer.py b/python/deltalake/writer.py index 4a61254a23..c01900fb0a 100644 --- a/python/deltalake/writer.py +++ b/python/deltalake/writer.py @@ -316,6 +316,7 @@ def write_deltalake( write_deltalake_rust( table_uri=table_uri, data=data, + data_schema=schema, partition_by=partition_by, mode=mode, max_rows_per_group=max_rows_per_group, diff --git a/python/src/lib.rs b/python/src/lib.rs index 5963a42216..65d18168a4 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1389,6 +1389,7 @@ impl From<&PyAddAction> for Add { fn write_to_deltalake( table_uri: String, data: PyArrowType, + data_schema: PyArrowType, mode: String, max_rows_per_group: i64, schema_mode: Option, @@ -1401,7 +1402,7 @@ fn write_to_deltalake( writer_properties: Option>>, custom_metadata: Option>, ) -> PyResult<()> { - let batches = data.0.map(|batch| batch.unwrap()).collect::>(); + let batches = data.0.map(|batch| batch.unwrap()).into_iter(); let save_mode = mode.parse().map_err(PythonError::from)?; let options = storage_options.clone().unwrap_or_default(); @@ -1412,7 +1413,7 @@ fn write_to_deltalake( .map_err(PythonError::from)?; let mut builder = table - .write(batches) + .write(batches, Arc::new(data_schema.0)) .with_save_mode(save_mode) .with_write_batch_size(max_rows_per_group as usize); if let Some(schema_mode) = schema_mode { From 3a52bb74539164024be0c35d931c8a9e7071db1c Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Fri, 8 Mar 2024 15:54:18 +0100 Subject: [PATCH 02/18] still learning :) --- crates/core/src/operations/mod.rs | 4 +- crates/core/src/operations/write.rs | 351 ++++++++++++++-------------- python/src/lib.rs | 4 +- 3 files changed, 174 insertions(+), 185 deletions(-) diff --git a/crates/core/src/operations/mod.rs b/crates/core/src/operations/mod.rs index b5276ff5d3..d0b98c14d4 100644 --- a/crates/core/src/operations/mod.rs +++ b/crates/core/src/operations/mod.rs @@ -137,8 +137,8 @@ impl DeltaOps { /// Write data to Delta table #[cfg(feature = "datafusion")] #[must_use] - pub fn write(self, batches: impl Iterator + Send + Sync, schema: arrow_schema::SchemaRef) -> WriteBuilder { - WriteBuilder::new(self.0.log_store, self.0.state).with_input_batches(batches).with_input_schema(schema) + pub fn write(self) -> WriteBuilder { + WriteBuilder::new(self.0.log_store, self.0.state) } /// Vacuum stale files from delta table diff --git a/crates/core/src/operations/write.rs b/crates/core/src/operations/write.rs index b0bb14407e..a775db2ba1 100644 --- a/crates/core/src/operations/write.rs +++ b/crates/core/src/operations/write.rs @@ -24,7 +24,6 @@ //! let table = ops.write(vec![batch], batch.schema()).await?; //! ```` -use std::borrow::BorrowMut; use std::collections::HashMap; use std::str::FromStr; use std::sync::Arc; @@ -42,7 +41,6 @@ use datafusion_common::DFSchema; use datafusion_expr::Expr; use futures::future::BoxFuture; use futures::StreamExt; -use itertools::Itertools; use parquet::file::properties::WriterProperties; use super::datafusion_utils::Expression; @@ -116,14 +114,25 @@ impl FromStr for SchemaMode { } } +/// Data you want to write to the file +pub enum WriteData { + /// A record batch with schema + RecordBatches( + ( + Box + Send>, + ArrowSchemaRef, + ), + ), + /// A Datafusion Execution plan + DataFusionPlan(Arc), +} + /// Write data into a DeltaTable pub struct WriteBuilder { /// A snapshot of the to-be-loaded table's state snapshot: Option, /// Delta object store for handling data files log_store: LogStoreRef, - /// The input plan - input: Option>, /// Datafusion session state relevant for executing the input plan state: Option, /// SaveMode defines how to treat data already written to table location @@ -136,10 +145,6 @@ pub struct WriteBuilder { target_file_size: Option, /// Number of records to be written in single batch to underlying writer write_batch_size: Option, - /// RecordBatches to be written into the table - batches: Option + Send + Sync>>, - /// The schema of the batches to be written - input_schema: Option, /// whether to overwrite the schema or to merge it. None means to fail on schmema drift schema_mode: Option, /// how to handle cast failures, either return NULL (safe=true) or return ERR (safe=false) @@ -162,21 +167,18 @@ impl WriteBuilder { Self { snapshot, log_store, - input: None, state: None, mode: SaveMode::Append, partition_columns: None, predicate: None, target_file_size: None, write_batch_size: None, - batches: None, safe_cast: false, schema_mode: None, writer_properties: None, app_metadata: None, name: None, description: None, - input_schema: None, configuration: Default::default(), } } @@ -209,30 +211,12 @@ impl WriteBuilder { self } - /// Execution plan that produces the data to be written to the delta table - pub fn with_input_execution_plan(mut self, plan: Arc) -> Self { - self.input = Some(plan); - self - } - /// A session state accompanying a given input plan, containing e.g. registered object stores pub fn with_input_session_state(mut self, state: SessionState) -> Self { self.state = Some(state); self } - /// Execution plan that produces the data to be written to the delta table - pub fn with_input_batches(mut self, batches: impl Iterator + Send + Sync) -> Self { - self.batches = Some(Box::new(batches)); - self - } - - /// Schema of the - pub fn with_input_schema(mut self, schema: ArrowSchemaRef) -> Self { - self.input_schema = Some(schema); - self - } - /// Specify the target file size for data files written to the delta table. pub fn with_target_file_size(mut self, target_file_size: usize) -> Self { @@ -292,54 +276,45 @@ impl WriteBuilder { .collect(); self } +} - async fn check_preconditions(&self) -> DeltaResult> { - match &self.snapshot { - Some(snapshot) => { - PROTOCOL.can_write_to(snapshot)?; - - if let Some(plan) = &self.input { - let schema: StructType = (plan.schema()).try_into()?; - PROTOCOL.check_can_write_timestamp_ntz(snapshot, &schema)?; - } else if let Some(schema) = &self.input_schema { - let schema: StructType = schema.as_ref().try_into()?; - PROTOCOL.check_can_write_timestamp_ntz(snapshot, &schema)?; - } - - match self.mode { - SaveMode::ErrorIfExists => { - Err(WriteError::AlreadyExists(self.log_store.root_uri()).into()) - } - _ => Ok(vec![]), +async fn check_preconditions( + builder: &WriteBuilder, + schema: ArrowSchemaRef, +) -> DeltaResult> { + match &builder.snapshot { + Some(snapshot) => { + PROTOCOL.can_write_to(snapshot)?; + + let schema: StructType = schema.try_into()?; + PROTOCOL.check_can_write_timestamp_ntz(snapshot, &schema)?; + match builder.mode { + SaveMode::ErrorIfExists => { + Err(WriteError::AlreadyExists(builder.log_store.root_uri()).into()) } + _ => Ok(vec![]), + } + } + None => { + let schema: StructType = schema.try_into()?; + let mut create_builder = CreateBuilder::new() + .with_log_store(builder.log_store.clone()) + .with_columns(schema.fields().clone()) + .with_configuration(builder.configuration.clone()); + if let Some(partition_columns) = builder.partition_columns.as_ref() { + create_builder = create_builder.with_partition_columns(partition_columns.clone()) } - None => { - let schema: StructType = if let Some(plan) = &self.input { - Ok(plan.schema().try_into()?) - } else if let Some(schema) = &self.input_schema { - Ok(schema.as_ref().try_into()?) - } else { - Err(WriteError::MissingData) - }?; - let mut builder = CreateBuilder::new() - .with_log_store(self.log_store.clone()) - .with_columns(schema.fields().clone()) - .with_configuration(self.configuration.clone()); - if let Some(partition_columns) = self.partition_columns.as_ref() { - builder = builder.with_partition_columns(partition_columns.clone()) - } - if let Some(name) = self.name.as_ref() { - builder = builder.with_table_name(name.clone()); - }; + if let Some(name) = builder.name.as_ref() { + create_builder = create_builder.with_table_name(name.clone()); + }; - if let Some(desc) = self.description.as_ref() { - builder = builder.with_comment(desc.clone()); - }; + if let Some(desc) = builder.description.as_ref() { + create_builder = create_builder.with_comment(desc.clone()); + }; - let (_, actions, _) = builder.into_table_and_actions()?; - Ok(actions) - } + let (_, actions, _) = create_builder.into_table_and_actions()?; + Ok(actions) } } } @@ -559,11 +534,8 @@ async fn prepare_predicate_actions( Ok(actions) } -impl std::future::IntoFuture for WriteBuilder { - type Output = DeltaResult; - type IntoFuture = BoxFuture<'static, Self::Output>; - - fn into_future(self) -> Self::IntoFuture { +impl WriteBuilder { + fn execute(self, input_data: WriteData) -> BoxFuture<'static, DeltaResult> { let this = self; Box::pin(async move { @@ -577,9 +549,13 @@ impl std::future::IntoFuture for WriteBuilder { "Schema overwrite not supported for Append".to_string(), )); } + let input_schema = match input_data { + WriteData::RecordBatches((_, schema)) => schema.clone(), + WriteData::DataFusionPlan(plan) => plan.as_ref().schema().clone(), + }; // Create table actions to initialize table in case it does not yet exist and should be created - let mut actions = this.check_preconditions().await?; + let mut actions = check_preconditions(&this, input_schema).await?; let active_partitions = this .snapshot @@ -604,14 +580,7 @@ impl std::future::IntoFuture for WriteBuilder { Ok(this.partition_columns.unwrap_or_default()) }?; let mut schema_drift = false; - let input_schema = if let Some(plan) = this.input { - plan.schema() - } else if let Some(schema) = this.input_schema { - schema - } else { - return Err(WriteError::MissingData.into()); - }; - + let mut new_schema = None; if let Some(snapshot) = &this.snapshot { let table_schema = snapshot @@ -637,7 +606,7 @@ impl std::future::IntoFuture for WriteBuilder { } } let target_schema = new_schema.clone().unwrap_or(input_schema.clone()); - + let state = match this.state { Some(state) => state, None => { @@ -662,77 +631,8 @@ impl std::future::IntoFuture for WriteBuilder { _ => (None, None), }; - if let Some(plan) = this.input { - let add_actions = write_execution_plan_with_predicate( - predicate.clone(), - this.snapshot.as_ref(), - state.clone(), - plan, - partition_columns.clone(), - this.log_store.object_store().clone(), - this.target_file_size, - this.write_batch_size, - this.writer_properties.clone(), - this.safe_cast, - this.schema_mode, - ) - .await?; - actions.extend(add_actions); - } else if let Some(input_batches) = this.batches { - for batches in chunks(input_batches, 10) { - let data = if !partition_columns.is_empty() { - // TODO partitioning should probably happen in its own plan ... - let mut partitions: HashMap> = HashMap::new(); - for batch in batches { - let real_batch = match new_schema.clone() { - Some(new_schema) => { - cast_record_batch(&batch, new_schema, false, true)? - } - None => batch, - }; - - let divided = divide_by_partition_values( - target_schema.clone(), - partition_columns.clone(), - &real_batch, - )?; - for part in divided { - let key = part.partition_values.hive_partition_path(); - match partitions.get_mut(&key) { - Some(part_batches) => { - part_batches.push(part.record_batch); - } - None => { - partitions.insert(key, vec![part.record_batch]); - } - } - } - } - partitions.into_values().collect::>() - } else { - match new_schema { - Some(ref new_schema) => { - let mut new_batches = vec![]; - for batch in batches { - new_batches.push(cast_record_batch( - &batch, - new_schema.clone(), - false, - true, - )?); - } - vec![new_batches] - } - None => vec![batches], - } - }; - - let plan = Arc::new(MemoryExec::try_new( - &data, - target_schema.clone(), - None, - )?); - + match input_data { + WriteData::DataFusionPlan(plan) => { let add_actions = write_execution_plan_with_predicate( predicate.clone(), this.snapshot.as_ref(), @@ -749,13 +649,78 @@ impl std::future::IntoFuture for WriteBuilder { .await?; actions.extend(add_actions); } - - } else { - return Err(WriteError::MissingData.into()) - }; + WriteData::RecordBatches((input_batches, _)) => { + for batches in ChunksIterator::new(input_batches, 10) { + let data = if !partition_columns.is_empty() { + // TODO partitioning should probably happen in its own plan ... + let mut partitions: HashMap> = HashMap::new(); + for batch in batches { + let real_batch = match new_schema.clone() { + Some(new_schema) => { + cast_record_batch(&batch, new_schema, false, true)? + } + None => batch, + }; + + let divided = divide_by_partition_values( + target_schema.clone(), + partition_columns.clone(), + &real_batch, + )?; + for part in divided { + let key = part.partition_values.hive_partition_path(); + match partitions.get_mut(&key) { + Some(part_batches) => { + part_batches.push(part.record_batch); + } + None => { + partitions.insert(key, vec![part.record_batch]); + } + } + } + } + partitions.into_values().collect::>() + } else { + match new_schema { + Some(ref new_schema) => { + let mut new_batches = vec![]; + for batch in batches { + new_batches.push(cast_record_batch( + &batch, + new_schema.clone(), + false, + true, + )?); + } + vec![new_batches] + } + None => vec![batches], + } + }; + + let plan = + Arc::new(MemoryExec::try_new(&data, target_schema.clone(), None)?); + + let add_actions = write_execution_plan_with_predicate( + predicate.clone(), + this.snapshot.as_ref(), + state.clone(), + plan, + partition_columns.clone(), + this.log_store.object_store().clone(), + this.target_file_size, + this.write_batch_size, + this.writer_properties.clone(), + this.safe_cast, + this.schema_mode, + ) + .await?; + actions.extend(add_actions); + } + } + } // Here we need to validate if the new data conforms to a predicate if one is provided - - + if this.schema_mode == Some(SchemaMode::Merge) && schema_drift { if let Some(snapshot) = &this.snapshot { let schema_struct: StructType = target_schema.clone().try_into()?; @@ -850,14 +815,22 @@ impl std::future::IntoFuture for WriteBuilder { } } -fn chunks( - mut iter: Box + Send + Sync>, +struct ChunksIterator { + iter: Box + Send>, chunk_size: usize, -) -> Box> + Send> { - Box::new(std::iter::from_fn(move || { - let mut chunk = Vec::with_capacity(chunk_size); - for _ in 0..chunk_size { - if let Some(batch) = iter.next() { +} +impl ChunksIterator { + fn new(iter: Box + Send>, chunk_size: usize) -> Self { + Self { iter, chunk_size } + } +} +impl Iterator for ChunksIterator { + type Item = Vec; + + fn next(&mut self) -> Option { + let mut chunk = Vec::with_capacity(self.chunk_size); + for _ in 0..self.chunk_size { + if let Some(batch) = self.iter.next() { chunk.push(batch); } else { if chunk.is_empty() { @@ -868,7 +841,7 @@ fn chunks( } } Some(chunk) - })) + } } fn try_cast_batch(from_fields: &Fields, to_fields: &Fields) -> Result<(), ArrowError> { @@ -1041,7 +1014,10 @@ mod tests { vec![Arc::new(Int32Array::from(vec![Some(0), None]))], ) .unwrap(); - let table = DeltaOps::new_in_memory().write(vec![batch], batch.schema()).await.unwrap(); + let table = DeltaOps::new_in_memory() + .write(vec![batch], batch.schema()) + .await + .unwrap(); let schema = Arc::new(ArrowSchema::new(vec![Field::new( "value", @@ -1080,7 +1056,9 @@ mod tests { let actual = get_data(&table).await; assert_batches_sorted_eq!(&expected, &actual); - let res = DeltaOps::from(table).write(vec![batch], batch.schema()).await; + let res = DeltaOps::from(table) + .write(vec![batch], batch.schema()) + .await; assert!(res.is_err()); // Validate the datetime -> string behavior @@ -1097,7 +1075,10 @@ mod tests { )]))], ) .unwrap(); - let table = DeltaOps::new_in_memory().write(vec![batch], batch.schema()).await.unwrap(); + let table = DeltaOps::new_in_memory() + .write(vec![batch], batch.schema()) + .await + .unwrap(); let schema = Arc::new(ArrowSchema::new(vec![Field::new( "value", @@ -1112,7 +1093,10 @@ mod tests { ) .unwrap(); - let _res = DeltaOps::from(table).write(vec![batch], batch.schema()).await.unwrap(); + let _res = DeltaOps::from(table) + .write(vec![batch], batch.schema()) + .await + .unwrap(); let expected = [ "+--------------------------+", "| value |", @@ -1394,7 +1378,10 @@ mod tests { .unwrap(); assert_eq!(table.version(), 0); - let table = DeltaOps(table).write(vec![batch].clone(), batch.schema()).await.unwrap(); + let table = DeltaOps(table) + .write(vec![batch].clone(), batch.schema()) + .await + .unwrap(); assert_eq!(table.version(), 1); let schema: StructType = serde_json::from_value(json!({ @@ -1416,7 +1403,9 @@ mod tests { .unwrap(); assert_eq!(table.version(), 0); - let table = DeltaOps(table).write(vec![batch].clone(), batch.schema()).await; + let table = DeltaOps(table) + .write(vec![batch].clone(), batch.schema()) + .await; assert!(table.is_err()) } @@ -1472,7 +1461,7 @@ mod tests { .unwrap(); let _table = ops - .write([batch.clone()]) + .write([batch.clone()], batch.schema()) .with_partition_columns(["string"]) .await .unwrap(); @@ -1531,7 +1520,7 @@ mod tests { .unwrap(); let table = DeltaOps(table) - .write(vec![batch_add]) + .write(vec![batch_add], batch_add.schema()) .with_save_mode(SaveMode::Overwrite) .with_replace_where(col("id").eq(lit("C"))) .await @@ -1593,7 +1582,7 @@ mod tests { .unwrap(); let table = DeltaOps(table) - .write(vec![batch_fail]) + .write(vec![batch_fail], batch_fail.schema()) .with_save_mode(SaveMode::Overwrite) .with_replace_where(col("id").eq(lit("C"))) .await; @@ -1633,7 +1622,7 @@ mod tests { .unwrap(); let table = DeltaOps(table) - .write(vec![batch_add]) + .write(vec![batch_add], batch_add.schema()) .with_save_mode(SaveMode::Overwrite) .with_replace_where(col("id").eq(lit("A"))) .await diff --git a/python/src/lib.rs b/python/src/lib.rs index 65d18168a4..2a79f104f7 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1413,7 +1413,7 @@ fn write_to_deltalake( .map_err(PythonError::from)?; let mut builder = table - .write(batches, Arc::new(data_schema.0)) + .write() .with_save_mode(save_mode) .with_write_batch_size(max_rows_per_group as usize); if let Some(schema_mode) = schema_mode { @@ -1452,7 +1452,7 @@ fn write_to_deltalake( }; rt()? - .block_on(builder.into_future()) + .block_on(builder.execte(batches, Arc::new(data_schema.0))) .map_err(PythonError::from)?; Ok(()) From 30a546331ac261126867abd4b1be444431a73648 Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Fri, 8 Mar 2024 21:14:40 +0100 Subject: [PATCH 03/18] some compile errors --- crates/benchmarks/src/bin/merge.rs | 7 +- crates/core/src/operations/mod.rs | 7 +- crates/core/src/operations/write.rs | 110 ++++++++++++++++------------ python/src/lib.rs | 4 +- 4 files changed, 77 insertions(+), 51 deletions(-) diff --git a/crates/benchmarks/src/bin/merge.rs b/crates/benchmarks/src/bin/merge.rs index ea43171052..e1010e7479 100644 --- a/crates/benchmarks/src/bin/merge.rs +++ b/crates/benchmarks/src/bin/merge.rs @@ -71,10 +71,13 @@ pub async fn convert_tpcds_web_returns(input_path: String, table_path: String) - .await .unwrap(); + let tbl = table.collect().await.unwrap(); + let schema = tbl[0].schema().clone(); + DeltaOps::try_from_uri(table_path) .await .unwrap() - .write(table.collect().await.unwrap()) + .write(Box::new(tbl.into_iter()), schema) .with_partition_columns(vec!["wr_returned_date_sk"]) .await .unwrap(); @@ -565,7 +568,7 @@ async fn main() { DeltaOps::try_from_uri(output) .await .unwrap() - .write(vec![batch]) + .write(Box::new(vec![batch].into_iter()), schema.clone()) .with_save_mode(SaveMode::Append) .await .unwrap(); diff --git a/crates/core/src/operations/mod.rs b/crates/core/src/operations/mod.rs index d0b98c14d4..b8aedb7277 100644 --- a/crates/core/src/operations/mod.rs +++ b/crates/core/src/operations/mod.rs @@ -137,8 +137,13 @@ impl DeltaOps { /// Write data to Delta table #[cfg(feature = "datafusion")] #[must_use] - pub fn write(self) -> WriteBuilder { + pub fn write( + self, + batches: Box + Send>, + schema: arrow_schema::SchemaRef, + ) -> WriteBuilder { WriteBuilder::new(self.0.log_store, self.0.state) + .with_data(write::WriteData::RecordBatches((batches, schema))) } /// Vacuum stale files from delta table diff --git a/crates/core/src/operations/write.rs b/crates/core/src/operations/write.rs index a775db2ba1..c40c6a51fa 100644 --- a/crates/core/src/operations/write.rs +++ b/crates/core/src/operations/write.rs @@ -117,18 +117,13 @@ impl FromStr for SchemaMode { /// Data you want to write to the file pub enum WriteData { /// A record batch with schema - RecordBatches( - ( - Box + Send>, - ArrowSchemaRef, - ), - ), + RecordBatches((Box + Send>, ArrowSchemaRef)), /// A Datafusion Execution plan DataFusionPlan(Arc), } -/// Write data into a DeltaTable -pub struct WriteBuilder { +/// Configuration for writing data to a DeltaTable +pub struct WriteBuilderConfig { /// A snapshot of the to-be-loaded table's state snapshot: Option, /// Delta object store for handling data files @@ -161,43 +156,60 @@ pub struct WriteBuilder { configuration: HashMap>, } +/// Write data into a DeltaTable +pub struct WriteBuilder { + /// Basic config for the write operation + config: WriteBuilderConfig, + /// The data to actually write + data: Option, +} + impl WriteBuilder { /// Create a new [`WriteBuilder`] pub fn new(log_store: LogStoreRef, snapshot: Option) -> Self { Self { - snapshot, - log_store, - state: None, - mode: SaveMode::Append, - partition_columns: None, - predicate: None, - target_file_size: None, - write_batch_size: None, - safe_cast: false, - schema_mode: None, - writer_properties: None, - app_metadata: None, - name: None, - description: None, - configuration: Default::default(), + config: WriteBuilderConfig { + snapshot, + log_store, + state: None, + mode: SaveMode::Append, + partition_columns: None, + predicate: None, + target_file_size: None, + write_batch_size: None, + safe_cast: false, + schema_mode: None, + writer_properties: None, + app_metadata: None, + name: None, + description: None, + configuration: Default::default(), + }, + data: None, } } + /// Specify the data to use to write this + pub fn with_data(mut self, data: WriteData) -> Self { + self.data = Some(data); + self + } + /// Specify the behavior when a table exists at location pub fn with_save_mode(mut self, save_mode: SaveMode) -> Self { - self.mode = save_mode; + self.config.mode = save_mode; self } /// Add Schema Write Mode pub fn with_schema_mode(mut self, schema_mode: SchemaMode) -> Self { - self.schema_mode = Some(schema_mode); + self.config.schema_mode = Some(schema_mode); self } /// When using `Overwrite` mode, replace data that matches a predicate pub fn with_replace_where(mut self, predicate: impl Into) -> Self { - self.predicate = Some(predicate.into()); + self.config.predicate = Some(predicate.into()); self } @@ -207,39 +219,39 @@ impl WriteBuilder { mut self, partition_columns: impl IntoIterator>, ) -> Self { - self.partition_columns = Some(partition_columns.into_iter().map(|s| s.into()).collect()); + self.config.partition_columns = + Some(partition_columns.into_iter().map(|s| s.into()).collect()); self } /// A session state accompanying a given input plan, containing e.g. registered object stores pub fn with_input_session_state(mut self, state: SessionState) -> Self { - self.state = Some(state); + self.config.state = Some(state); self } - /// Specify the target file size for data files written to the delta table. pub fn with_target_file_size(mut self, target_file_size: usize) -> Self { - self.target_file_size = Some(target_file_size); + self.config.target_file_size = Some(target_file_size); self } /// Specify the target batch size for row groups written to parquet files. pub fn with_write_batch_size(mut self, write_batch_size: usize) -> Self { - self.write_batch_size = Some(write_batch_size); + self.config.write_batch_size = Some(write_batch_size); self } /// Specify the safety of the casting operation /// how to handle cast failures, either return NULL (safe=true) or return ERR (safe=false) pub fn with_cast_safety(mut self, safe: bool) -> Self { - self.safe_cast = safe; + self.config.safe_cast = safe; self } /// Specify the writer properties to use when writing a parquet file pub fn with_writer_properties(mut self, writer_properties: WriterProperties) -> Self { - self.writer_properties = Some(writer_properties); + self.config.writer_properties = Some(writer_properties); self } @@ -248,20 +260,20 @@ impl WriteBuilder { mut self, metadata: impl IntoIterator, ) -> Self { - self.app_metadata = Some(HashMap::from_iter(metadata)); + self.config.app_metadata = Some(HashMap::from_iter(metadata)); self } /// Specify the table name. Optionally qualified with /// a database name [database_name.] table_name. pub fn with_table_name(mut self, name: impl Into) -> Self { - self.name = Some(name.into()); + self.config.name = Some(name.into()); self } /// Comment to describe the table. pub fn with_description(mut self, description: impl Into) -> Self { - self.description = Some(description.into()); + self.config.description = Some(description.into()); self } @@ -270,7 +282,7 @@ impl WriteBuilder { mut self, configuration: impl IntoIterator, Option>)>, ) -> Self { - self.configuration = configuration + self.config.configuration = configuration .into_iter() .map(|(k, v)| (k.into(), v.map(|s| s.into()))) .collect(); @@ -279,7 +291,7 @@ impl WriteBuilder { } async fn check_preconditions( - builder: &WriteBuilder, + builder: &WriteBuilderConfig, schema: ArrowSchemaRef, ) -> DeltaResult> { match &builder.snapshot { @@ -534,9 +546,13 @@ async fn prepare_predicate_actions( Ok(actions) } -impl WriteBuilder { - fn execute(self, input_data: WriteData) -> BoxFuture<'static, DeltaResult> { - let this = self; +impl std::future::IntoFuture for WriteBuilder { + type Output = DeltaResult; + type IntoFuture = BoxFuture<'static, Self::Output>; + + fn into_future(self) -> Self::IntoFuture { + let this = self.config; + let input_data = self.data; Box::pin(async move { if this.mode == SaveMode::Overwrite { @@ -550,12 +566,13 @@ impl WriteBuilder { )); } let input_schema = match input_data { - WriteData::RecordBatches((_, schema)) => schema.clone(), - WriteData::DataFusionPlan(plan) => plan.as_ref().schema().clone(), + Some(WriteData::RecordBatches((_, ref schema))) => schema.clone(), + Some(WriteData::DataFusionPlan(ref plan)) => plan.as_ref().schema().clone(), + None => return Err(WriteError::MissingData.into()), }; // Create table actions to initialize table in case it does not yet exist and should be created - let mut actions = check_preconditions(&this, input_schema).await?; + let mut actions = check_preconditions(&this, input_schema.clone()).await?; let active_partitions = this .snapshot @@ -632,7 +649,8 @@ impl WriteBuilder { }; match input_data { - WriteData::DataFusionPlan(plan) => { + None => return Err(WriteError::MissingData.into()), + Some(WriteData::DataFusionPlan(plan)) => { let add_actions = write_execution_plan_with_predicate( predicate.clone(), this.snapshot.as_ref(), @@ -649,7 +667,7 @@ impl WriteBuilder { .await?; actions.extend(add_actions); } - WriteData::RecordBatches((input_batches, _)) => { + Some(WriteData::RecordBatches((input_batches, _))) => { for batches in ChunksIterator::new(input_batches, 10) { let data = if !partition_columns.is_empty() { // TODO partitioning should probably happen in its own plan ... diff --git a/python/src/lib.rs b/python/src/lib.rs index 2a79f104f7..9d79cfe570 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1413,7 +1413,7 @@ fn write_to_deltalake( .map_err(PythonError::from)?; let mut builder = table - .write() + .write(Box::new(batches), Arc::new(data_schema.0)) .with_save_mode(save_mode) .with_write_batch_size(max_rows_per_group as usize); if let Some(schema_mode) = schema_mode { @@ -1452,7 +1452,7 @@ fn write_to_deltalake( }; rt()? - .block_on(builder.execte(batches, Arc::new(data_schema.0))) + .block_on(builder.into_future()) .map_err(PythonError::from)?; Ok(()) From cde4207ae8e2e7ea0cedd00c8d7a556bbb6be184 Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Fri, 8 Mar 2024 21:15:19 +0100 Subject: [PATCH 04/18] another bug fix --- crates/benchmarks/src/bin/merge.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/benchmarks/src/bin/merge.rs b/crates/benchmarks/src/bin/merge.rs index e1010e7479..b530c8f01b 100644 --- a/crates/benchmarks/src/bin/merge.rs +++ b/crates/benchmarks/src/bin/merge.rs @@ -554,7 +554,7 @@ async fn main() { ])); let batch = RecordBatch::try_new( - schema, + schema.clone(), vec![ Arc::new(StringArray::from(group_ids)), Arc::new(StringArray::from(name)), From 67433737338788241f11326461afee5765e57957 Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Fri, 8 Mar 2024 21:19:02 +0100 Subject: [PATCH 05/18] clippy feedback --- crates/core/src/operations/write.rs | 8 +++----- python/src/lib.rs | 2 +- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/crates/core/src/operations/write.rs b/crates/core/src/operations/write.rs index c40c6a51fa..c5e725675d 100644 --- a/crates/core/src/operations/write.rs +++ b/crates/core/src/operations/write.rs @@ -850,12 +850,10 @@ impl Iterator for ChunksIterator { for _ in 0..self.chunk_size { if let Some(batch) = self.iter.next() { chunk.push(batch); + } else if chunk.is_empty() { + return None; } else { - if chunk.is_empty() { - return None; - } else { - return Some(chunk); - } + return Some(chunk); } } Some(chunk) diff --git a/python/src/lib.rs b/python/src/lib.rs index 9d79cfe570..8fcb5d78c5 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1402,7 +1402,7 @@ fn write_to_deltalake( writer_properties: Option>>, custom_metadata: Option>, ) -> PyResult<()> { - let batches = data.0.map(|batch| batch.unwrap()).into_iter(); + let batches = data.0.map(|batch| batch.unwrap()); let save_mode = mode.parse().map_err(PythonError::from)?; let options = storage_options.clone().unwrap_or_default(); From 577442b40d6b8e498654c7c7f7403ac634ad782d Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Fri, 8 Mar 2024 21:28:25 +0100 Subject: [PATCH 06/18] test compilation --- crates/core/src/operations/write.rs | 7 +++++++ crates/core/tests/integration_datafusion.rs | 3 ++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/crates/core/src/operations/write.rs b/crates/core/src/operations/write.rs index c5e725675d..375135b12a 100644 --- a/crates/core/src/operations/write.rs +++ b/crates/core/src/operations/write.rs @@ -224,6 +224,13 @@ impl WriteBuilder { self } + + /// Execution plan that produces the data to be written to the delta table + pub fn with_input_execution_plan(mut self, plan: Arc) -> Self { + self.data = Some(WriteData::DataFusionPlan(plan)); + self + } + /// A session state accompanying a given input plan, containing e.g. registered object stores pub fn with_input_session_state(mut self, state: SessionState) -> Self { self.config.state = Some(state); diff --git a/crates/core/tests/integration_datafusion.rs b/crates/core/tests/integration_datafusion.rs index 4be66534fe..84d12fd6d9 100644 --- a/crates/core/tests/integration_datafusion.rs +++ b/crates/core/tests/integration_datafusion.rs @@ -603,8 +603,9 @@ mod local { let state = ctx.state(); async fn append_to_table(table: DeltaTable, batch: RecordBatch) -> DeltaTable { + let schema = batch.schema().clone(); DeltaOps(table) - .write(vec![batch]) + .write(vec![batch].into_iter(), schema) .with_save_mode(SaveMode::Append) .await .unwrap() From 4b276a78fea8d4d9bda0d19160c18a179eecb6c2 Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Fri, 8 Mar 2024 21:29:57 +0100 Subject: [PATCH 07/18] wip on tests --- crates/core/tests/integration_datafusion.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/core/tests/integration_datafusion.rs b/crates/core/tests/integration_datafusion.rs index 84d12fd6d9..8bfb23e6db 100644 --- a/crates/core/tests/integration_datafusion.rs +++ b/crates/core/tests/integration_datafusion.rs @@ -605,7 +605,7 @@ mod local { async fn append_to_table(table: DeltaTable, batch: RecordBatch) -> DeltaTable { let schema = batch.schema().clone(); DeltaOps(table) - .write(vec![batch].into_iter(), schema) + .write(Box::new(vec![batch].into_iter()), schema) .with_save_mode(SaveMode::Append) .await .unwrap() From d4d82ce9f74335d1789099edd197ddedb6182c4d Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Mon, 11 Mar 2024 11:08:03 +0100 Subject: [PATCH 08/18] cleanup --- crates/core/src/operations/write.rs | 171 +++++++++++----------------- 1 file changed, 65 insertions(+), 106 deletions(-) diff --git a/crates/core/src/operations/write.rs b/crates/core/src/operations/write.rs index d3657004f2..1c873bd1eb 100644 --- a/crates/core/src/operations/write.rs +++ b/crates/core/src/operations/write.rs @@ -34,12 +34,13 @@ use arrow_array::RecordBatch; use arrow_cast::can_cast_types; use arrow_schema::{ArrowError, DataType, Fields, SchemaRef as ArrowSchemaRef}; use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; +use datafusion::execution::SendableRecordBatchStream; use datafusion::physical_expr::create_physical_expr; use datafusion::physical_plan::filter::FilterExec; -use datafusion::physical_plan::{memory::MemoryExec, ExecutionPlan}; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; +use datafusion::physical_plan::ExecutionPlan; use datafusion_common::DFSchema; use datafusion_expr::Expr; -use futures::future::BoxFuture; use futures::StreamExt; use parquet::file::properties::WriterProperties; @@ -337,12 +338,24 @@ async fn check_preconditions( } } +fn plan_to_streams( + state: SessionState, + plan: Arc, +) -> Result, DeltaTableError> { + let task_ctx = Arc::new(TaskContext::from(&state)); + let mut result = Vec::new(); + for i in 0..plan.output_partitioning().partition_count() { + let stream = plan.execute(i, task_ctx.clone())?; + result.push(stream); + } + return Ok(result); +} + #[allow(clippy::too_many_arguments)] async fn write_execution_plan_with_predicate( predicate: Option, snapshot: Option<&DeltaTableState>, - state: SessionState, - plan: Arc, + write_data: Vec, partition_columns: Vec, object_store: ObjectStoreRef, target_file_size: Option, @@ -350,15 +363,26 @@ async fn write_execution_plan_with_predicate( writer_properties: Option, safe_cast: bool, schema_mode: Option, + target_schema: Option, ) -> DeltaResult> { - let schema: ArrowSchemaRef = if schema_mode.is_some() { - plan.schema() - } else { - snapshot - .and_then(|s| s.input_schema().ok()) - .unwrap_or(plan.schema()) + let target_schema = match target_schema { + Some(schema) => schema, + None => { + if write_data.is_empty() { + return Err(DeltaTableError::Generic( + "No schema provided and no data to infer schema from".to_string(), + )); + } + let write_schema = write_data[0].schema().clone(); + if schema_mode.is_some() { + write_schema + } else { + snapshot + .and_then(|s| s.input_schema().ok()) + .unwrap_or(write_schema) + } + } }; - let checker = if let Some(snapshot) = snapshot { DeltaDataChecker::new(snapshot) } else { @@ -375,20 +399,18 @@ async fn write_execution_plan_with_predicate( // Write data to disk let mut tasks = vec![]; - for i in 0..plan.output_partitioning().partition_count() { - let inner_plan = plan.clone(); - let inner_schema = schema.clone(); - let task_ctx = Arc::new(TaskContext::from(&state)); + for mut stream in write_data { + let inner_schema = target_schema.clone(); + let config = WriterConfig::new( - inner_schema.clone(), + target_schema.clone(), partition_columns.clone(), writer_properties.clone(), target_file_size, write_batch_size, ); let mut writer = DeltaWriter::new(object_store.clone(), config); - let checker_stream = checker.clone(); - let mut stream = inner_plan.execute(i, task_ctx)?; + let checker_stream: DeltaDataChecker = checker.clone(); let handle: tokio::task::JoinHandle>> = tokio::task::spawn(async move { while let Some(maybe_batch) = stream.next().await { @@ -441,8 +463,7 @@ pub(crate) async fn write_execution_plan( write_execution_plan_with_predicate( None, snapshot, - state, - plan, + plan_to_streams(state, plan)?, partition_columns, object_store, target_file_size, @@ -450,6 +471,7 @@ pub(crate) async fn write_execution_plan( writer_properties, safe_cast, schema_mode, + None, ) .await } @@ -654,95 +676,32 @@ impl std::future::IntoFuture for WriteBuilder { _ => (None, None), }; - match input_data { + let write_data = match input_data { None => return Err(WriteError::MissingData.into()), - Some(WriteData::DataFusionPlan(plan)) => { - let add_actions = write_execution_plan_with_predicate( - predicate.clone(), - this.snapshot.as_ref(), - state.clone(), - plan, - partition_columns.clone(), - this.log_store.object_store().clone(), - this.target_file_size, - this.write_batch_size, - this.writer_properties.clone(), - this.safe_cast, - this.schema_mode, - ) - .await?; - actions.extend(add_actions); - } + Some(WriteData::DataFusionPlan(plan)) => plan_to_streams(state.clone(), plan)?, Some(WriteData::RecordBatches((input_batches, _))) => { - for batches in ChunksIterator::new(input_batches, 10) { - let data = if !partition_columns.is_empty() { - // TODO partitioning should probably happen in its own plan ... - let mut partitions: HashMap> = HashMap::new(); - for batch in batches { - let real_batch = match new_schema.clone() { - Some(new_schema) => { - cast_record_batch(&batch, new_schema, false, true)? - } - None => batch, - }; - - let divided = divide_by_partition_values( - target_schema.clone(), - partition_columns.clone(), - &real_batch, - )?; - for part in divided { - let key = part.partition_values.hive_partition_path(); - match partitions.get_mut(&key) { - Some(part_batches) => { - part_batches.push(part.record_batch); - } - None => { - partitions.insert(key, vec![part.record_batch]); - } - } - } - } - partitions.into_values().collect::>() - } else { - match new_schema { - Some(ref new_schema) => { - let mut new_batches = vec![]; - for batch in batches { - new_batches.push(cast_record_batch( - &batch, - new_schema.clone(), - false, - true, - )?); - } - vec![new_batches] - } - None => vec![batches], - } - }; - - let plan = - Arc::new(MemoryExec::try_new(&data, target_schema.clone(), None)?); - - let add_actions = write_execution_plan_with_predicate( - predicate.clone(), - this.snapshot.as_ref(), - state.clone(), - plan, - partition_columns.clone(), - this.log_store.object_store().clone(), - this.target_file_size, - this.write_batch_size, - this.writer_properties.clone(), - this.safe_cast, - this.schema_mode, - ) - .await?; - actions.extend(add_actions); - } + let stream = futures::stream::iter(input_batches.map(Ok)); + let rec_batch_stream = + Box::pin(RecordBatchStreamAdapter::new(input_schema.clone(), stream)) + as SendableRecordBatchStream; + vec![rec_batch_stream] } - } + }; + let add_actions = write_execution_plan_with_predicate( + predicate.clone(), + this.snapshot.as_ref(), + write_data, + partition_columns.clone(), + this.log_store.object_store().clone(), + this.target_file_size, + this.write_batch_size, + this.writer_properties.clone(), + this.safe_cast, + this.schema_mode, + None, + ) + .await?; + actions.extend(add_actions); // Here we need to validate if the new data conforms to a predicate if one is provided if this.schema_mode == Some(SchemaMode::Merge) && schema_drift { From 385c93525f78c8bfa3a2e2b6da241ef415658278 Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Mon, 11 Mar 2024 13:15:00 +0100 Subject: [PATCH 09/18] wip on fixes --- crates/core/src/delta_datafusion/mod.rs | 4 ++-- crates/core/src/operations/constraints.rs | 12 ++++++------ crates/core/src/operations/delete.rs | 2 +- crates/core/src/operations/drop_constraints.rs | 6 +++--- crates/core/src/operations/load.rs | 4 ++-- crates/core/src/operations/merge/mod.rs | 2 +- crates/core/src/operations/optimize.rs | 2 +- crates/core/src/operations/write.rs | 11 ++++++----- crates/core/src/protocol/checkpoints.rs | 2 +- crates/core/src/writer/test_utils.rs | 2 +- crates/core/tests/command_restore.rs | 6 +++--- crates/core/tests/integration_datafusion.rs | 2 +- crates/deltalake/examples/basic_operations.rs | 4 ++-- 13 files changed, 30 insertions(+), 29 deletions(-) diff --git a/crates/core/src/delta_datafusion/mod.rs b/crates/core/src/delta_datafusion/mod.rs index 6ea60a0bda..6b9f8a98e7 100644 --- a/crates/core/src/delta_datafusion/mod.rs +++ b/crates/core/src/delta_datafusion/mod.rs @@ -1756,7 +1756,7 @@ mod tests { .unwrap(); // write some data let table = crate::DeltaOps(table) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .with_save_mode(crate::protocol::SaveMode::Append) .await .unwrap(); @@ -1820,7 +1820,7 @@ mod tests { .unwrap(); // write some data let table = crate::DeltaOps::new_in_memory() - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .with_save_mode(crate::protocol::SaveMode::Append) .await .unwrap(); diff --git a/crates/core/src/operations/constraints.rs b/crates/core/src/operations/constraints.rs index aaa0d62d3a..850eefa186 100644 --- a/crates/core/src/operations/constraints.rs +++ b/crates/core/src/operations/constraints.rs @@ -271,7 +271,7 @@ mod tests { async fn add_constraint_with_invalid_data() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .await?; let table = DeltaOps(write); @@ -287,7 +287,7 @@ mod tests { async fn add_valid_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .await?; let table = DeltaOps(write); @@ -312,7 +312,7 @@ mod tests { // Add constraint by providing a datafusion expression. let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .await?; let table = DeltaOps(write); @@ -378,7 +378,7 @@ mod tests { async fn add_conflicting_named_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .await?; let table = DeltaOps(write); @@ -400,7 +400,7 @@ mod tests { async fn write_data_that_violates_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .await?; let table = DeltaOps(write) @@ -423,7 +423,7 @@ mod tests { async fn write_data_that_does_not_violate_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .await?; let table = DeltaOps(write); diff --git a/crates/core/src/operations/delete.rs b/crates/core/src/operations/delete.rs index 072f4cd875..49bc3caf7c 100644 --- a/crates/core/src/operations/delete.rs +++ b/crates/core/src/operations/delete.rs @@ -401,7 +401,7 @@ mod tests { .unwrap(); // write some data let table = DeltaOps(table) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .with_save_mode(SaveMode::Append) .await .unwrap(); diff --git a/crates/core/src/operations/drop_constraints.rs b/crates/core/src/operations/drop_constraints.rs index 417f73d8e8..6387df535a 100644 --- a/crates/core/src/operations/drop_constraints.rs +++ b/crates/core/src/operations/drop_constraints.rs @@ -146,7 +146,7 @@ mod tests { async fn drop_valid_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .await?; let table = DeltaOps(write); @@ -170,7 +170,7 @@ mod tests { async fn drop_invalid_constraint_not_existing() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .await?; let table = DeltaOps(write) @@ -186,7 +186,7 @@ mod tests { async fn drop_invalid_constraint_ignore() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .await?; let version = write.version(); diff --git a/crates/core/src/operations/load.rs b/crates/core/src/operations/load.rs index 2eac151052..8cfa23ecdc 100644 --- a/crates/core/src/operations/load.rs +++ b/crates/core/src/operations/load.rs @@ -114,7 +114,7 @@ mod tests { #[tokio::test] async fn test_write_load() -> TestResult { let batch = get_record_batch(None, false); - let table = DeltaOps::new_in_memory().write(vec![batch.clone()]).await?; + let table = DeltaOps::new_in_memory().write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()).await?; let (_table, stream) = DeltaOps(table).load().await?; let data = collect_sendable_stream(stream).await?; @@ -145,7 +145,7 @@ mod tests { #[tokio::test] async fn test_load_with_columns() -> TestResult { let batch = get_record_batch(None, false); - let table = DeltaOps::new_in_memory().write(vec![batch.clone()]).await?; + let table = DeltaOps::new_in_memory().write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()).await?; let (_table, stream) = DeltaOps(table).load().with_columns(["id", "value"]).await?; let data = collect_sendable_stream(stream).await?; diff --git a/crates/core/src/operations/merge/mod.rs b/crates/core/src/operations/merge/mod.rs index 96c06fe548..ae2c889659 100644 --- a/crates/core/src/operations/merge/mod.rs +++ b/crates/core/src/operations/merge/mod.rs @@ -1609,7 +1609,7 @@ mod tests { .unwrap(); // write some data DeltaOps(table) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .with_save_mode(SaveMode::Append) .await .unwrap() diff --git a/crates/core/src/operations/optimize.rs b/crates/core/src/operations/optimize.rs index 90334e6de1..c95c967139 100644 --- a/crates/core/src/operations/optimize.rs +++ b/crates/core/src/operations/optimize.rs @@ -1356,7 +1356,7 @@ pub(super) mod zorder { .unwrap(); // write some data let table = crate::DeltaOps::new_in_memory() - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .with_save_mode(crate::protocol::SaveMode::Append) .await .unwrap(); diff --git a/crates/core/src/operations/write.rs b/crates/core/src/operations/write.rs index 1c873bd1eb..37752387ad 100644 --- a/crates/core/src/operations/write.rs +++ b/crates/core/src/operations/write.rs @@ -41,6 +41,7 @@ use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::ExecutionPlan; use datafusion_common::DFSchema; use datafusion_expr::Expr; +use futures::future::BoxFuture; use futures::StreamExt; use parquet::file::properties::WriterProperties; @@ -53,14 +54,14 @@ use crate::delta_datafusion::expr::parse_predicate_expression; use crate::delta_datafusion::DeltaDataChecker; use crate::delta_datafusion::{find_files, register_store, DeltaScanBuilder}; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::{Action, Add, Metadata, PartitionsExt, Remove, StructType}; +use crate::kernel::{Action, Add, Metadata, Remove, StructType}; use crate::logstore::LogStoreRef; -use crate::operations::cast::{cast_record_batch, merge_schema}; +use crate::operations::cast::{merge_schema}; use crate::protocol::{DeltaOperation, SaveMode}; use crate::storage::ObjectStoreRef; use crate::table::state::DeltaTableState; use crate::table::Constraint as DeltaConstraint; -use crate::writer::record_batch::divide_by_partition_values; + use crate::DeltaTable; #[derive(thiserror::Error, Debug)] @@ -348,7 +349,7 @@ fn plan_to_streams( let stream = plan.execute(i, task_ctx.clone())?; result.push(stream); } - return Ok(result); + Ok(result) } #[allow(clippy::too_many_arguments)] @@ -1603,7 +1604,7 @@ mod tests { .unwrap(); let table = DeltaOps(table) - .write(vec![batch_add], batch_add.schema()) + .write(Box::new(iter::once(batch_add)), batch_add.schema()) .with_save_mode(SaveMode::Overwrite) .with_replace_where(col("id").eq(lit("A"))) .await diff --git a/crates/core/src/protocol/checkpoints.rs b/crates/core/src/protocol/checkpoints.rs index 4b6f21e002..b4cadbab5b 100644 --- a/crates/core/src/protocol/checkpoints.rs +++ b/crates/core/src/protocol/checkpoints.rs @@ -1013,7 +1013,7 @@ mod tests { ("struct_with_list", struct_with_list_array), ]) .unwrap(); - let table = DeltaOps::new_in_memory().write(vec![batch]).await.unwrap(); + let table = DeltaOps::new_in_memory().write(Box::new(iter::once(batch)), batch.schema()).await.unwrap(); create_checkpoint(&table).await.unwrap(); } diff --git a/crates/core/src/writer/test_utils.rs b/crates/core/src/writer/test_utils.rs index 093ad7cbd0..aeb664a5f8 100644 --- a/crates/core/src/writer/test_utils.rs +++ b/crates/core/src/writer/test_utils.rs @@ -347,7 +347,7 @@ pub mod datafusion { pub async fn write_batch(table: DeltaTable, batch: RecordBatch) -> DeltaTable { DeltaOps(table) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .with_save_mode(SaveMode::Append) .await .expect("Failed to append") diff --git a/crates/core/tests/command_restore.rs b/crates/core/tests/command_restore.rs index 1e49132d23..2767267213 100644 --- a/crates/core/tests/command_restore.rs +++ b/crates/core/tests/command_restore.rs @@ -45,21 +45,21 @@ async fn setup_test() -> Result> { let batch = get_record_batch(); thread::sleep(Duration::from_secs(1)); let table = DeltaOps(table) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .with_save_mode(SaveMode::Append) .await .unwrap(); thread::sleep(Duration::from_secs(1)); let table = DeltaOps(table) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .with_save_mode(SaveMode::Overwrite) .await .unwrap(); thread::sleep(Duration::from_secs(1)); let table = DeltaOps(table) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .with_save_mode(SaveMode::Append) .await .unwrap(); diff --git a/crates/core/tests/integration_datafusion.rs b/crates/core/tests/integration_datafusion.rs index 8bfb23e6db..b2dbac7948 100644 --- a/crates/core/tests/integration_datafusion.rs +++ b/crates/core/tests/integration_datafusion.rs @@ -113,7 +113,7 @@ mod local { for batch in batches { table = DeltaOps(table) - .write(vec![batch]) + .write(std::iter::once(batch), batch.schema().clone()) .with_save_mode(save_mode.clone()) .await .unwrap(); diff --git a/crates/deltalake/examples/basic_operations.rs b/crates/deltalake/examples/basic_operations.rs index e697e4cf53..5369d91744 100644 --- a/crates/deltalake/examples/basic_operations.rs +++ b/crates/deltalake/examples/basic_operations.rs @@ -89,7 +89,7 @@ async fn main() -> Result<(), deltalake::errors::DeltaTableError> { let batch = get_table_batches(); let table = DeltaOps(table) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .with_writer_properties(writer_properties) .await?; @@ -101,7 +101,7 @@ async fn main() -> Result<(), deltalake::errors::DeltaTableError> { // To overwrite instead of append (which is the default), use `.with_save_mode`: let table = DeltaOps(table) - .write(vec![batch.clone()]) + .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) .with_save_mode(SaveMode::Overwrite) .with_writer_properties(writer_properties) .await?; From 023df0925e671329b71dc751c8b3e4f6479800a7 Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Mon, 11 Mar 2024 14:21:08 +0100 Subject: [PATCH 10/18] more fixes --- crates/benchmarks/src/bin/merge.rs | 4 +- crates/core/src/operations/constraints.rs | 6 +- crates/core/src/operations/delete.rs | 12 +- crates/core/src/operations/mod.rs | 8 +- crates/core/src/operations/update.rs | 2 +- crates/core/src/operations/write.rs | 103 ++++++++---------- crates/core/src/protocol/checkpoints.rs | 2 +- crates/core/src/writer/json.rs | 12 +- crates/core/tests/command_restore.rs | 7 +- crates/core/tests/integration_datafusion.rs | 3 +- crates/deltalake/examples/basic_operations.rs | 4 +- docs/src/rust/check_constraints.rs | 2 +- docs/src/rust/operations.rs | 2 +- python/src/lib.rs | 3 +- 14 files changed, 79 insertions(+), 91 deletions(-) diff --git a/crates/benchmarks/src/bin/merge.rs b/crates/benchmarks/src/bin/merge.rs index b530c8f01b..26a09c26f4 100644 --- a/crates/benchmarks/src/bin/merge.rs +++ b/crates/benchmarks/src/bin/merge.rs @@ -77,7 +77,7 @@ pub async fn convert_tpcds_web_returns(input_path: String, table_path: String) - DeltaOps::try_from_uri(table_path) .await .unwrap() - .write(Box::new(tbl.into_iter()), schema) + .write(deltalake_core::operations::write::WriteData::Vecs(tbl)) .with_partition_columns(vec!["wr_returned_date_sk"]) .await .unwrap(); @@ -568,7 +568,7 @@ async fn main() { DeltaOps::try_from_uri(output) .await .unwrap() - .write(Box::new(vec![batch].into_iter()), schema.clone()) + .write(deltalake_core::operations::write::WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Append) .await .unwrap(); diff --git a/crates/core/src/operations/constraints.rs b/crates/core/src/operations/constraints.rs index 850eefa186..bfd215d910 100644 --- a/crates/core/src/operations/constraints.rs +++ b/crates/core/src/operations/constraints.rs @@ -355,7 +355,7 @@ mod tests { ) .unwrap(); - let table = DeltaOps::new_in_memory().write(vec![batch]).await.unwrap(); + let table = DeltaOps::new_in_memory().write(WriteData::Vecs(vec![batch])).await.unwrap(); let mut table = DeltaOps(table) .add_constraint() @@ -414,7 +414,7 @@ mod tests { Arc::new(StringArray::from(vec!["2021-02-02"])), ]; let batch = RecordBatch::try_new(get_arrow_schema(&None), invalid_values)?; - let err = table.write(vec![batch]).await; + let err = table.write(WriteData::Vecs(vec![batch])).await; assert!(err.is_err()); Ok(()) } @@ -427,7 +427,7 @@ mod tests { .await?; let table = DeltaOps(write); - let err = table.write(vec![batch]).await; + let err = table.write(WriteData::Vecs(vec![batch])).await; assert!(err.is_ok()); Ok(()) diff --git a/crates/core/src/operations/delete.rs b/crates/core/src/operations/delete.rs index 49bc3caf7c..def5bb8661 100644 --- a/crates/core/src/operations/delete.rs +++ b/crates/core/src/operations/delete.rs @@ -401,7 +401,7 @@ mod tests { .unwrap(); // write some data let table = DeltaOps(table) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -463,7 +463,7 @@ mod tests { // write some data let table = DeltaOps(table) - .write(vec![batch]) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -487,7 +487,7 @@ mod tests { // write some data let table = DeltaOps(table) - .write(vec![batch]) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -554,7 +554,7 @@ mod tests { ) .unwrap(); - DeltaOps::new_in_memory().write(vec![batch]).await.unwrap() + DeltaOps::new_in_memory().write(WriteData::Vecs(vec![batch])).await.unwrap() } // Validate behaviour of greater than @@ -643,7 +643,7 @@ mod tests { // write some data let table = DeltaOps(table) - .write(vec![batch]) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -701,7 +701,7 @@ mod tests { // write some data let table = DeltaOps(table) - .write(vec![batch]) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Append) .await .unwrap(); diff --git a/crates/core/src/operations/mod.rs b/crates/core/src/operations/mod.rs index b8aedb7277..7714414c47 100644 --- a/crates/core/src/operations/mod.rs +++ b/crates/core/src/operations/mod.rs @@ -33,8 +33,7 @@ use self::{ }; #[cfg(feature = "datafusion")] pub use ::datafusion::physical_plan::common::collect as collect_sendable_stream; -#[cfg(feature = "datafusion")] -use arrow::record_batch::RecordBatch; + use optimize::OptimizeBuilder; use restore::RestoreBuilder; @@ -139,11 +138,10 @@ impl DeltaOps { #[must_use] pub fn write( self, - batches: Box + Send>, - schema: arrow_schema::SchemaRef, + data: write::WriteData, ) -> WriteBuilder { WriteBuilder::new(self.0.log_store, self.0.state) - .with_data(write::WriteData::RecordBatches((batches, schema))) + .with_data(data) } /// Vacuum stale files from delta table diff --git a/crates/core/src/operations/update.rs b/crates/core/src/operations/update.rs index 803b1d0312..c29addc56a 100644 --- a/crates/core/src/operations/update.rs +++ b/crates/core/src/operations/update.rs @@ -525,7 +525,7 @@ mod tests { ) .unwrap(); - DeltaOps::new_in_memory().write(vec![batch]).await.unwrap() + DeltaOps::new_in_memory().write(WriteData::Vecs(vec![batch])).await.unwrap() } #[tokio::test] diff --git a/crates/core/src/operations/write.rs b/crates/core/src/operations/write.rs index 37752387ad..397edab24e 100644 --- a/crates/core/src/operations/write.rs +++ b/crates/core/src/operations/write.rs @@ -21,7 +21,7 @@ //! let ids = arrow::array::Int32Array::from(vec![1, 2, 3, 4, 5]); //! let batch = RecordBatch::try_new(schema, vec![Arc::new(ids)])?; //! let ops = DeltaOps::try_from_uri("../path/to/empty/dir").await?; -//! let table = ops.write(vec![batch], batch.schema()).await?; +//! let table = ops.write(WriteData::Vecs(vec![batch])).await?; //! ```` use std::collections::HashMap; @@ -122,6 +122,8 @@ pub enum WriteData { RecordBatches((Box + Send>, ArrowSchemaRef)), /// A Datafusion Execution plan DataFusionPlan(Arc), + /// For convenience, a vector of record batches + Vecs(Vec), } /// Configuration for writing data to a DeltaTable @@ -597,6 +599,12 @@ impl std::future::IntoFuture for WriteBuilder { let input_schema = match input_data { Some(WriteData::RecordBatches((_, ref schema))) => schema.clone(), Some(WriteData::DataFusionPlan(ref plan)) => plan.as_ref().schema().clone(), + Some(WriteData::Vecs(ref batches)) => { + if batches.is_empty() { + return Err(WriteError::MissingData.into()); + } + batches[0].schema() + } None => return Err(WriteError::MissingData.into()), }; @@ -680,6 +688,13 @@ impl std::future::IntoFuture for WriteBuilder { let write_data = match input_data { None => return Err(WriteError::MissingData.into()), Some(WriteData::DataFusionPlan(plan)) => plan_to_streams(state.clone(), plan)?, + Some(WriteData::Vecs(input_batches)) => { + let stream = futures::stream::iter(input_batches.into_iter().map(Ok)); + let rec_batch_stream = + Box::pin(RecordBatchStreamAdapter::new(input_schema.clone(), stream)) + as SendableRecordBatchStream; + vec![rec_batch_stream] + } Some(WriteData::RecordBatches((input_batches, _))) => { let stream = futures::stream::iter(input_batches.map(Ok)); let rec_batch_stream = @@ -799,32 +814,6 @@ impl std::future::IntoFuture for WriteBuilder { } } -struct ChunksIterator { - iter: Box + Send>, - chunk_size: usize, -} -impl ChunksIterator { - fn new(iter: Box + Send>, chunk_size: usize) -> Self { - Self { iter, chunk_size } - } -} -impl Iterator for ChunksIterator { - type Item = Vec; - - fn next(&mut self) -> Option { - let mut chunk = Vec::with_capacity(self.chunk_size); - for _ in 0..self.chunk_size { - if let Some(batch) = self.iter.next() { - chunk.push(batch); - } else if chunk.is_empty() { - return None; - } else { - return Some(chunk); - } - } - Some(chunk) - } -} fn try_cast_batch(from_fields: &Fields, to_fields: &Fields) -> Result<(), ArrowError> { if from_fields.len() != to_fields.len() { @@ -892,7 +881,7 @@ mod tests { let table = write_batch(table, batch.clone()).await; // Overwrite let _err = DeltaOps(table) - .write(vec![batch].into_iter(), batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Overwrite) .await .expect_err("Remove action is included when Delta table is append-only. Should error"); @@ -914,7 +903,7 @@ mod tests { // write some data let metadata = HashMap::from_iter(vec![("k1".to_string(), json!("v1.1"))]); let mut table = DeltaOps(table) - .write(vec![batch].clone(), batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Append) .with_metadata(metadata.clone()) .await @@ -937,7 +926,7 @@ mod tests { let metadata: HashMap = HashMap::from_iter(vec![("k1".to_string(), json!("v1.2"))]); let mut table = DeltaOps(table) - .write(vec![batch].clone(), batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Append) .with_metadata(metadata.clone()) .await @@ -960,7 +949,7 @@ mod tests { let metadata: HashMap = HashMap::from_iter(vec![("k2".to_string(), json!("v2.1"))]); let mut table = DeltaOps(table) - .write(vec![batch], batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Overwrite) .with_metadata(metadata.clone()) .await @@ -997,7 +986,7 @@ mod tests { ) .unwrap(); let table = DeltaOps::new_in_memory() - .write(vec![batch], batch.schema()) + .write(WriteData::Vecs(vec![batch])) .await .unwrap(); @@ -1019,7 +1008,7 @@ mod tests { // Test cast options let table = DeltaOps::from(table) - .write(vec![batch].clone(), batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_cast_safety(true) .await .unwrap(); @@ -1039,7 +1028,7 @@ mod tests { assert_batches_sorted_eq!(&expected, &actual); let res = DeltaOps::from(table) - .write(vec![batch], batch.schema()) + .write(WriteData::Vecs(vec![batch])) .await; assert!(res.is_err()); @@ -1058,7 +1047,7 @@ mod tests { ) .unwrap(); let table = DeltaOps::new_in_memory() - .write(vec![batch], batch.schema()) + .write(WriteData::Vecs(vec![batch])) .await .unwrap(); @@ -1076,7 +1065,7 @@ mod tests { .unwrap(); let _res = DeltaOps::from(table) - .write(vec![batch], batch.schema()) + .write(WriteData::Vecs(vec![batch])) .await .unwrap(); let expected = [ @@ -1095,7 +1084,7 @@ mod tests { async fn test_write_nonexistent() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch], batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1107,7 +1096,7 @@ mod tests { async fn test_write_partitioned() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch].clone(), batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::ErrorIfExists) .with_partition_columns(["modified"]) .await @@ -1116,7 +1105,7 @@ mod tests { assert_eq!(table.get_files_count(), 2); let table = DeltaOps::new_in_memory() - .write(vec![batch], batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::ErrorIfExists) .with_partition_columns(["modified", "id"]) .await @@ -1129,7 +1118,7 @@ mod tests { async fn test_merge_schema() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch].clone(), batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1170,7 +1159,7 @@ mod tests { .unwrap(); let mut table = DeltaOps(table) - .write(vec![new_batch], new_batch.schema()) + .write(WriteData::Vecs(vec![new_batch])) .with_save_mode(SaveMode::Append) .with_schema_mode(SchemaMode::Merge) .await @@ -1187,7 +1176,7 @@ mod tests { async fn test_merge_schema_with_partitions() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch].clone(), batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_partition_columns(vec!["id", "value"]) .with_save_mode(SaveMode::ErrorIfExists) .await @@ -1229,7 +1218,7 @@ mod tests { .unwrap(); println!("new_batch: {:?}", new_batch.schema()); let table = DeltaOps(table) - .write(vec![new_batch], new_batch.schema()) + .write(WriteData::Vecs(vec![new_batch])) .with_save_mode(SaveMode::Append) .with_schema_mode(SchemaMode::Merge) .await @@ -1249,7 +1238,7 @@ mod tests { async fn test_overwrite_schema() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch].clone(), batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1290,7 +1279,7 @@ mod tests { .unwrap(); let table = DeltaOps(table) - .write(vec![new_batch], new_batch.schema()) + .write(WriteData::Vecs(vec![new_batch])) .with_save_mode(SaveMode::Append) .with_schema_mode(SchemaMode::Overwrite) .await; @@ -1302,7 +1291,7 @@ mod tests { // If you do not pass a schema mode, we want to check the schema let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch].clone(), batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1332,7 +1321,7 @@ mod tests { RecordBatch::try_new(Arc::new(new_schema), vec![Arc::new(inserted_by)]).unwrap(); let table = DeltaOps(table) - .write(vec![new_batch], new_batch.schema()) + .write(WriteData::Vecs(vec![new_batch])) .with_save_mode(SaveMode::Append) .await; assert!(table.is_err()); @@ -1361,7 +1350,7 @@ mod tests { assert_eq!(table.version(), 0); let table = DeltaOps(table) - .write(vec![batch].clone(), batch.schema()) + .write(WriteData::Vecs(vec![batch])) .await .unwrap(); assert_eq!(table.version(), 1); @@ -1386,7 +1375,7 @@ mod tests { assert_eq!(table.version(), 0); let table = DeltaOps(table) - .write(vec![batch].clone(), batch.schema()) + .write(WriteData::Vecs(vec![batch])) .await; assert!(table.is_err()) } @@ -1404,7 +1393,7 @@ mod tests { assert_eq!(table.version(), 0); let table = DeltaOps(table) - .write(vec![batch].clone(), batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -1443,7 +1432,7 @@ mod tests { .unwrap(); let _table = ops - .write([batch.clone()], batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_partition_columns(["string"]) .await .unwrap(); @@ -1485,7 +1474,7 @@ mod tests { .unwrap(); let table = DeltaOps::new_in_memory() - .write(vec![batch], batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -1502,7 +1491,7 @@ mod tests { .unwrap(); let table = DeltaOps(table) - .write(vec![batch_add], batch_add.schema()) + .write(WriteData::Vecs(vec![batch_add])) .with_save_mode(SaveMode::Overwrite) .with_replace_where(col("id").eq(lit("C"))) .await @@ -1541,7 +1530,7 @@ mod tests { .unwrap(); let table = DeltaOps::new_in_memory() - .write(vec![batch], batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -1564,7 +1553,7 @@ mod tests { .unwrap(); let table = DeltaOps(table) - .write(vec![batch_fail], batch_fail.schema()) + .write(WriteData::Vecs(vec![batch_fail])) .with_save_mode(SaveMode::Overwrite) .with_replace_where(col("id").eq(lit("C"))) .await; @@ -1582,7 +1571,7 @@ mod tests { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(vec![batch], batch.schema()) + .write(WriteData::Vecs(vec![batch])) .with_partition_columns(["id", "value"]) .with_save_mode(SaveMode::Append) .await @@ -1604,7 +1593,7 @@ mod tests { .unwrap(); let table = DeltaOps(table) - .write(Box::new(iter::once(batch_add)), batch_add.schema()) + .write(WriteData::Vecs(vec![batch_add])) .with_save_mode(SaveMode::Overwrite) .with_replace_where(col("id").eq(lit("A"))) .await diff --git a/crates/core/src/protocol/checkpoints.rs b/crates/core/src/protocol/checkpoints.rs index b4cadbab5b..e47c1252d9 100644 --- a/crates/core/src/protocol/checkpoints.rs +++ b/crates/core/src/protocol/checkpoints.rs @@ -1013,7 +1013,7 @@ mod tests { ("struct_with_list", struct_with_list_array), ]) .unwrap(); - let table = DeltaOps::new_in_memory().write(Box::new(iter::once(batch)), batch.schema()).await.unwrap(); + let table = DeltaOps::new_in_memory().write(crate::operations::write::WriteData::Vecs(vec![batch])).await.unwrap(); create_checkpoint(&table).await.unwrap(); } diff --git a/crates/core/src/writer/json.rs b/crates/core/src/writer/json.rs index 8cc908320e..f1266ccd53 100644 --- a/crates/core/src/writer/json.rs +++ b/crates/core/src/writer/json.rs @@ -468,7 +468,7 @@ mod tests { } ); - writer.write(vec![data]).await.unwrap(); + writer.write(WriteData::Vecs(vec![data])).await.unwrap(); let add_actions = writer.flush().await.unwrap(); let add = &add_actions[0]; let path = table_dir.path().join(&add.path); @@ -549,7 +549,7 @@ mod tests { } ); - let res = writer.write(vec![data]).await; + let res = writer.write(WriteData::Vecs(vec![data])).await; assert!(matches!( res, Err(DeltaTableError::Arrow { @@ -586,7 +586,7 @@ mod tests { } ); - writer.write(vec![data]).await.unwrap(); + writer.write(WriteData::Vecs(vec![data])).await.unwrap(); let add_actions = writer.flush().await.unwrap(); assert_eq!(add_actions.len(), 1); @@ -597,7 +597,7 @@ mod tests { } ); - match writer.write(vec![second_data]).await { + match writer.write(WriteData::Vecs(vec![second_data])).await { Ok(_) => { assert!(false, "Should not have successfully written"); } @@ -639,7 +639,7 @@ mod tests { } ); - writer.write(vec![data]).await.unwrap(); + writer.write(WriteData::Vecs(vec![data])).await.unwrap(); let add_actions = writer.flush().await.unwrap(); assert_eq!(add_actions.len(), 1); @@ -651,7 +651,7 @@ mod tests { ); // TODO This should fail because we haven't asked to evolve the schema - writer.write(vec![second_data]).await.unwrap(); + writer.write(WriteData::Vecs(vec![second_data])).await.unwrap(); writer.flush_and_commit(&mut table).await.unwrap(); assert_eq!(table.version(), 1); } diff --git a/crates/core/tests/command_restore.rs b/crates/core/tests/command_restore.rs index 2767267213..9b64abd248 100644 --- a/crates/core/tests/command_restore.rs +++ b/crates/core/tests/command_restore.rs @@ -3,6 +3,7 @@ use arrow_array::{Int32Array, RecordBatch}; use arrow_schema::{DataType as ArrowDataType, Field}; use chrono::{DateTime, NaiveDateTime, TimeZone, Utc}; use deltalake_core::kernel::{DataType, PrimitiveType, StructField}; +use deltalake_core::operations::write::WriteData; use deltalake_core::protocol::SaveMode; use deltalake_core::storage::commit_uri_from_version; use deltalake_core::{DeltaOps, DeltaTable}; @@ -45,21 +46,21 @@ async fn setup_test() -> Result> { let batch = get_record_batch(); thread::sleep(Duration::from_secs(1)); let table = DeltaOps(table) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(SaveMode::Append) .await .unwrap(); thread::sleep(Duration::from_secs(1)); let table = DeltaOps(table) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(SaveMode::Overwrite) .await .unwrap(); thread::sleep(Duration::from_secs(1)); let table = DeltaOps(table) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(SaveMode::Append) .await .unwrap(); diff --git a/crates/core/tests/integration_datafusion.rs b/crates/core/tests/integration_datafusion.rs index b2dbac7948..5b34cddecf 100644 --- a/crates/core/tests/integration_datafusion.rs +++ b/crates/core/tests/integration_datafusion.rs @@ -603,9 +603,8 @@ mod local { let state = ctx.state(); async fn append_to_table(table: DeltaTable, batch: RecordBatch) -> DeltaTable { - let schema = batch.schema().clone(); DeltaOps(table) - .write(Box::new(vec![batch].into_iter()), schema) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(SaveMode::Append) .await .unwrap() diff --git a/crates/deltalake/examples/basic_operations.rs b/crates/deltalake/examples/basic_operations.rs index 5369d91744..5b01734e41 100644 --- a/crates/deltalake/examples/basic_operations.rs +++ b/crates/deltalake/examples/basic_operations.rs @@ -89,7 +89,7 @@ async fn main() -> Result<(), deltalake::errors::DeltaTableError> { let batch = get_table_batches(); let table = DeltaOps(table) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(deltalake::operations::write::WriteData::Vecs(vec![batch.clone()])) .with_writer_properties(writer_properties) .await?; @@ -101,7 +101,7 @@ async fn main() -> Result<(), deltalake::errors::DeltaTableError> { // To overwrite instead of append (which is the default), use `.with_save_mode`: let table = DeltaOps(table) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(deltalake::operations::write::WriteData::Vecs(vec![batch.clone()])) .with_save_mode(SaveMode::Overwrite) .with_writer_properties(writer_properties) .await?; diff --git a/docs/src/rust/check_constraints.rs b/docs/src/rust/check_constraints.rs index fbc2cf18d6..a0f9742554 100644 --- a/docs/src/rust/check_constraints.rs +++ b/docs/src/rust/check_constraints.rs @@ -16,7 +16,7 @@ async fn main() -> Result<(), Box> { Arc::new(Int32Array::from(vec![-10])) ]; let batch = RecordBatch::try_new(schema, invalid_values)?; - table.write(vec![batch]).await?; + table.write(WriteData::Vecs(vec![batch])).await?; // --8<-- [end:add_data] Ok(()) diff --git a/docs/src/rust/operations.rs b/docs/src/rust/operations.rs index 55ab40604f..2b9f7dee58 100644 --- a/docs/src/rust/operations.rs +++ b/docs/src/rust/operations.rs @@ -22,7 +22,7 @@ async fn main() -> Result<(), Box> { let table = deltalake::open_table("/tmp/my_table").await.unwrap(); let table = DeltaOps(table) - .write(vec![data]) + .write(WriteData::Vecs(vec![data])) .with_save_mode(SaveMode::Overwrite) .with_replace_where(col("id").eq(lit("1"))) .await; diff --git a/python/src/lib.rs b/python/src/lib.rs index b36f844e3a..d3e52f974c 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -37,6 +37,7 @@ use deltalake::operations::restore::RestoreBuilder; use deltalake::operations::transaction::commit; use deltalake::operations::update::UpdateBuilder; use deltalake::operations::vacuum::VacuumBuilder; +use deltalake::operations::write::WriteData; use deltalake::parquet::basic::Compression; use deltalake::parquet::errors::ParquetError; use deltalake::parquet::file::properties::WriterProperties; @@ -1415,7 +1416,7 @@ fn write_to_deltalake( .map_err(PythonError::from)?; let mut builder = table - .write(Box::new(batches), Arc::new(data_schema.0)) + .write(WriteData::RecordBatches((Box::new(batches), Arc::new(data_schema.0)))) .with_save_mode(save_mode) .with_write_batch_size(max_rows_per_group as usize); if let Some(schema_mode) = schema_mode { From 0397a0c7adb6ff11147886d8398171e512caa524 Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Mon, 11 Mar 2024 15:16:36 +0100 Subject: [PATCH 11/18] more fixes --- crates/benchmarks/src/bin/merge.rs | 2 +- crates/core/src/delta_datafusion/mod.rs | 5 +++-- crates/core/src/operations/constraints.rs | 13 ++++++----- crates/core/src/operations/delete.rs | 3 ++- .../core/src/operations/drop_constraints.rs | 7 +++--- crates/core/src/operations/load.rs | 5 +++-- crates/core/src/operations/merge/mod.rs | 3 ++- crates/core/src/operations/optimize.rs | 4 +++- crates/core/src/operations/update.rs | 1 + crates/core/src/operations/write.rs | 22 +++++++++---------- crates/core/src/protocol/checkpoints.rs | 5 +++-- crates/core/src/writer/json.rs | 13 ++++++----- crates/core/src/writer/test_utils.rs | 3 ++- crates/core/tests/integration_datafusion.rs | 4 ++-- 14 files changed, 51 insertions(+), 39 deletions(-) diff --git a/crates/benchmarks/src/bin/merge.rs b/crates/benchmarks/src/bin/merge.rs index 26a09c26f4..8824b4a55a 100644 --- a/crates/benchmarks/src/bin/merge.rs +++ b/crates/benchmarks/src/bin/merge.rs @@ -72,7 +72,7 @@ pub async fn convert_tpcds_web_returns(input_path: String, table_path: String) - .unwrap(); let tbl = table.collect().await.unwrap(); - let schema = tbl[0].schema().clone(); + let _schema = tbl[0].schema().clone(); DeltaOps::try_from_uri(table_path) .await diff --git a/crates/core/src/delta_datafusion/mod.rs b/crates/core/src/delta_datafusion/mod.rs index 6b9f8a98e7..9c31e22390 100644 --- a/crates/core/src/delta_datafusion/mod.rs +++ b/crates/core/src/delta_datafusion/mod.rs @@ -1466,6 +1466,7 @@ impl From for DeltaColumn { #[cfg(test)] mod tests { + use crate::operations::write::WriteData; use crate::writer::test_utils::get_delta_schema; use arrow::array::StructArray; use arrow::datatypes::{DataType, Field, Schema}; @@ -1756,7 +1757,7 @@ mod tests { .unwrap(); // write some data let table = crate::DeltaOps(table) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(crate::protocol::SaveMode::Append) .await .unwrap(); @@ -1820,7 +1821,7 @@ mod tests { .unwrap(); // write some data let table = crate::DeltaOps::new_in_memory() - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(crate::protocol::SaveMode::Append) .await .unwrap(); diff --git a/crates/core/src/operations/constraints.rs b/crates/core/src/operations/constraints.rs index bfd215d910..57c8179a6c 100644 --- a/crates/core/src/operations/constraints.rs +++ b/crates/core/src/operations/constraints.rs @@ -239,6 +239,7 @@ mod tests { use arrow_schema::{DataType as ArrowDataType, Field, Schema as ArrowSchema}; use datafusion_expr::{col, lit}; + use crate::operations::write::WriteData; use crate::writer::test_utils::{create_bare_table, get_arrow_schema, get_record_batch}; use crate::{DeltaOps, DeltaResult, DeltaTable}; @@ -271,7 +272,7 @@ mod tests { async fn add_constraint_with_invalid_data() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .await?; let table = DeltaOps(write); @@ -287,7 +288,7 @@ mod tests { async fn add_valid_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .await?; let table = DeltaOps(write); @@ -312,7 +313,7 @@ mod tests { // Add constraint by providing a datafusion expression. let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .await?; let table = DeltaOps(write); @@ -378,7 +379,7 @@ mod tests { async fn add_conflicting_named_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .await?; let table = DeltaOps(write); @@ -400,7 +401,7 @@ mod tests { async fn write_data_that_violates_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .await?; let table = DeltaOps(write) @@ -423,7 +424,7 @@ mod tests { async fn write_data_that_does_not_violate_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .await?; let table = DeltaOps(write); diff --git a/crates/core/src/operations/delete.rs b/crates/core/src/operations/delete.rs index def5bb8661..7b055ad906 100644 --- a/crates/core/src/operations/delete.rs +++ b/crates/core/src/operations/delete.rs @@ -333,6 +333,7 @@ impl std::future::IntoFuture for DeleteBuilder { #[cfg(test)] mod tests { + use crate::operations::write::WriteData; use crate::operations::DeltaOps; use crate::protocol::*; use crate::writer::test_utils::datafusion::get_data; @@ -770,7 +771,7 @@ mod tests { ]; let batches = vec![RecordBatch::try_new(schema.clone(), data).unwrap()]; - let table = DeltaOps::new_in_memory().write(batches).await.unwrap(); + let table = DeltaOps::new_in_memory().write(WriteData::Vecs(batches)).await.unwrap(); let (table, _metrics) = DeltaOps(table) .delete() diff --git a/crates/core/src/operations/drop_constraints.rs b/crates/core/src/operations/drop_constraints.rs index 6387df535a..91084106ed 100644 --- a/crates/core/src/operations/drop_constraints.rs +++ b/crates/core/src/operations/drop_constraints.rs @@ -124,6 +124,7 @@ impl std::future::IntoFuture for DropConstraintBuilder { #[cfg(feature = "datafusion")] #[cfg(test)] mod tests { + use crate::operations::write::WriteData; use crate::writer::test_utils::{create_bare_table, get_record_batch}; use crate::{DeltaOps, DeltaResult, DeltaTable}; @@ -146,7 +147,7 @@ mod tests { async fn drop_valid_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .await?; let table = DeltaOps(write); @@ -170,7 +171,7 @@ mod tests { async fn drop_invalid_constraint_not_existing() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .await?; let table = DeltaOps(write) @@ -186,7 +187,7 @@ mod tests { async fn drop_invalid_constraint_ignore() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .await?; let version = write.version(); diff --git a/crates/core/src/operations/load.rs b/crates/core/src/operations/load.rs index 8cfa23ecdc..6d1ab60805 100644 --- a/crates/core/src/operations/load.rs +++ b/crates/core/src/operations/load.rs @@ -81,6 +81,7 @@ impl std::future::IntoFuture for LoadBuilder { #[cfg(test)] mod tests { + use crate::operations::write::WriteData; use crate::operations::{collect_sendable_stream, DeltaOps}; use crate::writer::test_utils::{get_record_batch, TestResult}; use crate::DeltaTableBuilder; @@ -114,7 +115,7 @@ mod tests { #[tokio::test] async fn test_write_load() -> TestResult { let batch = get_record_batch(None, false); - let table = DeltaOps::new_in_memory().write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()).await?; + let table = DeltaOps::new_in_memory().write(WriteData::Vecs(vec![batch.clone()])).await?; let (_table, stream) = DeltaOps(table).load().await?; let data = collect_sendable_stream(stream).await?; @@ -145,7 +146,7 @@ mod tests { #[tokio::test] async fn test_load_with_columns() -> TestResult { let batch = get_record_batch(None, false); - let table = DeltaOps::new_in_memory().write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()).await?; + let table = DeltaOps::new_in_memory().write(WriteData::Vecs(vec![batch.clone()])).await?; let (_table, stream) = DeltaOps(table).load().with_columns(["id", "value"]).await?; let data = collect_sendable_stream(stream).await?; diff --git a/crates/core/src/operations/merge/mod.rs b/crates/core/src/operations/merge/mod.rs index ae2c889659..b30f4ab2c7 100644 --- a/crates/core/src/operations/merge/mod.rs +++ b/crates/core/src/operations/merge/mod.rs @@ -1528,6 +1528,7 @@ mod tests { use crate::kernel::StructField; use crate::operations::merge::generalize_filter; use crate::operations::merge::try_construct_early_filter; + use crate::operations::write::WriteData; use crate::operations::DeltaOps; use crate::protocol::*; use crate::writer::test_utils::datafusion::get_data; @@ -1609,7 +1610,7 @@ mod tests { .unwrap(); // write some data DeltaOps(table) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(SaveMode::Append) .await .unwrap() diff --git a/crates/core/src/operations/optimize.rs b/crates/core/src/operations/optimize.rs index c95c967139..22aff1e9ce 100644 --- a/crates/core/src/operations/optimize.rs +++ b/crates/core/src/operations/optimize.rs @@ -1233,6 +1233,8 @@ pub(super) mod zorder { #[cfg(test)] mod tests { + use crate::operations::write::WriteData; + use super::*; use ::datafusion::assert_batches_eq; use arrow_array::{Int32Array, StringArray}; @@ -1356,7 +1358,7 @@ pub(super) mod zorder { .unwrap(); // write some data let table = crate::DeltaOps::new_in_memory() - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(crate::protocol::SaveMode::Append) .await .unwrap(); diff --git a/crates/core/src/operations/update.rs b/crates/core/src/operations/update.rs index c29addc56a..1236a585ab 100644 --- a/crates/core/src/operations/update.rs +++ b/crates/core/src/operations/update.rs @@ -475,6 +475,7 @@ mod tests { use crate::kernel::PrimitiveType; use crate::kernel::StructField; use crate::kernel::StructType; + use crate::operations::write::WriteData; use crate::operations::DeltaOps; use crate::writer::test_utils::datafusion::get_data; use crate::writer::test_utils::datafusion::write_batch; diff --git a/crates/core/src/operations/write.rs b/crates/core/src/operations/write.rs index 397edab24e..59306e073c 100644 --- a/crates/core/src/operations/write.rs +++ b/crates/core/src/operations/write.rs @@ -903,7 +903,7 @@ mod tests { // write some data let metadata = HashMap::from_iter(vec![("k1".to_string(), json!("v1.1"))]); let mut table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(SaveMode::Append) .with_metadata(metadata.clone()) .await @@ -926,7 +926,7 @@ mod tests { let metadata: HashMap = HashMap::from_iter(vec![("k1".to_string(), json!("v1.2"))]); let mut table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(SaveMode::Append) .with_metadata(metadata.clone()) .await @@ -949,7 +949,7 @@ mod tests { let metadata: HashMap = HashMap::from_iter(vec![("k2".to_string(), json!("v2.1"))]); let mut table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(SaveMode::Overwrite) .with_metadata(metadata.clone()) .await @@ -1008,7 +1008,7 @@ mod tests { // Test cast options let table = DeltaOps::from(table) - .write(WriteData::Vecs(vec![batch])) + .write(WriteData::Vecs(vec![batch.clone()])) .with_cast_safety(true) .await .unwrap(); @@ -1028,7 +1028,7 @@ mod tests { assert_batches_sorted_eq!(&expected, &actual); let res = DeltaOps::from(table) - .write(WriteData::Vecs(vec![batch])) + .write(WriteData::Vecs(vec![batch.clone()])) .await; assert!(res.is_err()); @@ -1096,7 +1096,7 @@ mod tests { async fn test_write_partitioned() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(SaveMode::ErrorIfExists) .with_partition_columns(["modified"]) .await @@ -1118,7 +1118,7 @@ mod tests { async fn test_merge_schema() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1176,7 +1176,7 @@ mod tests { async fn test_merge_schema_with_partitions() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) + .write(WriteData::Vecs(vec![batch.clone()])) .with_partition_columns(vec!["id", "value"]) .with_save_mode(SaveMode::ErrorIfExists) .await @@ -1238,7 +1238,7 @@ mod tests { async fn test_overwrite_schema() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1350,7 +1350,7 @@ mod tests { assert_eq!(table.version(), 0); let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(WriteData::Vecs(vec![batch.clone()])) .await .unwrap(); assert_eq!(table.version(), 1); @@ -1375,7 +1375,7 @@ mod tests { assert_eq!(table.version(), 0); let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(WriteData::Vecs(vec![batch.clone()])) .await; assert!(table.is_err()) } diff --git a/crates/core/src/protocol/checkpoints.rs b/crates/core/src/protocol/checkpoints.rs index e47c1252d9..c28b555f36 100644 --- a/crates/core/src/protocol/checkpoints.rs +++ b/crates/core/src/protocol/checkpoints.rs @@ -532,6 +532,7 @@ mod tests { use super::*; use crate::kernel::StructType; + use crate::operations::write::WriteData; use crate::operations::DeltaOps; use crate::protocol::Metadata; use crate::writer::test_utils::get_delta_schema; @@ -814,12 +815,12 @@ mod tests { let batches = vec![RecordBatch::try_new(schema.clone(), data).unwrap()]; let table = DeltaOps::new_in_memory() - .write(batches.clone()) + .write(WriteData::Vecs(batches.clone())) .await .unwrap(); DeltaOps(table) - .write(batches) + .write(WriteData::Vecs(batches)) .with_save_mode(crate::protocol::SaveMode::Overwrite) .await .unwrap() diff --git a/crates/core/src/writer/json.rs b/crates/core/src/writer/json.rs index f1266ccd53..81c014b63f 100644 --- a/crates/core/src/writer/json.rs +++ b/crates/core/src/writer/json.rs @@ -468,7 +468,7 @@ mod tests { } ); - writer.write(WriteData::Vecs(vec![data])).await.unwrap(); + writer.write(vec![data]).await.unwrap(); let add_actions = writer.flush().await.unwrap(); let add = &add_actions[0]; let path = table_dir.path().join(&add.path); @@ -549,7 +549,7 @@ mod tests { } ); - let res = writer.write(WriteData::Vecs(vec![data])).await; + let res = writer.write(vec![data]).await; assert!(matches!( res, Err(DeltaTableError::Arrow { @@ -561,6 +561,7 @@ mod tests { // The following sets of tests are related to #1386 and mergeSchema support // mod schema_evolution { + use super::*; #[tokio::test] @@ -586,7 +587,7 @@ mod tests { } ); - writer.write(WriteData::Vecs(vec![data])).await.unwrap(); + writer.write(vec![data]).await.unwrap(); let add_actions = writer.flush().await.unwrap(); assert_eq!(add_actions.len(), 1); @@ -597,7 +598,7 @@ mod tests { } ); - match writer.write(WriteData::Vecs(vec![second_data])).await { + match writer.write(vec![second_data]).await { Ok(_) => { assert!(false, "Should not have successfully written"); } @@ -639,7 +640,7 @@ mod tests { } ); - writer.write(WriteData::Vecs(vec![data])).await.unwrap(); + writer.write(vec![data]).await.unwrap(); let add_actions = writer.flush().await.unwrap(); assert_eq!(add_actions.len(), 1); @@ -651,7 +652,7 @@ mod tests { ); // TODO This should fail because we haven't asked to evolve the schema - writer.write(WriteData::Vecs(vec![second_data])).await.unwrap(); + writer.write(vec![second_data]).await.unwrap(); writer.flush_and_commit(&mut table).await.unwrap(); assert_eq!(table.version(), 1); } diff --git a/crates/core/src/writer/test_utils.rs b/crates/core/src/writer/test_utils.rs index aeb664a5f8..06b0b14373 100644 --- a/crates/core/src/writer/test_utils.rs +++ b/crates/core/src/writer/test_utils.rs @@ -307,6 +307,7 @@ pub async fn create_initialized_table(partition_cols: &[String]) -> DeltaTable { #[cfg(feature = "datafusion")] pub mod datafusion { + use crate::operations::write::WriteData; use crate::operations::DeltaOps; use crate::writer::SaveMode; use crate::DeltaTable; @@ -347,7 +348,7 @@ pub mod datafusion { pub async fn write_batch(table: DeltaTable, batch: RecordBatch) -> DeltaTable { DeltaOps(table) - .write(Box::new(vec![batch.clone()].into_iter()), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch.clone()])) .with_save_mode(SaveMode::Append) .await .expect("Failed to append") diff --git a/crates/core/tests/integration_datafusion.rs b/crates/core/tests/integration_datafusion.rs index 5b34cddecf..a1e94c1e7c 100644 --- a/crates/core/tests/integration_datafusion.rs +++ b/crates/core/tests/integration_datafusion.rs @@ -45,7 +45,7 @@ use std::error::Error; mod local { use datafusion::common::stats::Precision; - use deltalake_core::{logstore::default_logstore, writer::JsonWriter}; + use deltalake_core::{logstore::default_logstore, operations::write::WriteData, writer::JsonWriter}; use object_store::local::LocalFileSystem; use super::*; @@ -113,7 +113,7 @@ mod local { for batch in batches { table = DeltaOps(table) - .write(std::iter::once(batch), batch.schema().clone()) + .write(WriteData::Vecs(vec![batch])) .with_save_mode(save_mode.clone()) .await .unwrap(); From c83f9474a3664b1787b0c160e2262e53b62207e7 Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Mon, 11 Mar 2024 15:22:57 +0100 Subject: [PATCH 12/18] fmt --- crates/benchmarks/src/bin/merge.rs | 4 +++- crates/core/src/operations/constraints.rs | 5 ++++- crates/core/src/operations/delete.rs | 10 ++++++++-- crates/core/src/operations/load.rs | 8 ++++++-- crates/core/src/operations/mod.rs | 8 ++------ crates/core/src/operations/update.rs | 5 ++++- crates/core/src/operations/write.rs | 5 ++--- crates/core/src/protocol/checkpoints.rs | 5 ++++- crates/core/src/schema/partitions.rs | 2 +- crates/core/tests/integration_datafusion.rs | 4 +++- crates/deltalake/examples/basic_operations.rs | 8 ++++++-- python/src/lib.rs | 5 ++++- 12 files changed, 47 insertions(+), 22 deletions(-) diff --git a/crates/benchmarks/src/bin/merge.rs b/crates/benchmarks/src/bin/merge.rs index 8824b4a55a..9ab8639ab4 100644 --- a/crates/benchmarks/src/bin/merge.rs +++ b/crates/benchmarks/src/bin/merge.rs @@ -568,7 +568,9 @@ async fn main() { DeltaOps::try_from_uri(output) .await .unwrap() - .write(deltalake_core::operations::write::WriteData::Vecs(vec![batch])) + .write(deltalake_core::operations::write::WriteData::Vecs(vec![ + batch, + ])) .with_save_mode(SaveMode::Append) .await .unwrap(); diff --git a/crates/core/src/operations/constraints.rs b/crates/core/src/operations/constraints.rs index 57c8179a6c..fb532ae82a 100644 --- a/crates/core/src/operations/constraints.rs +++ b/crates/core/src/operations/constraints.rs @@ -356,7 +356,10 @@ mod tests { ) .unwrap(); - let table = DeltaOps::new_in_memory().write(WriteData::Vecs(vec![batch])).await.unwrap(); + let table = DeltaOps::new_in_memory() + .write(WriteData::Vecs(vec![batch])) + .await + .unwrap(); let mut table = DeltaOps(table) .add_constraint() diff --git a/crates/core/src/operations/delete.rs b/crates/core/src/operations/delete.rs index 7b055ad906..096bbeef50 100644 --- a/crates/core/src/operations/delete.rs +++ b/crates/core/src/operations/delete.rs @@ -555,7 +555,10 @@ mod tests { ) .unwrap(); - DeltaOps::new_in_memory().write(WriteData::Vecs(vec![batch])).await.unwrap() + DeltaOps::new_in_memory() + .write(WriteData::Vecs(vec![batch])) + .await + .unwrap() } // Validate behaviour of greater than @@ -771,7 +774,10 @@ mod tests { ]; let batches = vec![RecordBatch::try_new(schema.clone(), data).unwrap()]; - let table = DeltaOps::new_in_memory().write(WriteData::Vecs(batches)).await.unwrap(); + let table = DeltaOps::new_in_memory() + .write(WriteData::Vecs(batches)) + .await + .unwrap(); let (table, _metrics) = DeltaOps(table) .delete() diff --git a/crates/core/src/operations/load.rs b/crates/core/src/operations/load.rs index 6d1ab60805..4c803d61a7 100644 --- a/crates/core/src/operations/load.rs +++ b/crates/core/src/operations/load.rs @@ -115,7 +115,9 @@ mod tests { #[tokio::test] async fn test_write_load() -> TestResult { let batch = get_record_batch(None, false); - let table = DeltaOps::new_in_memory().write(WriteData::Vecs(vec![batch.clone()])).await?; + let table = DeltaOps::new_in_memory() + .write(WriteData::Vecs(vec![batch.clone()])) + .await?; let (_table, stream) = DeltaOps(table).load().await?; let data = collect_sendable_stream(stream).await?; @@ -146,7 +148,9 @@ mod tests { #[tokio::test] async fn test_load_with_columns() -> TestResult { let batch = get_record_batch(None, false); - let table = DeltaOps::new_in_memory().write(WriteData::Vecs(vec![batch.clone()])).await?; + let table = DeltaOps::new_in_memory() + .write(WriteData::Vecs(vec![batch.clone()])) + .await?; let (_table, stream) = DeltaOps(table).load().with_columns(["id", "value"]).await?; let data = collect_sendable_stream(stream).await?; diff --git a/crates/core/src/operations/mod.rs b/crates/core/src/operations/mod.rs index 7714414c47..ce61ff4fce 100644 --- a/crates/core/src/operations/mod.rs +++ b/crates/core/src/operations/mod.rs @@ -136,12 +136,8 @@ impl DeltaOps { /// Write data to Delta table #[cfg(feature = "datafusion")] #[must_use] - pub fn write( - self, - data: write::WriteData, - ) -> WriteBuilder { - WriteBuilder::new(self.0.log_store, self.0.state) - .with_data(data) + pub fn write(self, data: write::WriteData) -> WriteBuilder { + WriteBuilder::new(self.0.log_store, self.0.state).with_data(data) } /// Vacuum stale files from delta table diff --git a/crates/core/src/operations/update.rs b/crates/core/src/operations/update.rs index 1236a585ab..f179569425 100644 --- a/crates/core/src/operations/update.rs +++ b/crates/core/src/operations/update.rs @@ -526,7 +526,10 @@ mod tests { ) .unwrap(); - DeltaOps::new_in_memory().write(WriteData::Vecs(vec![batch])).await.unwrap() + DeltaOps::new_in_memory() + .write(WriteData::Vecs(vec![batch])) + .await + .unwrap() } #[tokio::test] diff --git a/crates/core/src/operations/write.rs b/crates/core/src/operations/write.rs index 59306e073c..eef44bd1b2 100644 --- a/crates/core/src/operations/write.rs +++ b/crates/core/src/operations/write.rs @@ -56,7 +56,7 @@ use crate::delta_datafusion::{find_files, register_store, DeltaScanBuilder}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::kernel::{Action, Add, Metadata, Remove, StructType}; use crate::logstore::LogStoreRef; -use crate::operations::cast::{merge_schema}; +use crate::operations::cast::merge_schema; use crate::protocol::{DeltaOperation, SaveMode}; use crate::storage::ObjectStoreRef; use crate::table::state::DeltaTableState; @@ -123,7 +123,7 @@ pub enum WriteData { /// A Datafusion Execution plan DataFusionPlan(Arc), /// For convenience, a vector of record batches - Vecs(Vec), + Vecs(Vec), } /// Configuration for writing data to a DeltaTable @@ -814,7 +814,6 @@ impl std::future::IntoFuture for WriteBuilder { } } - fn try_cast_batch(from_fields: &Fields, to_fields: &Fields) -> Result<(), ArrowError> { if from_fields.len() != to_fields.len() { return Err(ArrowError::SchemaError(format!( diff --git a/crates/core/src/protocol/checkpoints.rs b/crates/core/src/protocol/checkpoints.rs index c28b555f36..de9b48e749 100644 --- a/crates/core/src/protocol/checkpoints.rs +++ b/crates/core/src/protocol/checkpoints.rs @@ -1014,7 +1014,10 @@ mod tests { ("struct_with_list", struct_with_list_array), ]) .unwrap(); - let table = DeltaOps::new_in_memory().write(crate::operations::write::WriteData::Vecs(vec![batch])).await.unwrap(); + let table = DeltaOps::new_in_memory() + .write(crate::operations::write::WriteData::Vecs(vec![batch])) + .await + .unwrap(); create_checkpoint(&table).await.unwrap(); } diff --git a/crates/core/src/schema/partitions.rs b/crates/core/src/schema/partitions.rs index 3ecd5cbdad..c766c1d630 100644 --- a/crates/core/src/schema/partitions.rs +++ b/crates/core/src/schema/partitions.rs @@ -245,7 +245,7 @@ mod tests { use serde_json::json; fn check_json_serialize(filter: PartitionFilter, expected_json: &str) { - assert_eq!(serde_json::to_value(&filter).unwrap(), json!(expected_json)) + assert_eq!(serde_json::to_value(filter).unwrap(), json!(expected_json)) } #[test] diff --git a/crates/core/tests/integration_datafusion.rs b/crates/core/tests/integration_datafusion.rs index a1e94c1e7c..c41add713d 100644 --- a/crates/core/tests/integration_datafusion.rs +++ b/crates/core/tests/integration_datafusion.rs @@ -45,7 +45,9 @@ use std::error::Error; mod local { use datafusion::common::stats::Precision; - use deltalake_core::{logstore::default_logstore, operations::write::WriteData, writer::JsonWriter}; + use deltalake_core::{ + logstore::default_logstore, operations::write::WriteData, writer::JsonWriter, + }; use object_store::local::LocalFileSystem; use super::*; diff --git a/crates/deltalake/examples/basic_operations.rs b/crates/deltalake/examples/basic_operations.rs index 5b01734e41..eb5de28d25 100644 --- a/crates/deltalake/examples/basic_operations.rs +++ b/crates/deltalake/examples/basic_operations.rs @@ -89,7 +89,9 @@ async fn main() -> Result<(), deltalake::errors::DeltaTableError> { let batch = get_table_batches(); let table = DeltaOps(table) - .write(deltalake::operations::write::WriteData::Vecs(vec![batch.clone()])) + .write(deltalake::operations::write::WriteData::Vecs(vec![ + batch.clone() + ])) .with_writer_properties(writer_properties) .await?; @@ -101,7 +103,9 @@ async fn main() -> Result<(), deltalake::errors::DeltaTableError> { // To overwrite instead of append (which is the default), use `.with_save_mode`: let table = DeltaOps(table) - .write(deltalake::operations::write::WriteData::Vecs(vec![batch.clone()])) + .write(deltalake::operations::write::WriteData::Vecs(vec![ + batch.clone() + ])) .with_save_mode(SaveMode::Overwrite) .with_writer_properties(writer_properties) .await?; diff --git a/python/src/lib.rs b/python/src/lib.rs index d3e52f974c..1b81eeab6f 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1416,7 +1416,10 @@ fn write_to_deltalake( .map_err(PythonError::from)?; let mut builder = table - .write(WriteData::RecordBatches((Box::new(batches), Arc::new(data_schema.0)))) + .write(WriteData::RecordBatches(( + Box::new(batches), + Arc::new(data_schema.0), + ))) .with_save_mode(save_mode) .with_write_batch_size(max_rows_per_group as usize); if let Some(schema_mode) = schema_mode { From f131eb182954198056371e032bbf122c51222b60 Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Mon, 11 Mar 2024 15:28:30 +0100 Subject: [PATCH 13/18] adjust test --- python/tests/test_writer.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/tests/test_writer.py b/python/tests/test_writer.py index d41ef93225..5276992749 100644 --- a/python/tests/test_writer.py +++ b/python/tests/test_writer.py @@ -1448,8 +1448,8 @@ def test_schema_cols_diff_order(tmp_path: pathlib.Path, engine): assert dt.to_pyarrow_table(columns=["baz", "bar", "foo"]) == expected -def test_empty(existing_table: DeltaTable): +@pytest.mark.parametrize("engine", ["pyarrow", "rust"]) +def test_empty(existing_table: DeltaTable, engine): schema = existing_table.schema().to_pyarrow() empty_table = pa.Table.from_pylist([], schema=schema) - with pytest.raises(DeltaError, match="No data source supplied to write command"): - write_deltalake(existing_table, empty_table, mode="append", engine="rust") + write_deltalake(existing_table, empty_table, mode="append", engine=engine) From a3d55853655f49effb27fc63d6b3d340dddf0eef Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Tue, 12 Mar 2024 20:35:16 +0100 Subject: [PATCH 14/18] use into() --- crates/benchmarks/src/bin/merge.rs | 6 +- crates/core/src/delta_datafusion/mod.rs | 4 +- crates/core/src/operations/constraints.rs | 21 ++-- crates/core/src/operations/delete.rs | 17 ++- .../core/src/operations/drop_constraints.rs | 6 +- crates/core/src/operations/load.rs | 4 +- crates/core/src/operations/merge/mod.rs | 2 +- crates/core/src/operations/optimize.rs | 2 +- crates/core/src/operations/update.rs | 5 +- crates/core/src/operations/write.rs | 103 ++++++++++-------- crates/core/src/protocol/checkpoints.rs | 9 +- crates/core/src/writer/test_utils.rs | 2 +- crates/core/tests/command_restore.rs | 6 +- crates/core/tests/integration_datafusion.rs | 4 +- crates/deltalake/examples/basic_operations.rs | 8 +- docs/src/rust/check_constraints.rs | 2 +- docs/src/rust/operations.rs | 2 +- 17 files changed, 97 insertions(+), 106 deletions(-) diff --git a/crates/benchmarks/src/bin/merge.rs b/crates/benchmarks/src/bin/merge.rs index 9ab8639ab4..d29ee97ce4 100644 --- a/crates/benchmarks/src/bin/merge.rs +++ b/crates/benchmarks/src/bin/merge.rs @@ -77,7 +77,7 @@ pub async fn convert_tpcds_web_returns(input_path: String, table_path: String) - DeltaOps::try_from_uri(table_path) .await .unwrap() - .write(deltalake_core::operations::write::WriteData::Vecs(tbl)) + .write(tbl.into()) .with_partition_columns(vec!["wr_returned_date_sk"]) .await .unwrap(); @@ -568,9 +568,7 @@ async fn main() { DeltaOps::try_from_uri(output) .await .unwrap() - .write(deltalake_core::operations::write::WriteData::Vecs(vec![ - batch, - ])) + .write(batch.into()) .with_save_mode(SaveMode::Append) .await .unwrap(); diff --git a/crates/core/src/delta_datafusion/mod.rs b/crates/core/src/delta_datafusion/mod.rs index 9c31e22390..d2cbb514d2 100644 --- a/crates/core/src/delta_datafusion/mod.rs +++ b/crates/core/src/delta_datafusion/mod.rs @@ -1757,7 +1757,7 @@ mod tests { .unwrap(); // write some data let table = crate::DeltaOps(table) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(crate::protocol::SaveMode::Append) .await .unwrap(); @@ -1821,7 +1821,7 @@ mod tests { .unwrap(); // write some data let table = crate::DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(crate::protocol::SaveMode::Append) .await .unwrap(); diff --git a/crates/core/src/operations/constraints.rs b/crates/core/src/operations/constraints.rs index fb532ae82a..51d3eae6d0 100644 --- a/crates/core/src/operations/constraints.rs +++ b/crates/core/src/operations/constraints.rs @@ -272,7 +272,7 @@ mod tests { async fn add_constraint_with_invalid_data() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .await?; let table = DeltaOps(write); @@ -288,7 +288,7 @@ mod tests { async fn add_valid_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .await?; let table = DeltaOps(write); @@ -313,7 +313,7 @@ mod tests { // Add constraint by providing a datafusion expression. let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .await?; let table = DeltaOps(write); @@ -356,10 +356,7 @@ mod tests { ) .unwrap(); - let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) - .await - .unwrap(); + let table = DeltaOps::new_in_memory().write(batch.into()).await.unwrap(); let mut table = DeltaOps(table) .add_constraint() @@ -382,7 +379,7 @@ mod tests { async fn add_conflicting_named_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .await?; let table = DeltaOps(write); @@ -404,7 +401,7 @@ mod tests { async fn write_data_that_violates_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .await?; let table = DeltaOps(write) @@ -418,7 +415,7 @@ mod tests { Arc::new(StringArray::from(vec!["2021-02-02"])), ]; let batch = RecordBatch::try_new(get_arrow_schema(&None), invalid_values)?; - let err = table.write(WriteData::Vecs(vec![batch])).await; + let err = table.write(batch.into()).await; assert!(err.is_err()); Ok(()) } @@ -427,11 +424,11 @@ mod tests { async fn write_data_that_does_not_violate_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .await?; let table = DeltaOps(write); - let err = table.write(WriteData::Vecs(vec![batch])).await; + let err = table.write(batch.into()).await; assert!(err.is_ok()); Ok(()) diff --git a/crates/core/src/operations/delete.rs b/crates/core/src/operations/delete.rs index 096bbeef50..45719725ab 100644 --- a/crates/core/src/operations/delete.rs +++ b/crates/core/src/operations/delete.rs @@ -402,7 +402,7 @@ mod tests { .unwrap(); // write some data let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -464,7 +464,7 @@ mod tests { // write some data let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -488,7 +488,7 @@ mod tests { // write some data let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -555,10 +555,7 @@ mod tests { ) .unwrap(); - DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) - .await - .unwrap() + DeltaOps::new_in_memory().write(batch.into()).await.unwrap() } // Validate behaviour of greater than @@ -647,7 +644,7 @@ mod tests { // write some data let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -705,7 +702,7 @@ mod tests { // write some data let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -775,7 +772,7 @@ mod tests { let batches = vec![RecordBatch::try_new(schema.clone(), data).unwrap()]; let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(batches)) + .write(batches.into()) .await .unwrap(); diff --git a/crates/core/src/operations/drop_constraints.rs b/crates/core/src/operations/drop_constraints.rs index 91084106ed..f72433f6c4 100644 --- a/crates/core/src/operations/drop_constraints.rs +++ b/crates/core/src/operations/drop_constraints.rs @@ -147,7 +147,7 @@ mod tests { async fn drop_valid_constraint() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .await?; let table = DeltaOps(write); @@ -171,7 +171,7 @@ mod tests { async fn drop_invalid_constraint_not_existing() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .await?; let table = DeltaOps(write) @@ -187,7 +187,7 @@ mod tests { async fn drop_invalid_constraint_ignore() -> DeltaResult<()> { let batch = get_record_batch(None, false); let write = DeltaOps(create_bare_table()) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .await?; let version = write.version(); diff --git a/crates/core/src/operations/load.rs b/crates/core/src/operations/load.rs index 4c803d61a7..d3e9eacd15 100644 --- a/crates/core/src/operations/load.rs +++ b/crates/core/src/operations/load.rs @@ -116,7 +116,7 @@ mod tests { async fn test_write_load() -> TestResult { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .await?; let (_table, stream) = DeltaOps(table).load().await?; @@ -149,7 +149,7 @@ mod tests { async fn test_load_with_columns() -> TestResult { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .await?; let (_table, stream) = DeltaOps(table).load().with_columns(["id", "value"]).await?; diff --git a/crates/core/src/operations/merge/mod.rs b/crates/core/src/operations/merge/mod.rs index b30f4ab2c7..75d44d87ec 100644 --- a/crates/core/src/operations/merge/mod.rs +++ b/crates/core/src/operations/merge/mod.rs @@ -1610,7 +1610,7 @@ mod tests { .unwrap(); // write some data DeltaOps(table) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(SaveMode::Append) .await .unwrap() diff --git a/crates/core/src/operations/optimize.rs b/crates/core/src/operations/optimize.rs index 22aff1e9ce..0c093124b5 100644 --- a/crates/core/src/operations/optimize.rs +++ b/crates/core/src/operations/optimize.rs @@ -1358,7 +1358,7 @@ pub(super) mod zorder { .unwrap(); // write some data let table = crate::DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(crate::protocol::SaveMode::Append) .await .unwrap(); diff --git a/crates/core/src/operations/update.rs b/crates/core/src/operations/update.rs index f179569425..72c43de763 100644 --- a/crates/core/src/operations/update.rs +++ b/crates/core/src/operations/update.rs @@ -526,10 +526,7 @@ mod tests { ) .unwrap(); - DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) - .await - .unwrap() + DeltaOps::new_in_memory().write(batch.into()).await.unwrap() } #[tokio::test] diff --git a/crates/core/src/operations/write.rs b/crates/core/src/operations/write.rs index eef44bd1b2..49080b9250 100644 --- a/crates/core/src/operations/write.rs +++ b/crates/core/src/operations/write.rs @@ -21,7 +21,7 @@ //! let ids = arrow::array::Int32Array::from(vec![1, 2, 3, 4, 5]); //! let batch = RecordBatch::try_new(schema, vec![Arc::new(ids)])?; //! let ops = DeltaOps::try_from_uri("../path/to/empty/dir").await?; -//! let table = ops.write(WriteData::Vecs(vec![batch])).await?; +//! let table = ops.write(batch.into())).await?; //! ```` use std::collections::HashMap; @@ -126,6 +126,28 @@ pub enum WriteData { Vecs(Vec), } +impl From> for WriteData { + /// Convert an execution plan to write data, will basically just execute the plan + fn from(v: Arc) -> Self { + WriteData::DataFusionPlan(v) + } +} + +impl From> for WriteData { + /// Convert a vector of record batches to write data + fn from(v: Vec) -> Self { + WriteData::Vecs(v) + } +} + +impl From for WriteData { + /// Convert a single record batch to write data + fn from(v: RecordBatch) -> Self { + let schema = v.schema().clone(); + WriteData::RecordBatches((Box::new(std::iter::once(v)), schema)) + } +} + /// Configuration for writing data to a DeltaTable pub struct WriteBuilderConfig { /// A snapshot of the to-be-loaded table's state @@ -402,6 +424,9 @@ async fn write_execution_plan_with_predicate( // Write data to disk let mut tasks = vec![]; + + // write_data is a Vec, because we can have multiple streams if we have multiple partitions in DataFusion + // DataFusion Partitions are a performance-related thing and do not match the Delta Table partitions for mut stream in write_data { let inner_schema = target_schema.clone(); @@ -880,7 +905,7 @@ mod tests { let table = write_batch(table, batch.clone()).await; // Overwrite let _err = DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(SaveMode::Overwrite) .await .expect_err("Remove action is included when Delta table is append-only. Should error"); @@ -902,7 +927,7 @@ mod tests { // write some data let metadata = HashMap::from_iter(vec![("k1".to_string(), json!("v1.1"))]); let mut table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(SaveMode::Append) .with_metadata(metadata.clone()) .await @@ -925,7 +950,7 @@ mod tests { let metadata: HashMap = HashMap::from_iter(vec![("k1".to_string(), json!("v1.2"))]); let mut table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(SaveMode::Append) .with_metadata(metadata.clone()) .await @@ -948,7 +973,7 @@ mod tests { let metadata: HashMap = HashMap::from_iter(vec![("k2".to_string(), json!("v2.1"))]); let mut table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(SaveMode::Overwrite) .with_metadata(metadata.clone()) .await @@ -984,10 +1009,7 @@ mod tests { vec![Arc::new(Int32Array::from(vec![Some(0), None]))], ) .unwrap(); - let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) - .await - .unwrap(); + let table = DeltaOps::new_in_memory().write(batch.into()).await.unwrap(); let schema = Arc::new(ArrowSchema::new(vec![Field::new( "value", @@ -1007,7 +1029,7 @@ mod tests { // Test cast options let table = DeltaOps::from(table) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_cast_safety(true) .await .unwrap(); @@ -1026,9 +1048,7 @@ mod tests { let actual = get_data(&table).await; assert_batches_sorted_eq!(&expected, &actual); - let res = DeltaOps::from(table) - .write(WriteData::Vecs(vec![batch.clone()])) - .await; + let res = DeltaOps::from(table).write(batch.clone().into()).await; assert!(res.is_err()); // Validate the datetime -> string behavior @@ -1045,10 +1065,7 @@ mod tests { )]))], ) .unwrap(); - let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) - .await - .unwrap(); + let table = DeltaOps::new_in_memory().write(batch.into()).await.unwrap(); let schema = Arc::new(ArrowSchema::new(vec![Field::new( "value", @@ -1063,10 +1080,7 @@ mod tests { ) .unwrap(); - let _res = DeltaOps::from(table) - .write(WriteData::Vecs(vec![batch])) - .await - .unwrap(); + let _res = DeltaOps::from(table).write(batch.into()).await.unwrap(); let expected = [ "+--------------------------+", "| value |", @@ -1083,7 +1097,7 @@ mod tests { async fn test_write_nonexistent() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1095,7 +1109,7 @@ mod tests { async fn test_write_partitioned() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(SaveMode::ErrorIfExists) .with_partition_columns(["modified"]) .await @@ -1104,7 +1118,7 @@ mod tests { assert_eq!(table.get_files_count(), 2); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(SaveMode::ErrorIfExists) .with_partition_columns(["modified", "id"]) .await @@ -1117,7 +1131,7 @@ mod tests { async fn test_merge_schema() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1158,7 +1172,7 @@ mod tests { .unwrap(); let mut table = DeltaOps(table) - .write(WriteData::Vecs(vec![new_batch])) + .write(new_batch.into()) .with_save_mode(SaveMode::Append) .with_schema_mode(SchemaMode::Merge) .await @@ -1175,7 +1189,7 @@ mod tests { async fn test_merge_schema_with_partitions() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_partition_columns(vec!["id", "value"]) .with_save_mode(SaveMode::ErrorIfExists) .await @@ -1217,7 +1231,7 @@ mod tests { .unwrap(); println!("new_batch: {:?}", new_batch.schema()); let table = DeltaOps(table) - .write(WriteData::Vecs(vec![new_batch])) + .write(new_batch.into()) .with_save_mode(SaveMode::Append) .with_schema_mode(SchemaMode::Merge) .await @@ -1237,7 +1251,7 @@ mod tests { async fn test_overwrite_schema() { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1278,7 +1292,7 @@ mod tests { .unwrap(); let table = DeltaOps(table) - .write(WriteData::Vecs(vec![new_batch])) + .write(new_batch.into()) .with_save_mode(SaveMode::Append) .with_schema_mode(SchemaMode::Overwrite) .await; @@ -1290,7 +1304,7 @@ mod tests { // If you do not pass a schema mode, we want to check the schema let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); @@ -1320,7 +1334,7 @@ mod tests { RecordBatch::try_new(Arc::new(new_schema), vec![Arc::new(inserted_by)]).unwrap(); let table = DeltaOps(table) - .write(WriteData::Vecs(vec![new_batch])) + .write(new_batch.into()) .with_save_mode(SaveMode::Append) .await; assert!(table.is_err()); @@ -1348,10 +1362,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 0); - let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch.clone()])) - .await - .unwrap(); + let table = DeltaOps(table).write(batch.clone().into()).await.unwrap(); assert_eq!(table.version(), 1); let schema: StructType = serde_json::from_value(json!({ @@ -1373,9 +1384,7 @@ mod tests { .unwrap(); assert_eq!(table.version(), 0); - let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch.clone()])) - .await; + let table = DeltaOps(table).write(batch.clone().into()).await; assert!(table.is_err()) } @@ -1392,7 +1401,7 @@ mod tests { assert_eq!(table.version(), 0); let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -1431,7 +1440,7 @@ mod tests { .unwrap(); let _table = ops - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_partition_columns(["string"]) .await .unwrap(); @@ -1473,7 +1482,7 @@ mod tests { .unwrap(); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -1490,7 +1499,7 @@ mod tests { .unwrap(); let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch_add])) + .write(batch_add.into()) .with_save_mode(SaveMode::Overwrite) .with_replace_where(col("id").eq(lit("C"))) .await @@ -1529,7 +1538,7 @@ mod tests { .unwrap(); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(SaveMode::Append) .await .unwrap(); @@ -1552,7 +1561,7 @@ mod tests { .unwrap(); let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch_fail])) + .write(batch_fail.into()) .with_save_mode(SaveMode::Overwrite) .with_replace_where(col("id").eq(lit("C"))) .await; @@ -1570,7 +1579,7 @@ mod tests { let batch = get_record_batch(None, false); let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_partition_columns(["id", "value"]) .with_save_mode(SaveMode::Append) .await @@ -1592,7 +1601,7 @@ mod tests { .unwrap(); let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch_add])) + .write(batch_add.into()) .with_save_mode(SaveMode::Overwrite) .with_replace_where(col("id").eq(lit("A"))) .await diff --git a/crates/core/src/protocol/checkpoints.rs b/crates/core/src/protocol/checkpoints.rs index de9b48e749..4013f50949 100644 --- a/crates/core/src/protocol/checkpoints.rs +++ b/crates/core/src/protocol/checkpoints.rs @@ -815,12 +815,12 @@ mod tests { let batches = vec![RecordBatch::try_new(schema.clone(), data).unwrap()]; let table = DeltaOps::new_in_memory() - .write(WriteData::Vecs(batches.clone())) + .write(batches.clone().into()) .await .unwrap(); DeltaOps(table) - .write(WriteData::Vecs(batches)) + .write(batches.into()) .with_save_mode(crate::protocol::SaveMode::Overwrite) .await .unwrap() @@ -1014,10 +1014,7 @@ mod tests { ("struct_with_list", struct_with_list_array), ]) .unwrap(); - let table = DeltaOps::new_in_memory() - .write(crate::operations::write::WriteData::Vecs(vec![batch])) - .await - .unwrap(); + let table = DeltaOps::new_in_memory().write(batch.into()).await.unwrap(); create_checkpoint(&table).await.unwrap(); } diff --git a/crates/core/src/writer/test_utils.rs b/crates/core/src/writer/test_utils.rs index 06b0b14373..b07a18cef9 100644 --- a/crates/core/src/writer/test_utils.rs +++ b/crates/core/src/writer/test_utils.rs @@ -348,7 +348,7 @@ pub mod datafusion { pub async fn write_batch(table: DeltaTable, batch: RecordBatch) -> DeltaTable { DeltaOps(table) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(SaveMode::Append) .await .expect("Failed to append") diff --git a/crates/core/tests/command_restore.rs b/crates/core/tests/command_restore.rs index 9b64abd248..48be2a7197 100644 --- a/crates/core/tests/command_restore.rs +++ b/crates/core/tests/command_restore.rs @@ -46,21 +46,21 @@ async fn setup_test() -> Result> { let batch = get_record_batch(); thread::sleep(Duration::from_secs(1)); let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(SaveMode::Append) .await .unwrap(); thread::sleep(Duration::from_secs(1)); let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(SaveMode::Overwrite) .await .unwrap(); thread::sleep(Duration::from_secs(1)); let table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch.clone()])) + .write(batch.clone().into()) .with_save_mode(SaveMode::Append) .await .unwrap(); diff --git a/crates/core/tests/integration_datafusion.rs b/crates/core/tests/integration_datafusion.rs index c41add713d..bb744f44c8 100644 --- a/crates/core/tests/integration_datafusion.rs +++ b/crates/core/tests/integration_datafusion.rs @@ -115,7 +115,7 @@ mod local { for batch in batches { table = DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(save_mode.clone()) .await .unwrap(); @@ -606,7 +606,7 @@ mod local { async fn append_to_table(table: DeltaTable, batch: RecordBatch) -> DeltaTable { DeltaOps(table) - .write(WriteData::Vecs(vec![batch])) + .write(batch.into()) .with_save_mode(SaveMode::Append) .await .unwrap() diff --git a/crates/deltalake/examples/basic_operations.rs b/crates/deltalake/examples/basic_operations.rs index eb5de28d25..ede78ffebe 100644 --- a/crates/deltalake/examples/basic_operations.rs +++ b/crates/deltalake/examples/basic_operations.rs @@ -89,9 +89,7 @@ async fn main() -> Result<(), deltalake::errors::DeltaTableError> { let batch = get_table_batches(); let table = DeltaOps(table) - .write(deltalake::operations::write::WriteData::Vecs(vec![ - batch.clone() - ])) + .write(batch.clone().into()) .with_writer_properties(writer_properties) .await?; @@ -103,9 +101,7 @@ async fn main() -> Result<(), deltalake::errors::DeltaTableError> { // To overwrite instead of append (which is the default), use `.with_save_mode`: let table = DeltaOps(table) - .write(deltalake::operations::write::WriteData::Vecs(vec![ - batch.clone() - ])) + .write(batch.clone().into()) .with_save_mode(SaveMode::Overwrite) .with_writer_properties(writer_properties) .await?; diff --git a/docs/src/rust/check_constraints.rs b/docs/src/rust/check_constraints.rs index a0f9742554..c4d0f6d15c 100644 --- a/docs/src/rust/check_constraints.rs +++ b/docs/src/rust/check_constraints.rs @@ -16,7 +16,7 @@ async fn main() -> Result<(), Box> { Arc::new(Int32Array::from(vec![-10])) ]; let batch = RecordBatch::try_new(schema, invalid_values)?; - table.write(WriteData::Vecs(vec![batch])).await?; + table.write(batch.into()).await?; // --8<-- [end:add_data] Ok(()) diff --git a/docs/src/rust/operations.rs b/docs/src/rust/operations.rs index 2b9f7dee58..05263b0fa2 100644 --- a/docs/src/rust/operations.rs +++ b/docs/src/rust/operations.rs @@ -22,7 +22,7 @@ async fn main() -> Result<(), Box> { let table = deltalake::open_table("/tmp/my_table").await.unwrap(); let table = DeltaOps(table) - .write(WriteData::Vecs(vec![data])) + .write(data.into()) .with_save_mode(SaveMode::Overwrite) .with_replace_where(col("id").eq(lit("1"))) .await; From 965968cda0d3d6cace62905b5af5437960ea851a Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Wed, 13 Mar 2024 06:56:58 +0100 Subject: [PATCH 15/18] we need GIL, no? --- python/src/lib.rs | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/python/src/lib.rs b/python/src/lib.rs index 1b81eeab6f..635b377cd2 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1385,6 +1385,20 @@ impl From<&PyAddAction> for Add { } } +struct GilIterator { + reader: ArrowArrayStreamReader, +} + +impl<'a> Iterator for GilIterator { + type Item = RecordBatch; + + fn next(&mut self) -> Option { + let reader = &mut self.reader; + + Python::with_gil(|_| reader.next().map_or(None, |v| Some(v.unwrap()))) + } +} + #[pyfunction] #[allow(clippy::too_many_arguments)] fn write_to_deltalake( @@ -1405,7 +1419,7 @@ fn write_to_deltalake( custom_metadata: Option>, ) -> PyResult<()> { py.allow_threads(|| { - let batches = data.0.map(|batch| batch.unwrap()); + let batches = GilIterator { reader: data.0 }; // The pyarrow reader can be backed by a python object let save_mode = mode.parse().map_err(PythonError::from)?; let options = storage_options.clone().unwrap_or_default(); From 83d398f82087b7c56db9b1fc7d35530617e3cdd7 Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Wed, 13 Mar 2024 08:34:15 +0100 Subject: [PATCH 16/18] clippy, your so right --- python/src/lib.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/src/lib.rs b/python/src/lib.rs index 635b377cd2..3b21eba133 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1389,13 +1389,13 @@ struct GilIterator { reader: ArrowArrayStreamReader, } -impl<'a> Iterator for GilIterator { +impl Iterator for GilIterator { type Item = RecordBatch; fn next(&mut self) -> Option { let reader = &mut self.reader; - Python::with_gil(|_| reader.next().map_or(None, |v| Some(v.unwrap()))) + Python::with_gil(|_| reader.next().map(|v| v.unwrap())) } } From 98bf7ec3ef3c0398385928d4b99295054de69143 Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Wed, 13 Mar 2024 13:57:57 +0100 Subject: [PATCH 17/18] revert 965968cda0d3d6cace62905b5af5437960ea851a and 965968cda0d3d6cace62905b5af5437960ea851a --- python/src/lib.rs | 16 +--------------- 1 file changed, 1 insertion(+), 15 deletions(-) diff --git a/python/src/lib.rs b/python/src/lib.rs index 3b21eba133..1b81eeab6f 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1385,20 +1385,6 @@ impl From<&PyAddAction> for Add { } } -struct GilIterator { - reader: ArrowArrayStreamReader, -} - -impl Iterator for GilIterator { - type Item = RecordBatch; - - fn next(&mut self) -> Option { - let reader = &mut self.reader; - - Python::with_gil(|_| reader.next().map(|v| v.unwrap())) - } -} - #[pyfunction] #[allow(clippy::too_many_arguments)] fn write_to_deltalake( @@ -1419,7 +1405,7 @@ fn write_to_deltalake( custom_metadata: Option>, ) -> PyResult<()> { py.allow_threads(|| { - let batches = GilIterator { reader: data.0 }; // The pyarrow reader can be backed by a python object + let batches = data.0.map(|batch| batch.unwrap()); let save_mode = mode.parse().map_err(PythonError::from)?; let options = storage_options.clone().unwrap_or_default(); From 28eba65d64552d4cbd18675cc8d85061d81b1efc Mon Sep 17 00:00:00 2001 From: Adrian Ehrsam Date: Thu, 14 Mar 2024 10:48:53 +0100 Subject: [PATCH 18/18] fmt --- crates/core/src/protocol/checkpoints.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/core/src/protocol/checkpoints.rs b/crates/core/src/protocol/checkpoints.rs index 71138376bd..9211594349 100644 --- a/crates/core/src/protocol/checkpoints.rs +++ b/crates/core/src/protocol/checkpoints.rs @@ -532,8 +532,8 @@ mod tests { use super::*; use crate::kernel::StructType; - use crate::operations::write::WriteData; use crate::operations::transaction::{CommitBuilder, TableReference}; + use crate::operations::write::WriteData; use crate::operations::DeltaOps; use crate::protocol::Metadata; use crate::writer::test_utils::get_delta_schema;