From 0d627670a0fc509578daf5729bf28c68b670a838 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 27 Sep 2022 22:24:29 +0200 Subject: [PATCH 01/19] feat: rewrite operations --- Cargo.lock | 1 + rust/Cargo.toml | 4 +- rust/src/delta.rs | 30 +- rust/src/lib.rs | 3 +- rust/src/operations/create.rs | 509 +++++++++++------- rust/src/operations/load.rs | 139 +++++ rust/src/operations/mod.rs | 353 ++----------- rust/src/operations/transaction.rs | 349 +++++++------ rust/src/operations/write.rs | 801 ++++++++++++++--------------- rust/src/operations/writer.rs | 323 ++++++++++++ rust/src/table_properties.rs | 69 +++ rust/src/writer/mod.rs | 23 +- rust/src/writer/test_utils.rs | 2 + rust/src/writer/utils.rs | 47 +- rust/tests/datafusion_test.rs | 25 +- 15 files changed, 1528 insertions(+), 1150 deletions(-) create mode 100644 rust/src/operations/load.rs create mode 100644 rust/src/operations/writer.rs create mode 100644 rust/src/table_properties.rs diff --git a/Cargo.lock b/Cargo.lock index 7fc2438ecf..14724031a2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -738,6 +738,7 @@ dependencies = [ "num-traits", "object_store", "once_cell", + "parking_lot", "parquet", "parquet-format", "parquet2", diff --git a/rust/Cargo.toml b/rust/Cargo.toml index d88ec1cf95..d8767b521e 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -1,6 +1,7 @@ [package] name = "deltalake" version = "0.4.1" +rust-version = "1.64" authors = ["Qingping Hou "] homepage = "https://github.com/delta-io/delta.rs" license = "Apache-2.0" @@ -23,6 +24,7 @@ num-bigint = "0.4" num-traits = "0.2.15" object_store = "0.5.0" once_cell = "1.15.0" +parking_lot = "0.12" parquet = { version = "22", features = ["async"], optional = true } parquet2 = { version = "0.16", optional = true } parquet-format = { version = "~4.0.0" } @@ -30,7 +32,7 @@ percent-encoding = "2" serde = { version = "1", features = ["derive"] } serde_json = "1" thiserror = "1" -tokio = { version = "1", features = ["macros", "rt"] } +tokio = { version = "1", features = ["macros", "rt", "parking_lot"] } regex = "1" uuid = { version = "1", features = ["serde", "v4"] } url = "2.3" diff --git a/rust/src/delta.rs b/rust/src/delta.rs index 658d6de8c4..20f9aacfec 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -14,6 +14,7 @@ use super::partitions::{DeltaTablePartition, PartitionFilter}; use super::schema::*; use super::table_state::DeltaTableState; use crate::action::{Add, Stats}; +use crate::builder::{DeltaTableBuilder, DeltaTableConfig}; use crate::delta_config::DeltaConfigError; use crate::storage::ObjectStoreRef; use crate::vacuum::{Vacuum, VacuumError}; @@ -28,8 +29,6 @@ use serde::{Deserialize, Serialize}; use serde_json::{Map, Value}; use uuid::Uuid; -pub use crate::builder::{DeltaTableBuilder, DeltaTableConfig, DeltaVersion}; - /// Metadata for a checkpoint file #[derive(Serialize, Deserialize, Debug, Default, Clone, Copy)] pub struct CheckPoint { @@ -85,22 +84,21 @@ pub enum DeltaTableError { #[from] source: ObjectStoreError, }, - /// Error returned when reading the checkpoint failed. - #[cfg(feature = "parquet")] - #[error("Failed to read checkpoint: {}", .source)] + + /// Error returned when parsing checkpoint parquet. + // #[cfg(feature = "parquet")] + #[error("Failed to parse parquet: {}", .source)] Parquet { /// Parquet error details returned when reading the checkpoint failed. + #[cfg(feature = "parquet")] #[from] source: parquet::errors::ParquetError, - }, - /// Error returned when parsing checkpoint parquet using parquet2 crate. - #[cfg(feature = "parquet2")] - #[error("Failed to parse parquet: {}", .source)] - Parquet { - /// Parquet error details returned when parsing the checkpoint parquet + /// Parquet error details returned when reading the checkpoint failed. + #[cfg(feature = "parquet2")] #[from] source: parquet2::error::Error, }, + /// Error returned when converting the schema in Arrow format failed. #[cfg(feature = "arrow")] #[error("Failed to convert into Arrow schema: {}", .source)] @@ -109,6 +107,7 @@ pub enum DeltaTableError { #[from] source: arrow::error::ArrowError, }, + /// Error returned when the log record has an invalid JSON. #[error("Invalid JSON in log record: {}", .source)] InvalidJson { @@ -191,6 +190,12 @@ pub enum DeltaTableError { /// Generic Delta Table error #[error("Generic DeltaTable error: {0}")] Generic(String), + /// Generic Delta Table error + #[error("Generic error: {source}")] + GenericError { + /// Source error + source: Box, + }, } /// Delta table metadata @@ -410,8 +415,9 @@ pub struct DeltaTable { /// the load options used during load pub config: DeltaTableConfig, pub(crate) storage: ObjectStoreRef, - + /// file metadata for latest checkpoint last_check_point: Option, + /// table versions associated with timestamps version_timestamp: HashMap, } diff --git a/rust/src/lib.rs b/rust/src/lib.rs index 0e31b6b61d..ed65b0e25b 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -89,6 +89,7 @@ pub mod delta_config; pub mod partitions; pub mod schema; pub mod storage; +pub mod table_properties; pub mod table_state; pub mod time_utils; pub mod vacuum; @@ -99,7 +100,7 @@ pub mod checkpoints; pub mod delta_arrow; #[cfg(feature = "datafusion-ext")] pub mod delta_datafusion; -#[cfg(feature = "datafusion-ext")] +#[cfg(all(feature = "arrow", feature = "parquet"))] pub mod operations; #[cfg(feature = "parquet")] pub mod optimize; diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 793ca54fc2..2a9d1580fd 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -1,251 +1,366 @@ //! Command for creating a new delta table // https://github.com/delta-io/delta/blob/master/core/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala +use std::collections::HashMap; use std::sync::Arc; -use super::{ - to_datafusion_err, - transaction::{serialize_actions, OPERATION_SCHEMA}, - DeltaCommandError, *, -}; -use crate::{ - action::{Action, DeltaOperation, MetaData, Protocol, SaveMode}, - DeltaTableBuilder, DeltaTableMetaData, -}; - -use arrow::datatypes::SchemaRef; -use async_trait::async_trait; -use core::any::Any; -use datafusion::{ - execution::context::TaskContext, - physical_plan::{ - common::{compute_record_batch_statistics, SizedRecordBatchStream}, - expressions::PhysicalSortExpr, - metrics::{ExecutionPlanMetricsSet, MemTrackingMetrics}, - stream::RecordBatchStreamAdapter, - Distribution, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, - }, -}; -use datafusion_common::{DataFusionError, Result as DataFusionResult}; -use futures::{TryFutureExt, TryStreamExt}; - -/// Command for creating new delta table -pub struct CreateCommand { - table_uri: String, - mode: SaveMode, - metadata: DeltaTableMetaData, - protocol: Protocol, +use crate::{DeltaTableBuilder, DeltaTableMetaData}; + +use super::transaction::commit; +use super::{MAX_SUPPORTED_READER_VERSION, MAX_SUPPORTED_WRITER_VERSION}; +use crate::action::{Action, DeltaOperation, MetaData, Protocol, SaveMode}; +use crate::builder::StorageUrl; +use crate::schema::{SchemaDataType, SchemaField, SchemaTypeStruct}; +use crate::storage::{utils::flatten_list_stream, DeltaObjectStore}; +use crate::{DeltaResult, DeltaTable, DeltaTableError}; + +use futures::future::BoxFuture; +use serde_json::Value; + +#[derive(thiserror::Error, Debug)] +enum CreateError { + #[error("Location must be provided to create a table")] + MissingLocation, + + #[error("At least one column must be defined to create a table")] + MissingSchema, + + #[error("Please configure table meta data via the CreateBuilder.")] + MetadataSpecified, + + #[error("Please configure table meta data via the CreateBuilder.")] + TableAlreadyExists, + + #[error("SaveMode `append` is not allowed for create operation.")] + AppendNotAllowed, } -impl CreateCommand { - /// Create new CreateCommand - pub fn try_new(table_uri: T, operation: DeltaOperation) -> Result - where - T: Into, - { - match operation { - DeltaOperation::Create { - metadata, - mode, - protocol, - .. - } => Ok(Self { - table_uri: table_uri.into(), - mode, - metadata, - protocol, - }), - _ => Err(DeltaCommandError::UnsupportedCommand( - "WriteCommand only implemented for write operation".to_string(), - )), +impl From for DeltaTableError { + fn from(err: CreateError) -> Self { + DeltaTableError::GenericError { + source: Box::new(err), } } } -impl std::fmt::Debug for CreateCommand { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "CreateCommand") +/// Build a new [DeltaTable] +#[derive(Debug, Clone)] +pub struct CreateBuilder { + name: Option, + location: Option, + mode: SaveMode, + comment: Option, + columns: Vec, + partition_columns: Option>, + properties: HashMap, + storage_options: Option>, + actions: Vec, + object_store: Option>, +} + +impl Default for CreateBuilder { + fn default() -> Self { + Self::new() } } -#[async_trait] -impl ExecutionPlan for CreateCommand { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { +impl CreateBuilder { + /// Create a new [`CreateBuilder`] + pub fn new() -> Self { + Self { + name: None, + location: None, + mode: SaveMode::ErrorIfExists, + comment: None, + columns: Vec::new(), + partition_columns: None, + properties: HashMap::new(), + storage_options: None, + actions: Vec::new(), + object_store: None, + } + } + + /// 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 } - fn schema(&self) -> SchemaRef { - Arc::new(OPERATION_SCHEMA.clone()) + /// Specify the path to the location where table data is stored, + /// which could be a path on distributed storage. + pub fn with_location(mut self, location: impl Into) -> Self { + self.location = Some(location.into()); + self } - fn children(&self) -> Vec> { - vec![] + /// 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 } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) + /// Comment to describe the table. + pub fn with_comment(mut self, comment: impl Into) -> Self { + self.comment = Some(comment.into()); + self } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + /// Specify a column in the table + pub fn with_column( + mut self, + name: impl Into, + data_type: SchemaDataType, + nullable: bool, + metadata: Option>, + ) -> Self { + self.columns.push(SchemaField::new( + name.into(), + data_type, + nullable, + metadata.unwrap_or_default(), + )); + self } - fn required_child_distribution(&self) -> Distribution { - // TODO - Distribution::SinglePartition + /// Specify columns to append to schema + pub fn with_columns(mut self, columns: impl IntoIterator) -> Self { + self.columns.extend(columns); + self } - fn with_new_children( - self: Arc, - _children: Vec>, - ) -> DataFusionResult> { - Err(DataFusionError::Internal( - "SortExec wrong number of children".to_string(), - )) + /// Specify table partitioning + pub fn with_partition_columns( + mut self, + partition_columns: impl IntoIterator>, + ) -> Self { + self.partition_columns = Some(partition_columns.into_iter().map(|s| s.into()).collect()); + self } - fn execute( - &self, - partition: usize, - _context: Arc, - ) -> DataFusionResult { - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - futures::stream::once( - do_create( - self.table_uri.clone(), - partition, - self.mode.clone(), - self.metadata.clone(), - self.protocol.clone(), - ) - .map_err(|e| ArrowError::ExternalError(Box::new(e))), - ) - .try_flatten(), - ))) + /// Specify a table property + pub fn with_property(mut self, key: impl Into, value: impl Into) -> Self { + self.properties.insert(key.into(), value.into()); + self } - fn statistics(&self) -> Statistics { - compute_record_batch_statistics(&[], &self.schema(), None) + /// Set options used to initialize storage backend + /// + /// Options may be passed in the HashMap or set as environment variables. + /// + /// [crate::builder::s3_storage_options] describes the available options for the AWS or S3-compliant backend. + /// [dynamodb_lock::DynamoDbLockClient] describes additional options for the AWS atomic rename client. + pub fn with_storage_options(mut self, storage_options: HashMap) -> Self { + self.storage_options = Some(storage_options); + self } -} -async fn do_create( - table_uri: String, - partition_id: usize, - mode: SaveMode, - metadata: DeltaTableMetaData, - protocol: Protocol, -) -> DataFusionResult { - let mut table = DeltaTableBuilder::from_uri(&table_uri) - .build() - .map_err(to_datafusion_err)?; - - let actions = match table.load_version(0).await { - Err(_) => Ok(vec![ - Action::protocol(protocol.clone()), - Action::metaData(MetaData::try_from(metadata.clone()).unwrap()), - ]), - Ok(_) => match mode { - SaveMode::Ignore => Ok(Vec::new()), - SaveMode::ErrorIfExists => { - Err(DeltaCommandError::TableAlreadyExists(table_uri.clone())) - } - _ => todo!("Write mode not implemented {:?}", mode), - }, + /// Specify additional actions to be added to the commit. + pub fn with_actions(mut self, actions: impl IntoIterator) -> Self { + self.actions.extend(actions); + self + } + + /// Provide a [`DeltaObjectStore`] instance, that points at table location + pub fn with_object_store(mut self, object_store: Arc) -> Self { + self.object_store = Some(object_store); + self + } + + /// Consume self into uninitialized table with corresponding crate actions and operation meta + pub(crate) fn into_table_and_actions( + self, + ) -> DeltaResult<(DeltaTable, Vec, DeltaOperation)> { + if self + .actions + .iter() + .any(|a| matches!(a, Action::metaData(_))) + { + return Err(CreateError::MetadataSpecified.into()); + } + if self.columns.is_empty() { + return Err(CreateError::MissingSchema.into()); + } + + let (table, storage_url) = if let Some(object_store) = self.object_store { + let storage_url = StorageUrl::parse(object_store.root_uri())?; + ( + DeltaTable::new(object_store, Default::default()), + storage_url, + ) + } else { + let storage_url = + StorageUrl::parse(self.location.ok_or(CreateError::MissingLocation)?)?; + ( + DeltaTableBuilder::from_uri(&storage_url) + .with_storage_options(self.storage_options.unwrap_or_default()) + .build()?, + storage_url, + ) + }; + + // TODO configure more permissive versions based on configuration. Also how should this ideally be handled? + // We set the lowest protocol we can, and if subsequent writes use newer features we update metadata? + let protocol = self + .actions + .iter() + .find(|a| matches!(a, Action::protocol(_))) + .map(|a| match a { + Action::protocol(p) => p.clone(), + _ => unreachable!(), + }) + .unwrap_or_else(|| Protocol { + min_reader_version: MAX_SUPPORTED_READER_VERSION, + min_writer_version: MAX_SUPPORTED_WRITER_VERSION, + }); + + let metadata = DeltaTableMetaData::new( + self.name, + self.comment, + None, + SchemaTypeStruct::new(self.columns), + self.partition_columns.unwrap_or_default(), + HashMap::new(), + ); + + let operation = DeltaOperation::Create { + mode: self.mode.clone(), + metadata: metadata.clone(), + location: storage_url.to_string(), + protocol: protocol.clone(), + }; + + let mut actions = vec![ + Action::protocol(protocol), + Action::metaData(MetaData::try_from(metadata)?), + ]; + actions.extend( + self.actions + .into_iter() + .filter(|a| matches!(a, Action::protocol(_))), + ); + + Ok((table, actions, operation)) } - .map_err(to_datafusion_err)?; +} + +impl std::future::IntoFuture for CreateBuilder { + type Output = DeltaResult; + type IntoFuture = BoxFuture<'static, Self::Output>; - let serialized_batch = serialize_actions(actions)?; - let metrics = ExecutionPlanMetricsSet::new(); - let tracking_metrics = MemTrackingMetrics::new(&metrics, partition_id); - let stream = SizedRecordBatchStream::new( - serialized_batch.schema(), - vec![Arc::new(serialized_batch)], - tracking_metrics, - ); + fn into_future(self) -> Self::IntoFuture { + let this = self; + + Box::pin(async move { + let mode = this.mode.clone(); + let (mut table, actions, operation) = this.into_table_and_actions()?; + if table.object_store().is_delta_table_location().await? { + match mode { + SaveMode::ErrorIfExists => return Err(CreateError::TableAlreadyExists.into()), + SaveMode::Append => return Err(CreateError::AppendNotAllowed.into()), + SaveMode::Ignore => { + table.load().await?; + return Ok(table); + } + SaveMode::Overwrite => { + let curr_files = + flatten_list_stream(table.object_store().as_ref(), None).await?; + table.object_store().delete_batch(&curr_files).await?; + } + } + } + let version = commit(&table.object_store(), 0, actions, operation, None).await?; + table.load_version(version).await?; - Ok(Box::pin(stream)) + Ok(table) + }) + } } #[cfg(test)] mod tests { use super::*; - use crate::{ - action::{DeltaOperation, Protocol}, - open_table, - operations::transaction::DeltaTransactionPlan, - DeltaTableMetaData, - }; - use datafusion::{physical_plan::collect, prelude::SessionContext}; - use std::collections::HashMap; - use std::process::Command; + use crate::operations::DeltaOps; + use crate::writer::test_utils::get_delta_schema; #[tokio::test] - async fn create_table_without_partitions() { - let table_schema = crate::writer::test_utils::get_delta_schema(); - let metadata = - DeltaTableMetaData::new(None, None, None, table_schema, vec![], HashMap::new()); + async fn test_create() { + let table_schema = get_delta_schema(); - let table_dir = tempfile::tempdir().unwrap(); - let table_path = table_dir.path(); - let table_uri = table_path.to_str().unwrap().to_string(); + let table = DeltaOps::new_in_memory() + .create() + .with_columns(table_schema.get_fields().clone()) + .with_save_mode(SaveMode::Ignore) + .await + .unwrap(); + assert_eq!(table.version(), 0) + } - let session_ctx = SessionContext::new(); - let task_ctx = session_ctx.task_ctx(); + #[tokio::test] + async fn test_create_table_metadata() { + let schema = get_delta_schema(); + let table = CreateBuilder::new() + .with_location("memory://") + .with_columns(schema.get_fields().clone()) + .await + .unwrap(); + assert_eq!(table.version(), 0); + assert_eq!(table.get_min_reader_version(), MAX_SUPPORTED_READER_VERSION); + assert_eq!(table.get_min_writer_version(), MAX_SUPPORTED_WRITER_VERSION); + assert_eq!(table.schema().unwrap(), &schema); - let transaction = - get_transaction(table_uri.clone(), -1, metadata.clone(), SaveMode::Ignore); - let _ = collect(transaction.clone(), task_ctx.clone()) + // check we can overwrite default settings via adding actions + let protocol = Protocol { + min_reader_version: 0, + min_writer_version: 0, + }; + let table = CreateBuilder::new() + .with_location("memory://") + .with_columns(schema.get_fields().clone()) + .with_actions(vec![Action::protocol(protocol)]) .await .unwrap(); + assert_eq!(table.get_min_reader_version(), 0); + assert_eq!(table.get_min_writer_version(), 0) + } - let table_path = std::path::Path::new(&table_uri); - let log_path = table_path.join("_delta_log/00000000000000000000.json"); - assert!(log_path.exists()); + #[tokio::test] + async fn test_create_table_save_mode() { + let tmp_dir = tempdir::TempDir::new("").unwrap(); - let mut table = open_table(&table_uri).await.unwrap(); + let schema = get_delta_schema(); + let table = CreateBuilder::new() + .with_location(tmp_dir.path().to_str().unwrap()) + .with_columns(schema.get_fields().clone()) + .await + .unwrap(); assert_eq!(table.version(), 0); + let first_id = table.get_metadata().unwrap().id.clone(); - // Check we can create an existing table with ignore - let ts1 = table.get_version_timestamp(0).await.unwrap(); - let mut child = Command::new("sleep").arg("1").spawn().unwrap(); - let _result = child.wait().unwrap(); - let _ = collect(transaction, task_ctx.clone()).await.unwrap(); - let mut table = open_table(&table_uri).await.unwrap(); - assert_eq!(table.version(), 0); - let ts2 = table.get_version_timestamp(0).await.unwrap(); - assert_eq!(ts1, ts2); - - // Check error for ErrorIfExists mode - let transaction = get_transaction(table_uri, 0, metadata, SaveMode::ErrorIfExists); - let result = collect(transaction.clone(), task_ctx).await; - assert!(result.is_err()) - } - - fn get_transaction( - table_uri: String, - table_version: i64, - metadata: DeltaTableMetaData, - mode: SaveMode, - ) -> Arc { - let op = DeltaOperation::Create { - location: table_uri.clone(), - metadata, - mode, - protocol: Protocol { - min_reader_version: 1, - min_writer_version: 1, - }, - }; + // Check an error is raised when a table exists at location + let table = CreateBuilder::new() + .with_location(tmp_dir.path().to_str().unwrap()) + .with_columns(schema.get_fields().clone()) + .with_save_mode(SaveMode::ErrorIfExists) + .await; + assert!(table.is_err()); - Arc::new(DeltaTransactionPlan::new( - table_uri.clone(), - table_version, - Arc::new(CreateCommand::try_new(table_uri, op.clone()).unwrap()), - op, - None, - )) + // Check current table is returned when ignore option is chosen. + let table = CreateBuilder::new() + .with_location(tmp_dir.path().to_str().unwrap()) + .with_columns(schema.get_fields().clone()) + .with_save_mode(SaveMode::Ignore) + .await + .unwrap(); + assert_eq!(table.get_metadata().unwrap().id, first_id); + + // Check table is overwritten + let table = CreateBuilder::new() + .with_location(tmp_dir.path().to_str().unwrap()) + .with_columns(schema.get_fields().clone()) + .with_save_mode(SaveMode::Overwrite) + .await + .unwrap(); + assert_ne!(table.get_metadata().unwrap().id, first_id) } } diff --git a/rust/src/operations/load.rs b/rust/src/operations/load.rs new file mode 100644 index 0000000000..80487f4ec5 --- /dev/null +++ b/rust/src/operations/load.rs @@ -0,0 +1,139 @@ +use std::collections::HashMap; +use std::sync::Arc; + +use crate::storage::DeltaObjectStore; +use crate::{DeltaResult, DeltaTable}; + +use datafusion::datasource::TableProvider; +use datafusion::execution::context::{SessionContext, TaskContext}; +use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; +use futures::future::BoxFuture; + +// #[derive(thiserror::Error, Debug)] +// enum LoadError { +// Task, +// } +// +// impl From for DeltaTableError { +// fn from(err: LoadError) -> Self { +// DeltaTableError::GenericError { +// source: Box::new(err), +// } +// } +// } + +#[derive(Debug, Clone)] +pub struct LoadBuilder { + location: Option, + columns: Option>, + storage_options: Option>, + object_store: Option>, +} + +impl Default for LoadBuilder { + fn default() -> Self { + Self::new() + } +} + +impl LoadBuilder { + pub fn new() -> Self { + Self { + location: None, + columns: None, + storage_options: None, + object_store: None, + } + } + + /// Specify the path to the location where table data is stored, + /// which could be a path on distributed storage. + pub fn with_location(mut self, location: impl Into) -> Self { + self.location = Some(location.into()); + self + } + + /// Specify column selection to load + pub fn with_columns(mut self, columns: impl IntoIterator>) -> Self { + self.columns = Some(columns.into_iter().map(|s| s.into()).collect()); + self + } + + /// Set options used to initialize storage backend + /// + /// Options may be passed in the HashMap or set as environment variables. + /// + /// [crate::builder::s3_storage_options] describes the available options for the AWS or S3-compliant backend. + /// [dynamodb_lock::DynamoDbLockClient] describes additional options for the AWS atomic rename client. + pub fn with_storage_options(mut self, storage_options: HashMap) -> Self { + self.storage_options = Some(storage_options); + self + } + + /// Provide a [`DeltaObjectStore`] instance, that points at table location + pub fn with_object_store(mut self, object_store: Arc) -> Self { + self.object_store = Some(object_store); + self + } +} + +impl std::future::IntoFuture for LoadBuilder { + type Output = DeltaResult<(DeltaTable, SendableRecordBatchStream)>; + type IntoFuture = BoxFuture<'static, Self::Output>; + + fn into_future(self) -> Self::IntoFuture { + Box::pin(async move { + let object_store = self.object_store.unwrap(); + let mut table = DeltaTable::new(object_store, Default::default()); + table.load().await?; + + let ctx = SessionContext::new(); + let scan_plan = table.scan(&ctx.state(), &None, &[], None).await?; + let plan = CoalescePartitionsExec::new(scan_plan); + let task_ctx = Arc::new(TaskContext::from(&ctx.state())); + let stream = plan.execute(0, task_ctx)?; + Ok((table, stream)) + }) + } +} + +#[cfg(test)] +mod tests { + use crate::operations::{collect_sendable_stream, DeltaOps}; + use crate::writer::test_utils::{get_record_batch, TestResult}; + use arrow::record_batch::RecordBatch; + use datafusion::assert_batches_sorted_eq; + + #[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, stream) = DeltaOps(table).load().await?; + let data = collect_sendable_stream(stream).await?; + let batch_loaded = RecordBatch::concat(&data[0].schema(), &data)?; + + let expected = vec![ + "+----+-------+------------+", + "| id | value | modified |", + "+----+-------+------------+", + "| A | 1 | 2021-02-02 |", + "| B | 2 | 2021-02-02 |", + "| A | 3 | 2021-02-02 |", + "| B | 4 | 2021-02-01 |", + "| A | 5 | 2021-02-01 |", + "| A | 6 | 2021-02-01 |", + "| A | 7 | 2021-02-01 |", + "| B | 8 | 2021-02-01 |", + "| B | 9 | 2021-02-01 |", + "| A | 10 | 2021-02-01 |", + "| A | 11 | 2021-02-01 |", + "+----+-------+------------+", + ]; + + assert_batches_sorted_eq!(&expected, &[batch_loaded.clone()]); + assert_eq!(batch.schema(), batch_loaded.schema()); + Ok(()) + } +} diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index e417e3779e..05ae56bbf6 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -1,329 +1,90 @@ //! High level delta commands that can be executed against a delta table -// TODO -// - rename to delta operations -use std::collections::HashMap; -use std::convert::TryFrom; -use std::fmt::Debug; -use std::sync::Arc; -use crate::{ - action::{DeltaOperation, Protocol, SaveMode}, - builder::DeltaTableBuilder, - open_table, - operations::{create::CreateCommand, transaction::DeltaTransactionPlan, write::WriteCommand}, - writer::{record_batch::divide_by_partition_values, utils::PartitionPath}, - DeltaTable, DeltaTableError, DeltaTableMetaData, -}; - -use arrow::{datatypes::SchemaRef as ArrowSchemaRef, error::ArrowError, record_batch::RecordBatch}; -use datafusion::{ - physical_plan::{collect, memory::MemoryExec, ExecutionPlan}, - prelude::SessionContext, -}; -use datafusion_common::DataFusionError; +use self::create::CreateBuilder; +use crate::builder::DeltaTableBuilder; +use crate::{DeltaResult, DeltaTable, DeltaTableError}; pub mod create; pub mod transaction; -pub mod write; - -type DeltaCommandResult = Result; - -/// Enum representing an error when calling [DeltaCommands.execute]. -#[derive(thiserror::Error, Debug)] -pub enum DeltaCommandError { - /// Error returned when some data is expected but only an empty dataset is provided. - #[error("Received empty data partition {0}")] - EmptyPartition(String), - - /// Error returned when the provided command is not supported - #[error("Command not available {0}")] - UnsupportedCommand(String), - - /// Error returned when the table requires an unsupported writer version - #[error("Delta-rs does not support writer version {0}")] - UnsupportedWriterVersion(i32), - /// Error returned when the table to be created already exists - #[error("Table: '{0}' already exists")] - TableAlreadyExists(String), - - /// Error returned when errors occur in underlying delta table instance - #[error("DeltaTable error: {} ({:?})", source, source)] - DeltaTable { - /// Raw internal DeltaTableError - #[from] - source: DeltaTableError, - }, - - /// Error returned when errors occur in Arrow - #[error("Arrow error: {} ({:?})", source, source)] - Arrow { - /// Raw internal ArrowError - #[from] - source: ArrowError, - }, - - /// Error returned for errors internal to Datafusion - #[error("Datafusion error: {} ({:?})", source, source)] - DataFusion { - /// Raw internal DataFusionError - source: DataFusionError, - }, - - /// Error returned for errors internal to Datafusion - #[error("ObjectStore error: {} ({:?})", source, source)] - ObjectStore { - /// Raw internal DataFusionError - #[from] - source: object_store::Error, - }, -} +#[cfg(feature = "datafusion-ext")] +use self::{load::LoadBuilder, write::WriteBuilder}; +#[cfg(feature = "datafusion-ext")] +use arrow::record_batch::RecordBatch; +#[cfg(feature = "datafusion-ext")] +pub use datafusion::physical_plan::common::collect as collect_sendable_stream; -impl From for DeltaCommandError { - fn from(err: DataFusionError) -> Self { - match err { - DataFusionError::ArrowError(source) => DeltaCommandError::Arrow { source }, - DataFusionError::ObjectStore(source) => DeltaCommandError::ObjectStore { source }, - source => DeltaCommandError::DataFusion { source }, - } - } -} +#[cfg(feature = "datafusion-ext")] +mod load; +#[cfg(feature = "datafusion-ext")] +pub mod write; +// TODO the writer module does not actually depend on datafusion, +// eventually we should consolidate with the record batch writer +#[cfg(feature = "datafusion-ext")] +mod writer; -fn to_datafusion_err(e: impl std::error::Error) -> DataFusionError { - DataFusionError::Plan(e.to_string()) -} +/// Maximum supported writer version +pub const MAX_SUPPORTED_WRITER_VERSION: i32 = 1; +/// Maximum supported reader version +pub const MAX_SUPPORTED_READER_VERSION: i32 = 1; /// High level interface for executing commands against a DeltaTable -pub struct DeltaCommands { - table: DeltaTable, -} +pub struct DeltaOps(DeltaTable); -impl DeltaCommands { +impl DeltaOps { /// load table from uri - pub async fn try_from_uri(uri: T) -> DeltaCommandResult - where - T: Into, - { - let table_uri: String = uri.into(); - let table = if let Ok(tbl) = open_table(&table_uri).await { - Ok(tbl) - } else { - DeltaTableBuilder::from_uri(table_uri).build() - }?; - Ok(Self { table }) - } - - /// Get a reference to the underlying table - pub fn table(&self) -> &DeltaTable { - &self.table + pub async fn try_from_uri(uri: impl AsRef) -> DeltaResult { + let mut table = DeltaTableBuilder::from_uri(uri).build()?; + // We allow for uninitialized locations, since we may want to create the table + match table.load().await { + Ok(_) => Ok(table.into()), + Err(DeltaTableError::NotATable(_)) => Ok(table.into()), + Err(err) => Err(err), + } } - async fn execute( - &mut self, - operation: DeltaOperation, - plan: Arc, - ) -> DeltaCommandResult<()> { - let transaction = Arc::new(DeltaTransactionPlan::new( - self.table.table_uri(), - self.table.version(), - plan, - operation, - None, - )); - - let session_ctx = SessionContext::new(); - let task_ctx = session_ctx.task_ctx(); - let _ = collect(transaction, task_ctx).await?; - self.table.update().await?; - - Ok(()) + /// Create a new [`DeltaOps`] instance, backed by an un-initialized in memory table + pub fn new_in_memory() -> Self { + DeltaTableBuilder::from_uri("memory://") + .build() + .unwrap() + .into() } /// Create a new Delta table - pub async fn create( - &mut self, - metadata: DeltaTableMetaData, - mode: SaveMode, - ) -> DeltaCommandResult<()> { - let operation = DeltaOperation::Create { - mode, - metadata: metadata.clone(), - location: self.table.table_uri(), - // TODO get the protocol from somewhere central - protocol: Protocol { - min_reader_version: 1, - min_writer_version: 1, - }, - }; - let plan = Arc::new(CreateCommand::try_new( - self.table.table_uri(), - operation.clone(), - )?); - - self.execute(operation, plan).await + pub fn create(self) -> CreateBuilder { + CreateBuilder::default().with_object_store(self.0.object_store()) } /// Write data to Delta table - pub async fn write( - &mut self, - data: Vec, - mode: SaveMode, - partition_columns: Option>, - ) -> DeltaCommandResult<()> { - if data.is_empty() { - return Ok(()); - } - let schema = data[0].schema(); - if let Ok(meta) = self.table.get_metadata() { - let curr_schema: ArrowSchemaRef = Arc::new((&meta.schema).try_into()?); - if schema != curr_schema { - return Err(DeltaCommandError::UnsupportedCommand( - "Updating table schema not yet implemented".to_string(), - )); - } - if let Some(cols) = partition_columns.as_ref() { - if cols != &meta.partition_columns { - return Err(DeltaCommandError::UnsupportedCommand( - "Updating table partitions not yet implemented".to_string(), - )); - } - }; - }; - let data = if let Some(cols) = partition_columns.as_ref() { - // TODO partitioning should probably happen in its own plan ... - let mut partitions: HashMap> = HashMap::new(); - for batch in data { - let divided = - divide_by_partition_values(schema.clone(), cols.clone(), &batch).unwrap(); - for part in divided { - let key = PartitionPath::from_hashmap(cols, &part.partition_values) - .map_err(DeltaTableError::from)? - .into(); - match partitions.get_mut(&key) { - Some(batches) => { - batches.push(part.record_batch); - } - None => { - partitions.insert(key, vec![part.record_batch]); - } - } - } - } - partitions.into_values().collect::>() - } else { - vec![data] - }; + #[cfg(feature = "datafusion-ext")] + pub fn load(self) -> LoadBuilder { + LoadBuilder::default().with_object_store(self.0.object_store()) + } - let operation = DeltaOperation::Write { - mode, - partition_by: partition_columns.clone(), - predicate: None, - }; - let data_plan = Arc::new(MemoryExec::try_new(&data, schema, None)?); - let plan = Arc::new(WriteCommand::try_new( - self.table.table_uri(), - operation.clone(), - data_plan, - )?); - self.execute(operation, plan).await + /// Write data to Delta table + #[cfg(feature = "datafusion-ext")] + pub fn write(self, batches: Vec) -> WriteBuilder { + WriteBuilder::default() + .with_input_batches(batches) + .with_object_store(self.0.object_store()) } } -impl From for DeltaCommands { +impl From for DeltaOps { fn from(table: DeltaTable) -> Self { - Self { table } + Self(table) } } -impl From for DeltaTable { - fn from(comm: DeltaCommands) -> Self { - comm.table +impl From for DeltaTable { + fn from(comm: DeltaOps) -> Self { + comm.0 } } -#[cfg(test)] -mod tests { - use super::*; - use crate::{ - open_table, - writer::test_utils::{create_initialized_table, get_delta_schema, get_record_batch}, - }; - - #[tokio::test] - async fn test_create_command() { - let table_dir = tempfile::tempdir().unwrap(); - let table_path = table_dir.path(); - let table_uri = table_path.to_str().unwrap().to_string(); - - let mut commands = DeltaCommands::try_from_uri(table_uri.clone()) - .await - .unwrap(); - - let table_schema = get_delta_schema(); - let metadata = - DeltaTableMetaData::new(None, None, None, table_schema, vec![], HashMap::new()); - - commands - .create(metadata.clone(), SaveMode::Ignore) - .await - .unwrap(); - - let table = open_table(&table_uri).await.unwrap(); - assert_eq!(table.version(), 0); - - let res = commands.create(metadata, SaveMode::ErrorIfExists).await; - assert!(res.is_err()) - } - - #[tokio::test] - async fn test_write_command() { - let batch = get_record_batch(None, false); - let partition_cols = vec!["modified".to_string()]; - let mut table = create_initialized_table(&partition_cols).await; - assert_eq!(table.version(), 0); - - let mut commands = DeltaCommands::try_from_uri(table.table_uri()) - .await - .unwrap(); - - commands - .write( - vec![batch], - SaveMode::Append, - Some(vec!["modified".to_string()]), - ) - .await - .unwrap(); - - table.update().await.unwrap(); - assert_eq!(table.version(), 1); - - let files = table.get_file_uris(); - assert_eq!(files.count(), 2) - } - - #[tokio::test] - async fn test_create_and_write_command() { - let table_dir = tempfile::tempdir().unwrap(); - let table_path = table_dir.path(); - let table_uri = table_path.to_str().unwrap().to_string(); - - let mut commands = DeltaCommands::try_from_uri(&table_uri).await.unwrap(); - - let batch = get_record_batch(None, false); - commands - .write( - vec![batch], - SaveMode::Append, - Some(vec!["modified".to_string()]), - ) - .await - .unwrap(); - - let table = open_table(&table_uri).await.unwrap(); - assert_eq!(table.version(), 0); - - let files = table.get_file_uris(); - assert_eq!(files.count(), 2) +impl AsRef for DeltaOps { + fn as_ref(&self) -> &DeltaTable { + &self.0 } } diff --git a/rust/src/operations/transaction.rs b/rust/src/operations/transaction.rs index 8071cbd65c..27d3c283e7 100644 --- a/rust/src/operations/transaction.rs +++ b/rust/src/operations/transaction.rs @@ -1,194 +1,205 @@ -//! Wrapper Execution plan to handle distributed operations -use core::any::Any; +//! Delta transactions use std::sync::Arc; -use super::*; -use crate::action::Action; -use crate::schema::DeltaDataTypeVersion; - -use arrow::array::StringArray; -use arrow::datatypes::{ - DataType, Field as ArrowField, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, -}; -use arrow::record_batch::RecordBatch; -use datafusion::{ - execution::context::TaskContext, - physical_plan::{ - coalesce_partitions::CoalescePartitionsExec, common::compute_record_batch_statistics, - empty::EmptyExec, expressions::PhysicalSortExpr, stream::RecordBatchStreamAdapter, - Distribution, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, - }, -}; -use datafusion_common::Result as DataFusionResult; -use futures::{TryFutureExt, TryStreamExt}; -use lazy_static::lazy_static; - -lazy_static! { - /// Schema expected for plans wrapped by transaction - pub static ref OPERATION_SCHEMA: ArrowSchema = - ArrowSchema::new(vec![ArrowField::new("serialized", DataType::Utf8, false,)]); -} +use crate::action::{Action, DeltaOperation}; +use crate::storage::DeltaObjectStore; +use crate::{crate_version, DeltaDataTypeVersion, DeltaResult, DeltaTableError}; -pub(crate) fn serialize_actions(actions: Vec) -> DataFusionResult { - let serialized = StringArray::from( - actions - .iter() - .map(serde_json::to_string) - .collect::, _>>() - .map_err(to_datafusion_err)?, - ); - Ok(RecordBatch::try_new( - Arc::new(OPERATION_SCHEMA.clone()), - vec![Arc::new(serialized)], - )?) -} +use chrono::Utc; +use object_store::path::Path; +use object_store::{Error as ObjectStoreError, ObjectStore}; +use serde_json::{Map, Value}; -/// Write command -#[derive(Debug)] -pub struct DeltaTransactionPlan { - table_uri: String, - table_version: DeltaDataTypeVersion, - input: Arc, - operation: DeltaOperation, - app_metadata: Option>, -} +const DELTA_LOG_FOLDER: &str = "_delta_log"; -impl DeltaTransactionPlan { - /// Wrap partitioned delta operations in a DeltaTransaction - pub fn new( - table_uri: T, - table_version: DeltaDataTypeVersion, - input: Arc, - operation: DeltaOperation, - app_metadata: Option>, - ) -> Self - where - T: Into, - { - Self { - table_uri: table_uri.into(), - table_version, - input: Arc::new(CoalescePartitionsExec::new(input)), - operation, - app_metadata, - } - } +#[derive(thiserror::Error, Debug)] +enum TransactionError { + #[error("Tried committing existing table version: {0}")] + VersionAlreadyExists(DeltaDataTypeVersion), - /// Arrow schema expected to be produced by wrapped operations - pub fn input_operation_schema() -> ArrowSchemaRef { - Arc::new(OPERATION_SCHEMA.clone()) - } -} + /// Error returned when reading the delta log object failed. + #[error("Error serializing commit: {}", .source)] + Serialize { + /// Storage error details when reading the delta log object failed. + #[from] + source: serde_json::Error, + }, -impl ExecutionPlan for DeltaTransactionPlan { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } + /// Error returned when reading the delta log object failed. + #[error("Log storage error: {}", .source)] + ObjectStore { + /// Storage error details when reading the delta log object failed. + #[from] + source: ObjectStoreError, + }, +} - fn schema(&self) -> ArrowSchemaRef { - Arc::new(OPERATION_SCHEMA.clone()) +impl From for DeltaTableError { + fn from(err: TransactionError) -> Self { + match err { + TransactionError::VersionAlreadyExists(version) => { + DeltaTableError::VersionAlreadyExists(version) + } + TransactionError::Serialize { source } => DeltaTableError::InvalidJson { source }, + TransactionError::ObjectStore { source } => DeltaTableError::ObjectStore { source }, + } } +} - fn children(&self) -> Vec> { - vec![self.input.clone()] - } +/// Return the uri of commit version. +fn commit_uri_from_version(version: DeltaDataTypeVersion) -> Path { + let version = format!("{:020}.json", version); + Path::from_iter([DELTA_LOG_FOLDER, &version]) +} - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) +// Convert actions to their json representation +fn log_entry_from_actions(actions: &[Action]) -> Result { + let mut jsons = Vec::::new(); + for action in actions { + let json = serde_json::to_string(action)?; + jsons.push(json); } + Ok(jsons.join("\n")) +} - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None +/// Low-level transaction API. Creates a temporary commit file. Once created, +/// the transaction object could be dropped and the actual commit could be executed +/// with `DeltaTable.try_commit_transaction`. +async fn prepare_commit( + storage: &Arc, + operation: DeltaOperation, + mut actions: Vec, + app_metadata: Option>, +) -> Result { + if !actions.iter().any(|a| matches!(a, Action::commitInfo(..))) { + let mut commit_info = Map::::new(); + commit_info.insert( + "timestamp".to_string(), + Value::Number(serde_json::Number::from(Utc::now().timestamp_millis())), + ); + commit_info.insert( + "clientVersion".to_string(), + Value::String(format!("delta-rs.{}", crate_version())), + ); + commit_info.append(&mut operation.get_commit_info()); + if let Some(mut meta) = app_metadata { + commit_info.append(&mut meta) + } + actions.push(Action::commitInfo(commit_info)); } - fn required_child_distribution(&self) -> Distribution { - Distribution::UnspecifiedDistribution - } + // Serialize all actions that are part of this log entry. + let log_entry = bytes::Bytes::from(log_entry_from_actions(&actions)?); - fn with_new_children( - self: Arc, - _children: Vec>, - ) -> DataFusionResult> { - todo!() - } + // Write delta log entry as temporary file to storage. For the actual commit, + // the temporary file is moved (atomic rename) to the delta log folder within `commit` function. + let token = uuid::Uuid::new_v4().to_string(); + let file_name = format!("_commit_{}.json.tmp", token); + let path = Path::from_iter([DELTA_LOG_FOLDER, &file_name]); + storage.put(&path, log_entry).await?; - fn execute( - &self, - _partition: usize, - context: Arc, - ) -> DataFusionResult { - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - futures::stream::once( - do_transaction( - self.input.clone(), - self.table_uri.clone(), - self.table_version, - self.operation.clone(), - self.app_metadata.clone(), - context, - ) - .map_err(|e| ArrowError::ExternalError(Box::new(e))), - ) - .try_flatten(), - ))) - } + Ok(path) +} - fn statistics(&self) -> Statistics { - compute_record_batch_statistics(&[], &self.schema(), None) - } +/// Tries to commit a prepared commit file. Returns [`DeltaTableError::VersionAlreadyExists`] +/// if the given `version` already exists. The caller should handle the retry logic itself. +/// This is low-level transaction API. If user does not want to maintain the commit loop then +/// the `DeltaTransaction.commit` is desired to be used as it handles `try_commit_transaction` +/// with retry logic. +async fn try_commit_transaction( + storage: &Arc, + tmp_commit: &Path, + version: DeltaDataTypeVersion, +) -> Result { + // move temporary commit file to delta log directory + // rely on storage to fail if the file already exists - + storage + .rename_if_not_exists(tmp_commit, &commit_uri_from_version(version)) + .await + .map_err(|err| match err { + ObjectStoreError::AlreadyExists { .. } => { + TransactionError::VersionAlreadyExists(version) + } + _ => TransactionError::from(err), + })?; + Ok(version) } -async fn do_transaction( - input: Arc, - table_uri: String, - table_version: i64, +pub(crate) async fn commit( + storage: &Arc, + version: DeltaDataTypeVersion, + actions: Vec, operation: DeltaOperation, - app_metadata: Option>, - context: Arc, -) -> DataFusionResult { - let mut table = DeltaTableBuilder::from_uri(table_uri) - .build() - .map_err(to_datafusion_err)?; - let schema = input.schema().clone(); - - let data = collect(input, context.clone()).await?; - // TODO we assume that all children send a single column record batch with serialized actions - let actions = data - .iter() - .flat_map(|batch| match deserialize_actions(batch) { - Ok(vec) => vec.into_iter().map(Ok).collect(), - Err(er) => vec![Err(er)], - }) - .collect::, _>>()?; - - if actions.is_empty() { - let empty_plan = EmptyExec::new(false, schema); - return empty_plan.execute(0, context); + app_metadata: Option>, +) -> DeltaResult { + let tmp_commit = prepare_commit(storage, operation, actions, app_metadata).await?; + match try_commit_transaction(storage, &tmp_commit, version).await { + Ok(version) => Ok(version), + Err(TransactionError::VersionAlreadyExists(version)) => { + storage.delete(&tmp_commit).await?; + Err(DeltaTableError::VersionAlreadyExists(version)) + } + Err(err) => Err(err.into()), } - - let mut txn = table.create_transaction(None); - txn.add_actions(actions); - let prepared_commit = txn - .prepare_commit(Some(operation.clone()), app_metadata.clone()) - .await - .map_err(to_datafusion_err)?; - let _committed_version = table - .try_commit_transaction(&prepared_commit, table_version + 1) - .await - .map_err(to_datafusion_err)?; - - // TODO report some helpful data - at least current version - let empty_plan = EmptyExec::new(false, schema); - empty_plan.execute(0, context) } -fn deserialize_actions(data: &RecordBatch) -> DataFusionResult> { - let serialized_actions = arrow::array::as_string_array(data.column(0)); - serialized_actions - .iter() - .map(|val| serde_json::from_str::(val.unwrap_or(""))) - .collect::, _>>() - .map_err(to_datafusion_err) +#[cfg(test)] +mod tests { + use super::*; + use crate::action::{DeltaOperation, Protocol, SaveMode}; + use crate::storage::utils::flatten_list_stream; + use crate::writer::test_utils::get_delta_metadata; + use crate::{DeltaTable, DeltaTableBuilder}; + + #[test] + fn test_commit_version() { + let version = commit_uri_from_version(0); + assert_eq!(version, Path::from("_delta_log/00000000000000000000.json")); + let version = commit_uri_from_version(123); + assert_eq!(version, Path::from("_delta_log/00000000000000000123.json")) + } + + #[tokio::test] + async fn test_commits_writes_file() { + let metadata = get_delta_metadata(&vec![]); + let operation = DeltaOperation::Create { + mode: SaveMode::Append, + location: "memory://".into(), + protocol: Protocol { + min_reader_version: 1, + min_writer_version: 1, + }, + metadata, + }; + + let commit_path = Path::from("_delta_log/00000000000000000000.json"); + let storage = DeltaTableBuilder::from_uri("memory://") + .build_storage() + .unwrap(); + + // successfully write in clean location + commit(&storage, 0, vec![], operation.clone(), None) + .await + .unwrap(); + let head = storage.head(&commit_path).await; + assert!(head.is_ok()); + assert_eq!(head.as_ref().unwrap().location, commit_path); + + // fail on overwriting + let failed_commit = commit(&storage, 0, vec![], operation, None).await; + assert!(failed_commit.is_err()); + assert!(matches!( + failed_commit.unwrap_err(), + DeltaTableError::VersionAlreadyExists(_) + )); + + // check we clean up after ourselves + let objects = flatten_list_stream(storage.as_ref(), None).await.unwrap(); + assert_eq!(objects.len(), 1); + + // table can be loaded + let mut table = DeltaTable::new(storage, Default::default()); + table.load().await.unwrap(); + assert_eq!(table.version(), 0) + } } diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index d26b58a64e..956adfb1b3 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -1,4 +1,4 @@ -//! Used to write a [RecordBatch] into a delta table. +//! Used to write [RecordBatch]es into a delta table. //! //! New Table Semantics //! - The schema of the [RecordBatch] is used to initialize the table. @@ -6,7 +6,7 @@ //! //! Existing Table Semantics //! - The save mode will control how existing data is handled (i.e. overwrite, append, etc) -//! - The schema of the RecordBatch will be checked and if there are new columns present +//! - (NOT YET IMPLEMENTED) The schema of the RecordBatch will be checked and if there are new columns present //! they will be added to the tables schema. Conflicting columns (i.e. a INT, and a STRING) //! will result in an exception. //! - The partition columns, if present, are validated against the existing metadata. If not @@ -14,504 +14,455 @@ //! //! In combination with `Overwrite`, a `replaceWhere` option can be used to transactionally //! replace data that matches a predicate. +//! +//! # Example // https://github.com/delta-io/delta/blob/master/core/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala -use core::any::Any; + use std::collections::HashMap; use std::sync::Arc; use std::time::{SystemTime, UNIX_EPOCH}; -use super::{ - create::CreateCommand, - transaction::{serialize_actions, OPERATION_SCHEMA}, - *, -}; -use crate::action::{Action, Add, Remove, SaveMode}; -use crate::writer::{DeltaWriter, RecordBatchWriter}; -use crate::Schema; +use super::writer::{PartitionWriter, PartitionWriterConfig}; +use super::MAX_SUPPORTED_WRITER_VERSION; +use super::{transaction::commit, CreateBuilder}; +use crate::action::{Action, Add, DeltaOperation, Remove, SaveMode}; +use crate::builder::DeltaTableBuilder; +use crate::delta::{DeltaResult, DeltaTable, DeltaTableError}; +use crate::schema::Schema; +use crate::storage::DeltaObjectStore; +use crate::writer::record_batch::divide_by_partition_values; +use crate::writer::utils::PartitionPath; use arrow::datatypes::SchemaRef as ArrowSchemaRef; -use datafusion::{ - execution::context::TaskContext, - physical_plan::{ - common::{ - collect as collect_batch, compute_record_batch_statistics, SizedRecordBatchStream, - }, - expressions::PhysicalSortExpr, - metrics::{ExecutionPlanMetricsSet, MemTrackingMetrics}, - stream::RecordBatchStreamAdapter, - Distribution, EmptyRecordBatchStream, ExecutionPlan, Partitioning, - SendableRecordBatchStream, Statistics, - }, -}; -use datafusion_common::Result as DataFusionResult; -use futures::{TryFutureExt, TryStreamExt}; - -const MAX_SUPPORTED_WRITER_VERSION: i32 = 1; - -/// Command for writing data into Delta table -#[derive(Debug)] -pub struct WriteCommand { - table_uri: String, - /// The save mode used in operation +use arrow::record_batch::RecordBatch; +use datafusion::execution::context::{SessionContext, TaskContext}; +use datafusion::physical_plan::{memory::MemoryExec, ExecutionPlan}; +use futures::future::BoxFuture; +use futures::StreamExt; + +#[derive(thiserror::Error, Debug)] +enum WriteError { + #[error("No data source supplied to write command.")] + MissingData, + + #[error("Failed to execute write task: {source}")] + WriteTask { source: tokio::task::JoinError }, + + #[error("Delta-rs does not support writer version requirement: {0}")] + UnsupportedWriterVersion(i32), + + #[error("A table already exists at: {0}")] + AlreadyExists(String), +} + +impl From for DeltaTableError { + fn from(err: WriteError) -> Self { + DeltaTableError::GenericError { + source: Box::new(err), + } + } +} + +/// Builder for write operations +#[derive(Debug, Clone)] +pub struct WriteBuilder { + /// The input plan + input: Option>, + /// Location where the table is stored + location: Option, + /// SaveMode defines how to treat data already written to table location mode: SaveMode, - /// Column names for table partitioning partition_columns: Option>, /// When using `Overwrite` mode, replace data that matches a predicate predicate: Option, - /// Schema of data to be written to disk - schema: ArrowSchemaRef, - /// The input plan - input: Arc, + /// Size above which we will write a buffered parquet file to disk. + target_file_size: Option, + object_store: Option>, + storage_options: Option>, + batches: Option>, } -impl WriteCommand { - /// Create a new write command - pub fn try_new( - table_uri: T, - operation: DeltaOperation, - input: Arc, - ) -> Result - where - T: Into + Clone, - { - match operation { - DeltaOperation::Write { - mode, - partition_by, - predicate, - } => { - let uri = table_uri.into(); - let plan = Arc::new(WritePartitionCommand::new( - uri.clone(), - partition_by.clone(), - input.clone(), - )); - Ok(Self { - table_uri: uri, - mode, - partition_columns: partition_by, - predicate, - schema: input.schema(), - input: plan, - }) - } - _ => Err(DeltaCommandError::UnsupportedCommand( - "WriteCommand only implemented for write operation".to_string(), - )), - } +impl Default for WriteBuilder { + fn default() -> Self { + Self::new() } } -impl ExecutionPlan for WriteCommand { - fn as_any(&self) -> &dyn Any { +impl WriteBuilder { + /// Create a new [`WriteBuilder`] + pub fn new() -> Self { + Self { + input: None, + location: None, + mode: SaveMode::Append, + partition_columns: None, + predicate: None, + storage_options: None, + target_file_size: None, + object_store: None, + batches: None, + } + } + + /// Specify the path to the location where table data is stored, + /// which could be a path on distributed storage. + pub fn with_location(mut self, location: impl Into) -> Self { + self.location = Some(location.into()); self } - fn schema(&self) -> ArrowSchemaRef { - Arc::new(OPERATION_SCHEMA.clone()) + /// 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 } - fn children(&self) -> Vec> { - vec![self.input.clone()] + /// 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 } - fn output_partitioning(&self) -> Partitioning { - self.input.output_partitioning() + /// (Optional) Specify table partitioning. If specified, the partitioning is validated, + /// if the table already exists. In case a new table is created, the partitioning is applied. + pub fn with_partition_columns( + mut self, + partition_columns: impl IntoIterator>, + ) -> Self { + self.partition_columns = Some(partition_columns.into_iter().map(|s| s.into()).collect()); + self } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + /// 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 } - fn required_child_distribution(&self) -> Distribution { - Distribution::UnspecifiedDistribution + /// 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()); + self } - fn with_new_children( - self: Arc, - _children: Vec>, - ) -> DataFusionResult> { - todo!() + /// Set options used to initialize storage backend + /// + /// Options may be passed in the HashMap or set as environment variables. + /// + /// [crate::builder::s3_storage_options] describes the available options for the AWS or S3-compliant backend. + /// [dynamodb_lock::DynamoDbLockClient] describes additional options for the AWS atomic rename client. + pub fn with_storage_options(mut self, storage_options: HashMap) -> Self { + self.storage_options = Some(storage_options); + self } - fn execute( - &self, - partition: usize, - context: Arc, - ) -> DataFusionResult { - let input = self.input.execute(partition, context.clone())?; - - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - futures::stream::once( - do_write( - input, - self.table_uri.clone(), - partition, - self.partition_columns.clone(), - self.schema.clone(), - self.predicate.clone(), - self.mode.clone(), - context, - ) - .map_err(|e| ArrowError::ExternalError(Box::new(e))), - ) - .try_flatten(), - ))) + /// Provide a [`DeltaObjectStore`] instance, that points at table location + pub fn with_object_store(mut self, object_store: Arc) -> Self { + self.object_store = Some(object_store); + self } - fn statistics(&self) -> Statistics { - compute_record_batch_statistics(&[], &self.schema(), None) + /// 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 } } -#[allow(clippy::too_many_arguments)] -async fn do_write( - input: SendableRecordBatchStream, - table_uri: String, - partition_id: usize, - partition_columns: Option>, - schema: ArrowSchemaRef, - predicate: Option, - mode: SaveMode, - context: Arc, -) -> DataFusionResult { - let mut table = DeltaTableBuilder::from_uri(&table_uri) - .build() - .map_err(to_datafusion_err)?; - let metrics = ExecutionPlanMetricsSet::new(); - let tracking_metrics = MemTrackingMetrics::new(&metrics, partition_id); - - let mut actions = match table.load().await { - // Table is not yet created - Err(_) => { - let schema = Schema::try_from(schema.clone()).map_err(to_datafusion_err)?; - let metadata = DeltaTableMetaData::new( - None, - None, - None, - schema, - partition_columns.unwrap_or_default(), - HashMap::new(), - ); - let op = DeltaOperation::Create { - location: table_uri.clone(), - metadata: metadata.clone(), - mode: SaveMode::ErrorIfExists, - // TODO get the protocol from somewhere central - protocol: Protocol { - min_reader_version: 1, - min_writer_version: 1, - }, - }; - let plan = - Arc::new(CreateCommand::try_new(table_uri.clone(), op).map_err(to_datafusion_err)?); +impl std::future::IntoFuture for WriteBuilder { + type Output = DeltaResult; + type IntoFuture = BoxFuture<'static, Self::Output>; - let create_actions = collect(plan.clone(), context.clone()).await?; + fn into_future(self) -> Self::IntoFuture { + let this = self; - Ok(create_actions) - } - Ok(_) => { - if table.get_min_writer_version() > MAX_SUPPORTED_WRITER_VERSION { - Err(DeltaCommandError::UnsupportedWriterVersion( - table.get_min_writer_version(), - )) + Box::pin(async move { + let object_store = if let Some(store) = this.object_store { + Ok(store) } else { - match mode { - SaveMode::ErrorIfExists => { - Err(DeltaCommandError::TableAlreadyExists(table_uri.clone())) + DeltaTableBuilder::from_uri(&this.location.unwrap()) + .with_storage_options(this.storage_options.unwrap_or_default()) + .build_storage() + }?; + + // TODO we can find a more optimized config. Of course we want to pass in the state anyhow.. + let mut table = DeltaTable::new(object_store.clone(), Default::default()); + let mut actions = match table.load().await { + Err(DeltaTableError::NotATable(_)) => { + let schema: Schema = if let Some(plan) = &this.input { + Ok(plan.schema().try_into()?) + } else if let Some(batches) = &this.batches { + if batches.is_empty() { + return Err(WriteError::MissingData.into()); + } + Ok(batches[0].schema().try_into()?) + } else { + Err(WriteError::MissingData) + }?; + let mut builder = CreateBuilder::new() + .with_object_store(table.object_store()) + .with_columns(schema.get_fields().clone()); + if let Some(partition_columns) = this.partition_columns.as_ref() { + builder = builder.with_partition_columns(partition_columns.clone()) } - SaveMode::Ignore => { - return Ok(Box::pin(SizedRecordBatchStream::new( - schema, - vec![], - tracking_metrics, - ))) + let (_, actions, _) = builder.into_table_and_actions()?; + Ok(actions) + } + Ok(_) => { + if table.get_min_writer_version() > MAX_SUPPORTED_WRITER_VERSION { + Err( + WriteError::UnsupportedWriterVersion(table.get_min_writer_version()) + .into(), + ) + } else { + match this.mode { + SaveMode::ErrorIfExists => { + Err(WriteError::AlreadyExists(table.table_uri()).into()) + } + _ => Ok(vec![]), + } } - _ => Ok(vec![]), } + Err(err) => Err(err), + }?; + + let plan = if let Some(plan) = this.input { + Ok(plan) + } else if let Some(batches) = this.batches { + if batches.is_empty() { + Err(WriteError::MissingData) + } else { + let schema = batches[0].schema(); + + if let Ok(meta) = table.get_metadata() { + let curr_schema: ArrowSchemaRef = Arc::new((&meta.schema).try_into()?); + if schema != curr_schema { + return Err(DeltaTableError::Generic( + "Updating table schema not yet implemented".to_string(), + )); + } + if let Some(cols) = this.partition_columns.as_ref() { + if cols != &meta.partition_columns { + return Err(DeltaTableError::Generic( + "Updating table partitions not yet implemented".to_string(), + )); + } + }; + }; + + let data = if let Some(cols) = this.partition_columns.as_ref() { + // TODO partitioning should probably happen in its own plan ... + let mut partitions: HashMap> = HashMap::new(); + for batch in batches { + let divided = + divide_by_partition_values(schema.clone(), cols.clone(), &batch) + .unwrap(); + for part in divided { + let key = PartitionPath::from_hashmap(cols, &part.partition_values) + .map_err(DeltaTableError::from)? + .into(); + 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 { + vec![batches] + }; + + Ok(Arc::new(MemoryExec::try_new(&data, schema, None)?) + as Arc) + } + } else { + Err(WriteError::MissingData) + }?; + + // Write data to disk + let mut tasks = vec![]; + for i in 0..plan.output_partitioning().partition_count() { + let inner_plan = plan.clone(); + let state = SessionContext::new(); + let task_ctx = Arc::new(TaskContext::from(&state)); + let config = PartitionWriterConfig::try_new( + inner_plan.schema(), + HashMap::new(), + Vec::new(), + None, + this.target_file_size, + None, + )?; + let mut writer = PartitionWriter::try_with_config(object_store.clone(), config)?; + + let mut stream = inner_plan.execute(i, task_ctx)?; + let handle: tokio::task::JoinHandle>> = + tokio::task::spawn(async move { + while let Some(batch) = stream.next().await { + writer.write(&batch?).await?; + } + writer.close().await + }); + + tasks.push(handle); } - } - } - .map_err(to_datafusion_err)?; - - actions.append(&mut collect_batch(input).await?); - - if let SaveMode::Overwrite = mode { - // This should never error, since SystemTime::now() will always be larger than UNIX_EPOCH - let deletion_timestamp = SystemTime::now().duration_since(UNIX_EPOCH).unwrap(); - let deletion_timestamp = deletion_timestamp.as_millis() as i64; - - let to_remove_action = |add: &Add| { - Action::remove(Remove { - path: add.path.clone(), - deletion_timestamp: Some(deletion_timestamp), - data_change: true, - // TODO add file metadata to remove action (tags missing) - extended_file_metadata: Some(false), - partition_values: Some(add.partition_values.clone()), - size: Some(add.size), - tags: None, - }) - }; - - match predicate { - Some(_) => todo!("Overwriting data based on predicate is not yet implemented"), - _ => { - let remove_actions = table - .get_state() - .files() - .iter() - .map(to_remove_action) - .collect::>(); - actions.push(serialize_actions(remove_actions)?); - } - } - }; - - let stream = SizedRecordBatchStream::new( - schema, - actions - .iter() - .map(|b| Arc::new(b.to_owned())) - .collect::>(), - tracking_metrics, - ); - Ok(Box::pin(stream)) -} - -#[derive(Debug)] -/// Writes the partitioned input data into separate batches -/// and forwards the add actions as record batches -struct WritePartitionCommand { - table_uri: String, - /// Column names for table partitioning - partition_columns: Option>, - // TODO When using `Overwrite` mode, replace data that matches a predicate - // predicate: Option, - /// The input plan - input: Arc, -} - -impl WritePartitionCommand { - pub fn new( - table_uri: T, - partition_columns: Option>, - // predicate: Option, - input: Arc, - ) -> Self - where - T: Into, - { - Self { - table_uri: table_uri.into(), - partition_columns, - // predicate, - input, - } - } -} - -impl ExecutionPlan for WritePartitionCommand { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> ArrowSchemaRef { - Arc::new(OPERATION_SCHEMA.clone()) - } - - fn children(&self) -> Vec> { - vec![self.input.clone()] - } - - fn output_partitioning(&self) -> Partitioning { - self.input.output_partitioning() - } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - - fn required_child_distribution(&self) -> Distribution { - Distribution::UnspecifiedDistribution - } - - fn with_new_children( - self: Arc, - _children: Vec>, - ) -> DataFusionResult> { - todo!() - } + // Collect add actions to add to commit + let add_actions = futures::future::join_all(tasks) + .await + .into_iter() + .collect::, _>>() + .map_err(|err| WriteError::WriteTask { source: err })? + .into_iter() + .collect::, _>>()? + .concat() + .into_iter() + .map(Action::add) + .collect::>(); + actions.extend(add_actions); + + // Collect remove actions if we are overwriting the table + if matches!(this.mode, SaveMode::Overwrite) { + // This should never error, since now() will always be larger than UNIX_EPOCH + let deletion_timestamp = SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_millis() as i64; + + let to_remove_action = |add: &Add| { + Action::remove(Remove { + path: add.path.clone(), + deletion_timestamp: Some(deletion_timestamp), + data_change: true, + extended_file_metadata: Some(false), + partition_values: Some(add.partition_values.clone()), + size: Some(add.size), + // TODO add file metadata to remove action (tags missing) + tags: None, + }) + }; + + match this.predicate { + Some(_pred) => { + todo!("Overwriting data based on predicate is not yet implemented") + } + _ => { + let remove_actions = table + .get_state() + .files() + .iter() + .map(to_remove_action) + .collect::>(); + actions.extend(remove_actions); + } + } + }; - fn execute( - &self, - partition: usize, - context: Arc, - ) -> DataFusionResult { - let input = self.input.execute(partition, context)?; - - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - futures::stream::once( - do_write_partition( - input, - self.table_uri.clone(), - partition, - self.partition_columns.clone(), - ) - .map_err(|e| ArrowError::ExternalError(Box::new(e))), + // Finally, commit ... + let operation = DeltaOperation::Write { + mode: this.mode, + partition_by: this.partition_columns, + predicate: this.predicate, + }; + let _version = commit( + &table.storage, + table.version() + 1, + actions, + operation, + None, ) - .try_flatten(), - ))) - } - - fn statistics(&self) -> Statistics { - compute_record_batch_statistics(&[], &self.schema(), None) - } -} + .await?; + table.update().await?; -async fn do_write_partition( - input: SendableRecordBatchStream, - table_uri: String, - partition_id: usize, - partition_columns: Option>, -) -> DataFusionResult { - let schema = input.schema().clone(); - let data = collect_batch(input).await?; - if data.is_empty() { - let stream = EmptyRecordBatchStream::new(Arc::new(OPERATION_SCHEMA.clone())); - return Ok(Box::pin(stream)); - } - let mut writer = - RecordBatchWriter::try_new(table_uri.clone(), schema, partition_columns.clone(), None) - .map_err(to_datafusion_err)?; + // TODO should we build checkpoints based on config? - for batch in data { - // TODO we should have an API that allows us to circumvent internal partitioning - writer.write(batch).await.map_err(to_datafusion_err)?; + Ok(table) + }) } - let actions = writer - .flush() - .await - .map_err(to_datafusion_err)? - .iter() - .map(|e| Action::add(e.clone())) - .collect::>(); - - let serialized_actions = serialize_actions(actions)?; - let metrics = ExecutionPlanMetricsSet::new(); - let tracking_metrics = MemTrackingMetrics::new(&metrics, partition_id); - let stream = SizedRecordBatchStream::new( - serialized_actions.schema(), - vec![Arc::new(serialized_actions)], - tracking_metrics, - ); - - Ok(Box::pin(stream)) } #[cfg(test)] mod tests { use super::*; - use crate::{ - open_table, - writer::test_utils::{create_initialized_table, get_record_batch}, - }; + use crate::operations::DeltaOps; + use crate::writer::test_utils::{get_delta_schema, get_record_batch}; #[tokio::test] - async fn test_append_data() { - let partition_cols = vec!["modified".to_string()]; - let mut table = create_initialized_table(&partition_cols).await; - assert_eq!(table.version(), 0); - - let transaction = get_transaction(table.table_uri(), 0, SaveMode::Append); - let session_ctx = SessionContext::new(); - let task_ctx = session_ctx.task_ctx(); + async fn test_create_write() { + let table_schema = get_delta_schema(); + let batch = get_record_batch(None, false); - let _ = collect(transaction.clone(), task_ctx.clone()) + let table = DeltaOps::new_in_memory() + .create() + .with_columns(table_schema.get_fields().clone()) .await .unwrap(); - table.update().await.unwrap(); - assert_eq!(table.get_file_uris().count(), 2); - assert_eq!(table.version(), 1); - - let transaction = get_transaction(table.table_uri(), 1, SaveMode::Append); - let _ = collect(transaction.clone(), task_ctx).await.unwrap(); - table.update().await.unwrap(); - assert_eq!(table.get_file_uris().count(), 4); - assert_eq!(table.version(), 2); - } - - #[tokio::test] - async fn test_overwrite_data() { - let partition_cols = vec!["modified".to_string()]; - let mut table = create_initialized_table(&partition_cols).await; assert_eq!(table.version(), 0); - let transaction = get_transaction(table.table_uri(), 0, SaveMode::Overwrite); - let session_ctx = SessionContext::new(); - let task_ctx = session_ctx.task_ctx(); - - let _ = collect(transaction.clone(), task_ctx.clone()) + // write some data + let table = DeltaOps(table) + .write(vec![batch.clone()]) + .with_save_mode(SaveMode::Append) .await .unwrap(); - table.update().await.unwrap(); - assert_eq!(table.get_file_uris().count(), 2); assert_eq!(table.version(), 1); + assert_eq!(table.get_file_uris().count(), 1); - let transaction = get_transaction(table.table_uri(), 1, SaveMode::Overwrite); - let _ = collect(transaction.clone(), task_ctx).await.unwrap(); - table.update().await.unwrap(); - assert_eq!(table.get_file_uris().count(), 2); + // append some data + let table = DeltaOps(table) + .write(vec![batch.clone()]) + .with_save_mode(SaveMode::Append) + .await + .unwrap(); assert_eq!(table.version(), 2); + assert_eq!(table.get_file_uris().count(), 2); + + // overwrite table + let table = DeltaOps(table) + .write(vec![batch]) + .with_save_mode(SaveMode::Overwrite) + .await + .unwrap(); + assert_eq!(table.version(), 3); + assert_eq!(table.get_file_uris().count(), 1) } #[tokio::test] - async fn test_write_non_existent() { - let table_dir = tempfile::tempdir().unwrap(); - let table_path = table_dir.path(); - - let transaction = get_transaction( - table_path.to_str().unwrap().to_string(), - -1, - SaveMode::Overwrite, - ); - let session_ctx = SessionContext::new(); - let task_ctx = session_ctx.task_ctx(); - - let _ = collect(transaction.clone(), task_ctx.clone()) + async fn test_write_nonexistent() { + let batch = get_record_batch(None, false); + let table = DeltaOps::new_in_memory() + .write(vec![batch]) + .with_save_mode(SaveMode::ErrorIfExists) .await .unwrap(); - - // THe table should be created on write and thus have version 0 - let table = open_table(table_path.to_str().unwrap()).await.unwrap(); - assert_eq!(table.get_file_uris().count(), 2); assert_eq!(table.version(), 0); + assert_eq!(table.get_file_uris().count(), 1) } - fn get_transaction( - table_uri: String, - table_version: i64, - mode: SaveMode, - ) -> Arc { + #[tokio::test] + async fn test_write_partitioned() { let batch = get_record_batch(None, false); - let schema = batch.schema(); - let data_plan = Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()); - let op = DeltaOperation::Write { - partition_by: Some(vec!["modified".to_string()]), - mode, - predicate: None, - }; - let command = WriteCommand::try_new(&table_uri, op.clone(), data_plan).unwrap(); - - Arc::new(DeltaTransactionPlan::new( - table_uri, - table_version, - Arc::new(command), - op, - None, - )) + let table = DeltaOps::new_in_memory() + .write(vec![batch.clone()]) + .with_save_mode(SaveMode::ErrorIfExists) + .with_partition_columns(["modified"]) + .await + .unwrap(); + assert_eq!(table.version(), 0); + assert_eq!(table.get_file_uris().count(), 2); + + let table = DeltaOps::new_in_memory() + .write(vec![batch]) + .with_save_mode(SaveMode::ErrorIfExists) + .with_partition_columns(["modified", "id"]) + .await + .unwrap(); + assert_eq!(table.version(), 0); + assert_eq!(table.get_file_uris().count(), 4) } } diff --git a/rust/src/operations/writer.rs b/rust/src/operations/writer.rs new file mode 100644 index 0000000000..dfb0b6c4e0 --- /dev/null +++ b/rust/src/operations/writer.rs @@ -0,0 +1,323 @@ +use std::collections::HashMap; + +use crate::action::Add; +use crate::storage::ObjectStoreRef; +use crate::writer::stats::{apply_null_counts, create_add, NullCounts}; +use crate::writer::utils::{PartitionPath, ShareableBuffer}; +use crate::{crate_version, DeltaResult, DeltaTableError}; + +use arrow::datatypes::SchemaRef as ArrowSchemaRef; +use arrow::record_batch::RecordBatch; +use bytes::Bytes; +use log::warn; +use object_store::{path::Path, ObjectStore}; +use parquet::arrow::ArrowWriter; +use parquet::file::properties::WriterProperties; + +// TODO databricks often suggests a file size of 100mb, should we set this default? +const DEFAULT_TARGET_FILE_SIZE: usize = 104_857_600; +const DEFAULT_WRITE_BATCH_SIZE: usize = 1024; + +#[derive(thiserror::Error, Debug)] +enum WriteError { + #[error("Unexpected Arrow schema: got: {schema}, expected: {expected_schema}")] + SchemaMismatch { + schema: ArrowSchemaRef, + expected_schema: ArrowSchemaRef, + }, + + #[error("Error creating add action: {source}")] + CreateAdd { + source: Box, + }, + + #[error("Error creating file name from partition info: {source}")] + FileName { + source: Box, + }, +} + +impl From for DeltaTableError { + fn from(err: WriteError) -> Self { + match err { + WriteError::SchemaMismatch { .. } => DeltaTableError::SchemaMismatch { + msg: err.to_string(), + }, + _ => DeltaTableError::GenericError { + source: Box::new(err), + }, + } + } +} + +pub(crate) struct PartitionWriterConfig { + /// Schema of the data written to disk + file_schema: ArrowSchemaRef, + /// Prefix applied to all paths + prefix: Path, + /// Values for all partition columns + partition_values: HashMap>, + /// Properties passed to underlying parquet writer + writer_properties: WriterProperties, + /// Size above which we will write a buffered parquet file to disk. + target_file_size: usize, + /// Row chunks passed to parquet writer. This and the internal parquet writer settings + /// determine how fine granular we can track / control the size of resulting files. + write_batch_size: usize, +} + +impl PartitionWriterConfig { + pub fn try_new( + file_schema: ArrowSchemaRef, + partition_values: HashMap>, + partition_columns: Vec, + writer_properties: Option, + target_file_size: Option, + write_batch_size: Option, + ) -> DeltaResult { + let part_path = PartitionPath::from_hashmap(&partition_columns, &partition_values) + .map_err(|err| WriteError::FileName { + source: Box::new(err), + })?; + let prefix = Path::from(part_path.as_ref()); + let writer_properties = writer_properties.unwrap_or_else(|| { + WriterProperties::builder() + .set_created_by(format!("delta-rs version {}", crate_version())) + .build() + }); + let target_file_size = target_file_size.unwrap_or(DEFAULT_TARGET_FILE_SIZE); + let write_batch_size = write_batch_size.unwrap_or(DEFAULT_WRITE_BATCH_SIZE); + + Ok(Self { + file_schema, + prefix, + partition_values, + writer_properties, + target_file_size, + write_batch_size, + }) + } +} + +pub(crate) struct PartitionWriter { + object_store: ObjectStoreRef, + writer_id: uuid::Uuid, + config: PartitionWriterConfig, + buffer: ShareableBuffer, + arrow_writer: ArrowWriter, + part_counter: usize, + null_counts: NullCounts, + files_written: Vec, +} + +impl PartitionWriter { + /// Create a new instance of [`PartitionWriter`] from [`PartitionWriterConfig`] + pub fn try_with_config( + object_store: ObjectStoreRef, + config: PartitionWriterConfig, + ) -> DeltaResult { + let buffer = ShareableBuffer::default(); + let arrow_writer = ArrowWriter::try_new( + buffer.clone(), + config.file_schema.clone(), + Some(config.writer_properties.clone()), + )?; + + Ok(Self { + object_store, + writer_id: uuid::Uuid::new_v4(), + config, + buffer, + arrow_writer, + part_counter: 0, + null_counts: NullCounts::new(), + files_written: Vec::new(), + }) + } + + fn next_data_path(&mut self) -> Path { + let part = format!("{:0>5}", self.part_counter); + self.part_counter += 1; + // TODO: what does c000 mean? + // TODO handle file name for different compressions + let file_name = format!("part-{}-{}-c000.snappy.parquet", part, self.writer_id); + self.config.prefix.child(file_name) + } + + fn replace_arrow_buffer( + &mut self, + seed: impl AsRef<[u8]>, + ) -> DeltaResult<(ArrowWriter, ShareableBuffer)> { + let new_buffer = ShareableBuffer::from_bytes(seed.as_ref()); + let arrow_writer = ArrowWriter::try_new( + new_buffer.clone(), + self.config.file_schema.clone(), + Some(self.config.writer_properties.clone()), + )?; + Ok(( + std::mem::replace(&mut self.arrow_writer, arrow_writer), + std::mem::replace(&mut self.buffer, new_buffer), + )) + } + + fn write_batch(&mut self, batch: &RecordBatch) -> DeltaResult<()> { + // copy current cursor bytes so we can recover from failures + // TODO is copying this something we should be doing? + let buffer_bytes = self.buffer.to_vec(); + match self.arrow_writer.write(batch) { + Ok(_) => { + apply_null_counts(&batch.clone().into(), &mut self.null_counts, 0); + Ok(()) + } + Err(err) => { + // if a write fails we need to reset the state of the PartitionWriter + warn!("error writing to arrow buffer, resetting writer state."); + self.replace_arrow_buffer(buffer_bytes)?; + Err(err.into()) + } + } + } + + async fn flush_arrow_writer(&mut self) -> DeltaResult<()> { + // replace counter / buffers adn close the current writer + let (writer, buffer) = self.replace_arrow_buffer(vec![])?; + let null_counts = std::mem::take(&mut self.null_counts); + let metadata = writer.close()?; + + // collect metadata + let path = self.next_data_path(); + let obj_bytes = Bytes::from(buffer.to_vec()); + let file_size = obj_bytes.len() as i64; + + // write file to object store + self.object_store.put(&path, obj_bytes).await?; + self.files_written.push( + create_add( + &self.config.partition_values, + null_counts, + path.to_string(), + file_size, + &metadata, + ) + .map_err(|err| WriteError::CreateAdd { + source: Box::new(err), + })?, + ); + + Ok(()) + } + + /// Buffers record batches in-memory up to appx. `target_file_size`. + /// Flushes data to storage once a full file can be written. + /// + /// The `close` method has to be invoked to write all data still buffered + /// and get the list of all written files. + pub async fn write(&mut self, batch: &RecordBatch) -> DeltaResult<()> { + if batch.schema() != self.config.file_schema { + return Err(WriteError::SchemaMismatch { + schema: batch.schema(), + expected_schema: self.config.file_schema.clone(), + } + .into()); + } + + let max_offset = batch.num_rows(); + for offset in (0..max_offset).step_by(self.config.write_batch_size) { + let length = usize::min(self.config.write_batch_size, max_offset - offset); + self.write_batch(&batch.slice(offset, length))?; + // flush currently buffered data to disc once we meet ot exceed the tart file size. + if self.buffer.len() >= self.config.target_file_size { + log::debug!("Writing file with size {:?} to disk.", self.buffer.len()); + self.flush_arrow_writer().await?; + } + } + + Ok(()) + } + + pub async fn close(mut self) -> DeltaResult> { + self.flush_arrow_writer().await?; + Ok(self.files_written) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::storage::utils::flatten_list_stream as list; + use crate::writer::test_utils::get_record_batch; + use crate::DeltaTableBuilder; + use arrow::array::{Int32Array, StringArray}; + use arrow::datatypes::{DataType, Field, Schema as ArrowSchema}; + use std::sync::Arc; + + #[tokio::test] + async fn test_write_partition() { + let object_store = DeltaTableBuilder::from_uri("memory://") + .build_storage() + .unwrap(); + let batch = get_record_batch(None, false); + + // write single un-partitioned batch + let mut writer = get_writer(object_store.clone(), &batch, None, None); + writer.write(&batch).await.unwrap(); + let files = list(object_store.as_ref(), None).await.unwrap(); + assert_eq!(files.len(), 0); + let adds = writer.close().await.unwrap(); + let files = list(object_store.as_ref(), None).await.unwrap(); + assert_eq!(files.len(), 1); + assert_eq!(files.len(), adds.len()); + let head = object_store + .head(&Path::from(adds[0].path.clone())) + .await + .unwrap(); + assert_eq!(head.size, adds[0].size as usize) + } + + #[tokio::test] + async fn test_write_partition_with_parts() { + let base_int = Arc::new(Int32Array::from((0..10000).collect::>())); + let base_str = Arc::new(StringArray::from(vec!["A"; 10000])); + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Utf8, true), + Field::new("value", DataType::Int32, true), + ])); + let batch = RecordBatch::try_new(schema, vec![base_str, base_int]).unwrap(); + + let object_store = DeltaTableBuilder::from_uri("memory://") + .build_storage() + .unwrap(); + let properties = WriterProperties::builder() + .set_max_row_group_size(1024) + .build(); + // configure small target file size and and row group size so we can observe multiple files written + let mut writer = get_writer(object_store.clone(), &batch, Some(properties), Some(10_000)); + writer.write(&batch).await.unwrap(); + + // check that we have written more then once file, and no more then 1 is below target size + let adds = writer.close().await.unwrap(); + assert!(adds.len() > 1); + let target_file_count = adds + .iter() + .fold(0, |acc, add| acc + (add.size > 10_000) as i32); + assert!(target_file_count >= adds.len() as i32 - 1) + } + + fn get_writer( + object_store: ObjectStoreRef, + batch: &RecordBatch, + writer_properties: Option, + target_file_size: Option, + ) -> PartitionWriter { + let config = PartitionWriterConfig::try_new( + batch.schema(), + HashMap::new(), + Vec::new(), + writer_properties, + target_file_size, + None, + ) + .unwrap(); + PartitionWriter::try_with_config(object_store, config).unwrap() + } +} diff --git a/rust/src/table_properties.rs b/rust/src/table_properties.rs new file mode 100644 index 0000000000..1c69d423fd --- /dev/null +++ b/rust/src/table_properties.rs @@ -0,0 +1,69 @@ +//! properties defined on Delta Tables +//! + +/// true for this Delta table to be append-only. If append-only, +/// existing records cannot be deleted, and existing values cannot be updated. +pub const APPEND_ONLY: &str = "delta.appendOnly"; +/// true for Delta Lake to automatically optimize the layout of the files for this Delta table. +pub const AUTO_OPTIMIZE_AUTO_COMPACT: &str = "delta.autoOptimize.autoCompact"; +/// true for Delta Lake to automatically optimize the layout of the files for this Delta table during writes. +pub const AUTO_OPTIMIZE_OPTIMIZE_WRITE: &str = "delta.autoOptimize.optimizeWrite"; +/// true for Delta Lake to write file statistics in checkpoints in JSON format for the stats column. +pub const CHECKPOINT_WRITE_STATS_AS_JSON: &str = "delta.checkpoint.writeStatsAsJson"; +/// true for Delta Lake to write file statistics to checkpoints in struct format for the +/// stats_parsed column and to write partition values as a struct for partitionValues_parsed. +pub const CHECKPOINT_WRITE_STATS_AS_STRUCT: &str = "delta.checkpoint.writeStatsAsStruct"; +/// Whether column mapping is enabled for Delta table columns and the corresponding Parquet columns that use different names. +pub const COLUMN_MAPPING_MODE: &str = "delta.columnMapping.mode"; +/// Whether column mapping is enabled for Delta table columns and the corresponding Parquet columns that use different names. +pub const COMPATIBILITY_SYMLINK_FORMAT_MANIFEST_ENABLED: &str = + "delta.compatibility.symlinkFormatManifest.enabled"; +/// The number of columns for Delta Lake to collect statistics about for data skipping. +/// A value of -1 means to collect statistics for all columns. Updating this property does +/// not automatically collect statistics again; instead, it redefines the statistics schema +/// of the Delta table. Specifically, it changes the behavior of future statistics collection +/// (such as during appends and optimizations) as well as data skipping (such as ignoring column +/// statistics beyond this number, even when such statistics exist). +pub const DATA_SKIPPING_NUM_INDEXED_COLS: &str = "delta.dataSkippingNumIndexedCols"; +/// The shortest duration for Delta Lake to keep logically deleted data files before deleting +/// them physically. This is to prevent failures in stale readers after compactions or partition overwrites. +/// +/// This value should be large enough to ensure that: +/// +/// * It is larger than the longest possible duration of a job if you run VACUUM when there are +/// concurrent readers or writers accessing the Delta table. +/// * If you run a streaming query that reads from the table, that query does not stop for longer +/// than this value. Otherwise, the query may not be able to restart, as it must still read old files. +pub const DELETED_FILE_RETENTION_DURATION: &str = "delta.deletedFileRetentionDuration"; +/// true to enable change data feed. +pub const ENABLE_CHANGE_DATA_FEED: &str = "delta.enableChangeDataFeed"; +/// The degree to which a transaction must be isolated from modifications made by concurrent transactions. +/// +/// Valid values are `Serializable` and `WriteSerializable`. +pub const ISOLATION_LEVEL: &str = "delta.isolationLevel"; +/// How long the history for a Delta table is kept. +/// +/// Each time a checkpoint is written, Delta Lake automatically cleans up log entries older +/// than the retention interval. If you set this property to a large enough value, many log +/// entries are retained. This should not impact performance as operations against the log are +/// constant time. Operations on history are parallel but will become more expensive as the log size increases. +pub const LOG_RETENTION_DURATION: &str = "delta.logRetentionDuration"; +/// The minimum required protocol reader version for a reader that allows to read from this Delta table. +pub const MIN_READER_VERSION: &str = "delta.minReaderVersion"; +/// The minimum required protocol writer version for a writer that allows to write to this Delta table. +pub const MIN_WRITER_VERSION: &str = "delta.minWriterVersion"; +/// true for Delta Lake to generate a random prefix for a file path instead of partition information. +/// +/// For example, this may improve Amazon S3 performance when Delta Lake needs to send very high volumes +/// of Amazon S3 calls to better partition across S3 servers. +pub const RANDOMIZE_FILE_PREFIXES: &str = "delta.randomizeFilePrefixes"; +/// When delta.randomizeFilePrefixes is set to true, the number of characters that Delta Lake generates for random prefixes. +pub const RANDOM_PREFIX_LENGTH: &str = "delta.randomPrefixLength"; +/// The shortest duration within which new snapshots will retain transaction identifiers (for example, SetTransactions). +/// When a new snapshot sees a transaction identifier older than or equal to the duration specified by this property, +/// the snapshot considers it expired and ignores it. The SetTransaction identifier is used when making the writes idempotent. +pub const SET_TRANSACTION_RETENTION_DURATION: &str = "delta.setTransactionRetentionDuration"; +/// The target file size in bytes or higher units for file tuning. For example, 104857600 (bytes) or 100mb. +pub const TARGET_FILE_SIZE: &str = "delta.targetFileSize"; +/// The target file size in bytes or higher units for file tuning. For example, 104857600 (bytes) or 100mb. +pub const TUNE_FILE_SIZES_FOR_REWRITES: &str = "delta.tuneFileSizesForRewrites"; diff --git a/rust/src/writer/mod.rs b/rust/src/writer/mod.rs index 1392d1a553..d3e5fd2138 100644 --- a/rust/src/writer/mod.rs +++ b/rust/src/writer/mod.rs @@ -1,19 +1,8 @@ -//! Abstractions and implementations for writing data to delta tables -// TODO -// - consider file size when writing parquet files -// - handle writer version #![cfg(all(feature = "arrow", feature = "parquet"))] - -pub mod json; -pub mod record_batch; -mod stats; -#[cfg(test)] -pub mod test_utils; -pub mod utils; +//! Abstractions and implementations for writing data to delta tables use crate::action::{Action, Add, ColumnCountStat, Stats}; -use crate::delta::DeltaTable; -use crate::{DeltaDataTypeVersion, DeltaTableError}; +use crate::{DeltaDataTypeVersion, DeltaTable, DeltaTableError}; use arrow::{datatypes::SchemaRef, datatypes::*, error::ArrowError}; use async_trait::async_trait; @@ -24,6 +13,14 @@ use serde_json::Value; pub use json::JsonWriter; pub use record_batch::RecordBatchWriter; +pub mod json; +pub mod record_batch; +pub(crate) mod stats; +pub mod utils; + +#[cfg(test)] +pub mod test_utils; + /// Enum representing an error when calling [`DeltaWriter`]. #[derive(thiserror::Error, Debug)] pub(crate) enum DeltaWriterError { diff --git a/rust/src/writer/test_utils.rs b/rust/src/writer/test_utils.rs index d74533bb06..62684c3f05 100644 --- a/rust/src/writer/test_utils.rs +++ b/rust/src/writer/test_utils.rs @@ -13,6 +13,8 @@ use arrow::{ use std::collections::HashMap; use std::sync::Arc; +pub type TestResult = Result<(), Box>; + pub fn get_record_batch(part: Option, with_null: bool) -> RecordBatch { let (base_int, base_str, base_mod) = if with_null { data_with_null() diff --git a/rust/src/writer/utils.rs b/rust/src/writer/utils.rs index a2a0a513d9..5738ed9268 100644 --- a/rust/src/writer/utils.rs +++ b/rust/src/writer/utils.rs @@ -1,21 +1,22 @@ //! Handle JSON messages when writing to delta tables +use std::collections::HashMap; +use std::fmt::Display; +use std::io::Write; +use std::sync::Arc; + use crate::writer::DeltaWriterError; use crate::DeltaTableError; -use arrow::{ - array::{as_primitive_array, Array}, - datatypes::{ - DataType, Int16Type, Int32Type, Int64Type, Int8Type, Schema as ArrowSchema, - SchemaRef as ArrowSchemaRef, UInt16Type, UInt32Type, UInt64Type, UInt8Type, - }, - json::reader::{Decoder, DecoderOptions}, - record_batch::*, + +use arrow::array::{as_primitive_array, Array}; +use arrow::datatypes::{ + DataType, Int16Type, Int32Type, Int64Type, Int8Type, Schema as ArrowSchema, + SchemaRef as ArrowSchemaRef, UInt16Type, UInt32Type, UInt64Type, UInt8Type, }; +use arrow::json::reader::{Decoder, DecoderOptions}; +use arrow::record_batch::*; use object_store::path::Path; +use parking_lot::RwLock; use serde_json::Value; -use std::collections::HashMap; -use std::fmt::Display; -use std::io::Write; -use std::sync::{Arc, RwLock}; use uuid::Uuid; const NULL_PARTITION_VALUE_DATA_PATH: &str = "__HIVE_DEFAULT_PARTITION__"; @@ -197,25 +198,25 @@ impl ShareableBuffer { pub fn into_inner(self) -> Option> { Arc::try_unwrap(self.buffer) .ok() - .and_then(|lock| lock.into_inner().ok()) + .map(|lock| lock.into_inner()) } /// Returns a clone of the the underlying buffer as a `Vec`. pub fn to_vec(&self) -> Vec { - let inner = self.buffer.read().unwrap(); - inner.to_vec() + let inner = self.buffer.read(); + (*inner).to_vec() } /// Returns the number of bytes in the underlying buffer. pub fn len(&self) -> usize { - let inner = self.buffer.read().unwrap(); - inner.len() + let inner = self.buffer.read(); + (*inner).len() } /// Returns true if the underlying buffer is empty. pub fn is_empty(&self) -> bool { - let inner = self.buffer.read().unwrap(); - inner.is_empty() + let inner = self.buffer.read(); + (*inner).is_empty() } /// Creates a new instance with buffer initialized from the underylying bytes. @@ -228,12 +229,12 @@ impl ShareableBuffer { impl Write for ShareableBuffer { fn write(&mut self, buf: &[u8]) -> std::io::Result { - let mut inner = self.buffer.write().unwrap(); - inner.write(buf) + let mut inner = self.buffer.write(); + (*inner).write(buf) } fn flush(&mut self) -> std::io::Result<()> { - let mut inner = self.buffer.write().unwrap(); - inner.flush() + let mut inner = self.buffer.write(); + (*inner).flush() } } diff --git a/rust/tests/datafusion_test.rs b/rust/tests/datafusion_test.rs index ba61b61e6e..a4ab9f442b 100644 --- a/rust/tests/datafusion_test.rs +++ b/rust/tests/datafusion_test.rs @@ -1,11 +1,10 @@ #![cfg(feature = "datafusion-ext")] - use std::collections::{HashMap, HashSet}; +use std::future::IntoFuture; use std::sync::Arc; use deltalake::action::SaveMode; -use deltalake::operations::DeltaCommands; -use deltalake::{DeltaTable, DeltaTableMetaData, Schema}; +use deltalake::{operations::DeltaOps, DeltaTable, Schema}; use arrow::array::*; use arrow::datatypes::{DataType as ArrowDataType, Field as ArrowField, Schema as ArrowSchema}; @@ -57,26 +56,26 @@ async fn prepare_table( let table_dir = tempfile::tempdir().unwrap(); let table_path = table_dir.path(); let table_uri = table_path.to_str().unwrap().to_string(); + let table_schema: Schema = batches[0].schema().clone().try_into().unwrap(); - let mut commands = DeltaCommands::try_from_uri(table_uri.clone()) + let mut table = DeltaOps::try_from_uri(table_uri) .await - .unwrap(); - - let table_schema: Schema = batches[0].schema().try_into().unwrap(); - let metadata = DeltaTableMetaData::new(None, None, None, table_schema, vec![], HashMap::new()); - commands - .create(metadata.clone(), SaveMode::Ignore) + .unwrap() + .create() + .with_save_mode(SaveMode::Ignore) + .with_columns(table_schema.get_fields().clone()) .await .unwrap(); for batch in batches { - commands - .write(vec![batch], save_mode.clone(), None) + table = DeltaOps::from(table) + .write(vec![batch]) + .with_save_mode(save_mode.clone()) .await .unwrap(); } - (table_dir, Arc::new(commands.into())) + (table_dir, Arc::new(table)) } #[tokio::test] From 032d9a744eec3ed88878dae54c4aa2b3d309dad2 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Wed, 28 Sep 2022 00:37:05 +0200 Subject: [PATCH 02/19] chore: cleanup --- rust/src/lib.rs | 2 ++ rust/src/operations/create.rs | 3 ++- rust/src/operations/mod.rs | 14 ++++++++++++-- rust/src/operations/transaction.rs | 12 +++++------- rust/src/operations/write.rs | 6 +++++- rust/src/operations/writer.rs | 3 +-- rust/tests/datafusion_test.rs | 3 +-- 7 files changed, 28 insertions(+), 15 deletions(-) diff --git a/rust/src/lib.rs b/rust/src/lib.rs index ed65b0e25b..2d59daa10a 100644 --- a/rust/src/lib.rs +++ b/rust/src/lib.rs @@ -119,6 +119,8 @@ pub use object_store::{path::Path, Error as ObjectStoreError, ObjectMeta, Object pub use arrow; #[cfg(feature = "datafusion-ext")] pub use datafusion; +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub use operations::DeltaOps; #[cfg(feature = "parquet")] pub use parquet; #[cfg(feature = "parquet2")] diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 2a9d1580fd..56c2aaa9be 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -269,7 +269,8 @@ impl std::future::IntoFuture for CreateBuilder { } } } - let version = commit(&table.object_store(), 0, actions, operation, None).await?; + let version = + commit(table.object_store().as_ref(), 0, actions, operation, None).await?; table.load_version(version).await?; Ok(table) diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index 05ae56bbf6..6c990e5d8d 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -52,6 +52,16 @@ impl DeltaOps { } /// Create a new Delta table + /// + /// ``` + /// use deltalake::DeltaOps; + /// + /// async { + /// let ops = DeltaOps::try_from_uri("memory://").await.unwrap(); + /// let table = ops.create().with_table_name("my_table").await.unwrap(); + /// assert_eq!(table.version(), 0); + /// }; + /// ``` pub fn create(self) -> CreateBuilder { CreateBuilder::default().with_object_store(self.0.object_store()) } @@ -78,8 +88,8 @@ impl From for DeltaOps { } impl From for DeltaTable { - fn from(comm: DeltaOps) -> Self { - comm.0 + fn from(ops: DeltaOps) -> Self { + ops.0 } } diff --git a/rust/src/operations/transaction.rs b/rust/src/operations/transaction.rs index 27d3c283e7..7f513f7ed5 100644 --- a/rust/src/operations/transaction.rs +++ b/rust/src/operations/transaction.rs @@ -1,6 +1,4 @@ //! Delta transactions -use std::sync::Arc; - use crate::action::{Action, DeltaOperation}; use crate::storage::DeltaObjectStore; use crate::{crate_version, DeltaDataTypeVersion, DeltaResult, DeltaTableError}; @@ -66,7 +64,7 @@ fn log_entry_from_actions(actions: &[Action]) -> Result, + storage: &DeltaObjectStore, operation: DeltaOperation, mut actions: Vec, app_metadata: Option>, @@ -107,7 +105,7 @@ async fn prepare_commit( /// the `DeltaTransaction.commit` is desired to be used as it handles `try_commit_transaction` /// with retry logic. async fn try_commit_transaction( - storage: &Arc, + storage: &DeltaObjectStore, tmp_commit: &Path, version: DeltaDataTypeVersion, ) -> Result { @@ -126,7 +124,7 @@ async fn try_commit_transaction( } pub(crate) async fn commit( - storage: &Arc, + storage: &DeltaObjectStore, version: DeltaDataTypeVersion, actions: Vec, operation: DeltaOperation, @@ -178,7 +176,7 @@ mod tests { .unwrap(); // successfully write in clean location - commit(&storage, 0, vec![], operation.clone(), None) + commit(storage.as_ref(), 0, vec![], operation.clone(), None) .await .unwrap(); let head = storage.head(&commit_path).await; @@ -186,7 +184,7 @@ mod tests { assert_eq!(head.as_ref().unwrap().location, commit_path); // fail on overwriting - let failed_commit = commit(&storage, 0, vec![], operation, None).await; + let failed_commit = commit(storage.as_ref(), 0, vec![], operation, None).await; assert!(failed_commit.is_err()); assert!(matches!( failed_commit.unwrap_err(), diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index 956adfb1b3..61bb8443eb 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -64,7 +64,7 @@ impl From for DeltaTableError { } } -/// Builder for write operations +/// Write data into a DeltaTable #[derive(Debug, Clone)] pub struct WriteBuilder { /// The input plan @@ -73,13 +73,17 @@ pub struct WriteBuilder { location: Option, /// SaveMode defines how to treat data already written to table location mode: SaveMode, + /// Column names for table partitioning partition_columns: Option>, /// When using `Overwrite` mode, replace data that matches a predicate predicate: Option, /// Size above which we will write a buffered parquet file to disk. target_file_size: Option, + /// An object store to be used as backend for delta table object_store: Option>, + /// Storage options used to create a new storage backend storage_options: Option>, + /// RecordBatches to be written into the table batches: Option>, } diff --git a/rust/src/operations/writer.rs b/rust/src/operations/writer.rs index dfb0b6c4e0..520c54eb24 100644 --- a/rust/src/operations/writer.rs +++ b/rust/src/operations/writer.rs @@ -138,7 +138,6 @@ impl PartitionWriter { fn next_data_path(&mut self) -> Path { let part = format!("{:0>5}", self.part_counter); self.part_counter += 1; - // TODO: what does c000 mean? // TODO handle file name for different compressions let file_name = format!("part-{}-{}-c000.snappy.parquet", part, self.writer_id); self.config.prefix.child(file_name) @@ -179,7 +178,7 @@ impl PartitionWriter { } async fn flush_arrow_writer(&mut self) -> DeltaResult<()> { - // replace counter / buffers adn close the current writer + // replace counter / buffers and close the current writer let (writer, buffer) = self.replace_arrow_buffer(vec![])?; let null_counts = std::mem::take(&mut self.null_counts); let metadata = writer.close()?; diff --git a/rust/tests/datafusion_test.rs b/rust/tests/datafusion_test.rs index a4ab9f442b..ee1e60d406 100644 --- a/rust/tests/datafusion_test.rs +++ b/rust/tests/datafusion_test.rs @@ -1,6 +1,5 @@ #![cfg(feature = "datafusion-ext")] -use std::collections::{HashMap, HashSet}; -use std::future::IntoFuture; +use std::collections::HashSet; use std::sync::Arc; use deltalake::action::SaveMode; From f9a9d49eb94d56e824627d9c58c9036376865eaa Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Wed, 28 Sep 2022 08:38:26 +0200 Subject: [PATCH 03/19] docs: add basic docstrings to some DeltaOps methods --- rust/src/operations/mod.rs | 24 +++++++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index 6c990e5d8d..57a5e4153b 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -32,7 +32,16 @@ pub const MAX_SUPPORTED_READER_VERSION: i32 = 1; pub struct DeltaOps(DeltaTable); impl DeltaOps { - /// load table from uri + /// Create a new [`DeltaOps`] instance, operating on [`DeltaTable`] at given uri. + /// + /// ``` + /// use deltalake::DeltaOps; + /// + /// async { + /// let ops = DeltaOps::try_from_uri("memory://").await.unwrap(); + /// }; + /// ``` + #[must_use] pub async fn try_from_uri(uri: impl AsRef) -> DeltaResult { let mut table = DeltaTableBuilder::from_uri(uri).build()?; // We allow for uninitialized locations, since we may want to create the table @@ -44,6 +53,16 @@ impl DeltaOps { } /// Create a new [`DeltaOps`] instance, backed by an un-initialized in memory table + /// + /// Using this will not persist any changes beyond the lifetime of the table object. + /// THe main purpose of in-memory tables is for use in testing. + /// + /// ``` + /// use deltalake::DeltaOps; + /// + /// let ops = DeltaOps::new_in_memory(); + /// ``` + #[must_use] pub fn new_in_memory() -> Self { DeltaTableBuilder::from_uri("memory://") .build() @@ -62,18 +81,21 @@ impl DeltaOps { /// assert_eq!(table.version(), 0); /// }; /// ``` + #[must_use] pub fn create(self) -> CreateBuilder { CreateBuilder::default().with_object_store(self.0.object_store()) } /// Write data to Delta table #[cfg(feature = "datafusion-ext")] + #[must_use] pub fn load(self) -> LoadBuilder { LoadBuilder::default().with_object_store(self.0.object_store()) } /// Write data to Delta table #[cfg(feature = "datafusion-ext")] + #[must_use] pub fn write(self, batches: Vec) -> WriteBuilder { WriteBuilder::default() .with_input_batches(batches) From 71606976d05c021949781307f63d615d7587419d Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Wed, 28 Sep 2022 18:44:00 +0200 Subject: [PATCH 04/19] try fixing windows tests --- rust/src/operations/create.rs | 8 +++++--- rust/src/operations/mod.rs | 1 - 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 56c2aaa9be..0ba8372fda 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -338,9 +338,11 @@ mod tests { assert_eq!(table.version(), 0); let first_id = table.get_metadata().unwrap().id.clone(); + let object_store = table.object_store(); + // Check an error is raised when a table exists at location let table = CreateBuilder::new() - .with_location(tmp_dir.path().to_str().unwrap()) + .with_object_store(object_store.clone()) .with_columns(schema.get_fields().clone()) .with_save_mode(SaveMode::ErrorIfExists) .await; @@ -348,7 +350,7 @@ mod tests { // Check current table is returned when ignore option is chosen. let table = CreateBuilder::new() - .with_location(tmp_dir.path().to_str().unwrap()) + .with_object_store(object_store.clone()) .with_columns(schema.get_fields().clone()) .with_save_mode(SaveMode::Ignore) .await @@ -357,7 +359,7 @@ mod tests { // Check table is overwritten let table = CreateBuilder::new() - .with_location(tmp_dir.path().to_str().unwrap()) + .with_object_store(object_store.clone()) .with_columns(schema.get_fields().clone()) .with_save_mode(SaveMode::Overwrite) .await diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index 57a5e4153b..7ac482d636 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -41,7 +41,6 @@ impl DeltaOps { /// let ops = DeltaOps::try_from_uri("memory://").await.unwrap(); /// }; /// ``` - #[must_use] pub async fn try_from_uri(uri: impl AsRef) -> DeltaResult { let mut table = DeltaTableBuilder::from_uri(uri).build()?; // We allow for uninitialized locations, since we may want to create the table From fa775b69b24da523ed08c823dcc050862d8aee0a Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Wed, 12 Oct 2022 10:52:30 +0200 Subject: [PATCH 05/19] feat: writer updates --- Cargo.lock | 130 ++++++++++++++--------- rust/Cargo.toml | 13 ++- rust/src/checkpoints.rs | 11 +- rust/src/delta.rs | 5 +- rust/src/operations/load.rs | 12 ++- rust/src/operations/mod.rs | 2 +- rust/src/operations/write.rs | 86 ++++++++++----- rust/src/operations/writer.rs | 194 +++++++++++++++++++++++++++++++++- rust/src/time_utils.rs | 6 +- rust/src/writer/stats.rs | 10 +- rust/tests/datafusion_test.rs | 2 +- 11 files changed, 364 insertions(+), 107 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 14724031a2..317a1facf2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -91,11 +91,15 @@ checksum = "8da52d66c7071e2e3fa2a1e5c6d088fec47b593032b254f5e980de8ea54454d6" [[package]] name = "arrow" -version = "22.0.0" +version = "24.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5936b4185aa57cb9790d8742aab22859045ce5cc6a3023796240cd101c19335" +checksum = "d68391300d5237f6725f0f869ae7cb65d45fcf8a6d18f6ceecd328fb803bef93" dependencies = [ "ahash 0.8.0", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", "bitflags", "chrono", "comfy-table", @@ -111,10 +115,52 @@ dependencies = [ "pyo3", "regex", "regex-syntax", - "serde", "serde_json", ] +[[package]] +name = "arrow-array" +version = "24.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0bb00c5862b5eea683812083c495bef01a9a5149da46ad2f4c0e4aa8800f64d" +dependencies = [ + "ahash 0.8.0", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "half 2.1.0", + "hashbrown", + "num", +] + +[[package]] +name = "arrow-buffer" +version = "24.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3e594d0fe0026a8bc2459bdc5ac9623e5fb666724a715e0acbc96ba30c5d4cc7" +dependencies = [ + "half 2.1.0", +] + +[[package]] +name = "arrow-data" +version = "24.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8500df05060d86fdc53e9b5cb32e51bfeaacc040fdeced3eb99ac0d59200ff45" +dependencies = [ + "arrow-buffer", + "arrow-schema", + "half 2.1.0", + "num", +] + +[[package]] +name = "arrow-schema" +version = "24.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "86d1fef01f25e1452c86fa6887f078de8e0aaeeb828370feab205944cfc30e27" + [[package]] name = "async-stream" version = "0.3.3" @@ -579,9 +625,9 @@ dependencies = [ [[package]] name = "datafusion" -version = "12.0.0" +version = "13.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2aca80caa2b0f7fdf267799b8895ac8b6341ea879db6b1e2d361ec49b47bc676" +checksum = "a2bdec06a3db088da76fc28cb0877b8b5438ca6b6025e04d975bace0fd85df19" dependencies = [ "ahash 0.8.0", "arrow", @@ -619,23 +665,22 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "12.0.0" +version = "13.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7721fd550f6a28ad7235b62462aa51e9a43b08f8346d5cbe4d61f1e83f5df511" +checksum = "506eab038bf2d39ac02c22be30b019873ca01f887148b939d309a0e9523f4515" dependencies = [ "arrow", "object_store", "ordered-float 3.0.0", "parquet", - "serde_json", "sqlparser", ] [[package]] name = "datafusion-expr" -version = "12.0.0" +version = "13.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d81255d043dc594c0ded6240e8a9be6ce8d7c22777a5093357cdb97af3d29ce" +checksum = "b3d2810e369c735d69479e27fe8410e97a76ed07484aa9b3ad7c039efa504257" dependencies = [ "ahash 0.8.0", "arrow", @@ -645,9 +690,9 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "12.0.0" +version = "13.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71b39f8c75163691fff72b4a71816ad5a912e7c6963ee55f29ed1910b5a6993f" +checksum = "60f3b80326243629d02e33f37e955a7114781c6c44caf9d8b254618157de7143" dependencies = [ "arrow", "async-trait", @@ -661,9 +706,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "12.0.0" +version = "13.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "109c4138220a109feafb63bf05418b86b17a42ece4bf047c38e4fd417572a9f7" +checksum = "e9bf3b7ae861d351a85174fd4fddb29d249950b2f23671318971960452b4b9ab" dependencies = [ "ahash 0.8.0", "arrow", @@ -686,9 +731,9 @@ dependencies = [ [[package]] name = "datafusion-row" -version = "12.0.0" +version = "13.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87a178fc0fd7693d9c9f608f7b605823eb982c6731ede0cccd99e2319cacabbc" +checksum = "3f44a2a722719c569b437b3aa2108a99dc911369e8d86c44e6293225c3387147" dependencies = [ "arrow", "datafusion-common", @@ -698,17 +743,14 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "12.0.0" +version = "13.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "148cb56e7635faff3b16019393c49b988188c3fdadd1ca90eadb322a80aa1128" +checksum = "e98493e04385c924d1d3d7ab8739c41f1ebf676a46863181103a0fb9c7168fa9" dependencies = [ - "ahash 0.8.0", "arrow", "datafusion-common", "datafusion-expr", - "hashbrown", "sqlparser", - "tokio", ] [[package]] @@ -740,7 +782,6 @@ dependencies = [ "once_cell", "parking_lot", "parquet", - "parquet-format", "parquet2", "percent-encoding", "pretty_assertions", @@ -1166,6 +1207,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ad6a9459c9c30b177b925162351f97e7d967c7ea8bab3b8352805327daf45554" dependencies = [ "crunchy", + "num-traits", ] [[package]] @@ -1359,9 +1401,9 @@ dependencies = [ [[package]] name = "integer-encoding" -version = "1.1.7" +version = "3.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48dc51180a9b377fd75814d0cc02199c20f8e99433d6762f650d39cdbbd3b56f" +checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" [[package]] name = "ipnet" @@ -1538,6 +1580,12 @@ version = "0.2.133" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c0f80d65747a3e43d1596c7c5492d95d5edddaabd45a7fcdb02b95f644164966" +[[package]] +name = "libm" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "292a948cd991e376cf75541fe5b97a1081d713c618b4f1b9500f8844e49eb565" + [[package]] name = "lock_api" version = "0.4.8" @@ -1757,6 +1805,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "578ede34cf02f8924ab9447f50c28075b4d3e5b269972345e7e0372b38c6cdcd" dependencies = [ "autocfg", + "libm", ] [[package]] @@ -1928,9 +1977,9 @@ dependencies = [ [[package]] name = "parquet" -version = "22.0.0" +version = "24.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "474c423be6f10921adab3b94b42ec7fe87c1b87e1360dee150976caee444224f" +checksum = "74fd590f0672998df84503d1bcbebc69732583d03cc3495c7dd8d3e5a1d8437f" dependencies = [ "ahash 0.8.0", "arrow", @@ -1944,7 +1993,6 @@ dependencies = [ "lz4", "num", "num-bigint", - "parquet-format", "rand 0.8.5", "seq-macro", "snap", @@ -1953,15 +2001,6 @@ dependencies = [ "zstd", ] -[[package]] -name = "parquet-format" -version = "4.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f0c06cdcd5460967c485f9c40a821746f5955ad81990533c7fae95dbd9bc0b5" -dependencies = [ - "thrift", -] - [[package]] name = "parquet-format-safe" version = "0.2.4" @@ -2845,9 +2884,9 @@ checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" [[package]] name = "sqlparser" -version = "0.23.0" +version = "0.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0beb13adabbdda01b63d595f38c8bfd19a361e697fd94ce0098a634077bc5b25" +checksum = "0781f2b6bd03e5adf065c8e772b49eaea9f640d06a1b9130330fe8bd2563f4fd" dependencies = [ "log", ] @@ -2968,26 +3007,15 @@ dependencies = [ "syn", ] -[[package]] -name = "threadpool" -version = "1.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa" -dependencies = [ - "num_cpus", -] - [[package]] name = "thrift" -version = "0.13.0" +version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c6d965454947cc7266d22716ebfd07b18d84ebaf35eec558586bbb2a8cb6b5b" +checksum = "09678c4cdbb4eed72e18b7c2af1329c69825ed16fcbac62d083fc3e2b0590ff0" dependencies = [ "byteorder", "integer-encoding", - "log", "ordered-float 1.1.1", - "threadpool", ] [[package]] diff --git a/rust/Cargo.toml b/rust/Cargo.toml index d8767b521e..9c3b513159 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -10,7 +10,7 @@ description = "Native Delta Lake implementation in Rust" edition = "2021" [dependencies] -arrow = { version = "22", optional = true } +arrow = { version = "24", optional = true } async-trait = "0.1" bytes = "1" chrono = "0.4.22" @@ -25,9 +25,8 @@ num-traits = "0.2.15" object_store = "0.5.0" once_cell = "1.15.0" parking_lot = "0.12" -parquet = { version = "22", features = ["async"], optional = true } +parquet = { version = "24", features = ["async"], optional = true } parquet2 = { version = "0.16", optional = true } -parquet-format = { version = "~4.0.0" } percent-encoding = "2" serde = { version = "1", features = ["derive"] } serde_json = "1" @@ -47,9 +46,9 @@ rusoto_dynamodb = { version = "0.48", default-features = false, optional = true rusoto_glue = { version = "0.48", default-features = false, optional = true } # Datafusion -datafusion = { version = "12", optional = true } -datafusion-expr = { version = "12", optional = true } -datafusion-common = { version = "12", optional = true } +datafusion = { version = "13", optional = true } +datafusion-expr = { version = "13", optional = true } +datafusion-common = { version = "13", optional = true } # NOTE dependencies only for integration tests fs_extra = { version = "1.2.0", optional = true } @@ -76,7 +75,7 @@ utime = "0.3" glibc_version = "0" [features] -default = ["arrow", "parquet"] +default = ["arrow", "parquet", "datafusion-ext"] datafusion-ext = [ "datafusion", "datafusion-expr", diff --git a/rust/src/checkpoints.rs b/rust/src/checkpoints.rs index b13428b75f..be41867dfd 100644 --- a/rust/src/checkpoints.rs +++ b/rust/src/checkpoints.rs @@ -365,7 +365,7 @@ fn parquet_bytes_from_state(state: &DeltaTableState) -> Result Result { - let mut v = serde_json::to_value(action::Action::add(add.clone()))?; + let mut v = serde_json::to_value(action::Action::add(add.clone())) + .map_err(|err| ArrowError::JsonError(err.to_string()))?; v["add"]["dataChange"] = Value::Bool(false); @@ -413,12 +414,14 @@ fn checkpoint_add_from_state( } } - let partition_values_parsed = serde_json::to_value(partition_values_parsed)?; + let partition_values_parsed = serde_json::to_value(partition_values_parsed) + .map_err(|err| ArrowError::JsonError(err.to_string()))?; v["add"]["partitionValues_parsed"] = partition_values_parsed; } if let Ok(Some(stats)) = add.get_stats() { - let mut stats = serde_json::to_value(stats)?; + let mut stats = + serde_json::to_value(stats).map_err(|err| ArrowError::JsonError(err.to_string()))?; let min_values = stats.get_mut("minValues").and_then(|v| v.as_object_mut()); if let Some(min_values) = min_values { diff --git a/rust/src/delta.rs b/rust/src/delta.rs index d3839aa161..c4445f21d1 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -14,7 +14,6 @@ use super::partitions::{DeltaTablePartition, PartitionFilter}; use super::schema::*; use super::table_state::DeltaTableState; use crate::action::{Add, Stats}; -use crate::builder::{DeltaTableBuilder, DeltaTableConfig}; use crate::delta_config::DeltaConfigError; use crate::storage::ObjectStoreRef; use crate::vacuum::{Vacuum, VacuumError}; @@ -29,6 +28,9 @@ use serde::{Deserialize, Serialize}; use serde_json::{Map, Value}; use uuid::Uuid; +// TODO re-exports only for transition +pub use crate::builder::{DeltaTableBuilder, DeltaTableConfig, DeltaVersion}; + /// Metadata for a checkpoint file #[derive(Serialize, Deserialize, Debug, Default, Clone, Copy)] pub struct CheckPoint { @@ -93,7 +95,6 @@ pub enum DeltaTableError { #[cfg(feature = "parquet")] #[from] source: parquet::errors::ParquetError, - /// Parquet error details returned when reading the checkpoint failed. #[cfg(feature = "parquet2")] #[from] source: parquet2::error::Error, diff --git a/rust/src/operations/load.rs b/rust/src/operations/load.rs index 80487f4ec5..39a4780a7f 100644 --- a/rust/src/operations/load.rs +++ b/rust/src/operations/load.rs @@ -66,6 +66,8 @@ impl LoadBuilder { /// /// [crate::builder::s3_storage_options] describes the available options for the AWS or S3-compliant backend. /// [dynamodb_lock::DynamoDbLockClient] describes additional options for the AWS atomic rename client. + /// [crate::builder::azure_storage_options] describes the available options for the Azure backend. + /// [crate::builder::gcp_storage_options] describes the available options for the Google Cloud Platform backend. pub fn with_storage_options(mut self, storage_options: HashMap) -> Self { self.storage_options = Some(storage_options); self @@ -83,8 +85,10 @@ impl std::future::IntoFuture for LoadBuilder { type IntoFuture = BoxFuture<'static, Self::Output>; fn into_future(self) -> Self::IntoFuture { + let this = self; + Box::pin(async move { - let object_store = self.object_store.unwrap(); + let object_store = this.object_store.unwrap(); let mut table = DeltaTable::new(object_store, Default::default()); table.load().await?; @@ -102,7 +106,6 @@ impl std::future::IntoFuture for LoadBuilder { mod tests { use crate::operations::{collect_sendable_stream, DeltaOps}; use crate::writer::test_utils::{get_record_batch, TestResult}; - use arrow::record_batch::RecordBatch; use datafusion::assert_batches_sorted_eq; #[tokio::test] @@ -112,7 +115,6 @@ mod tests { let (_table, stream) = DeltaOps(table).load().await?; let data = collect_sendable_stream(stream).await?; - let batch_loaded = RecordBatch::concat(&data[0].schema(), &data)?; let expected = vec![ "+----+-------+------------+", @@ -132,8 +134,8 @@ mod tests { "+----+-------+------------+", ]; - assert_batches_sorted_eq!(&expected, &[batch_loaded.clone()]); - assert_eq!(batch.schema(), batch_loaded.schema()); + assert_batches_sorted_eq!(&expected, &data); + assert_eq!(batch.schema(), data[0].schema()); Ok(()) } } diff --git a/rust/src/operations/mod.rs b/rust/src/operations/mod.rs index 7ac482d636..ffc0bafad7 100644 --- a/rust/src/operations/mod.rs +++ b/rust/src/operations/mod.rs @@ -29,7 +29,7 @@ pub const MAX_SUPPORTED_WRITER_VERSION: i32 = 1; pub const MAX_SUPPORTED_READER_VERSION: i32 = 1; /// High level interface for executing commands against a DeltaTable -pub struct DeltaOps(DeltaTable); +pub struct DeltaOps(pub DeltaTable); impl DeltaOps { /// Create a new [`DeltaOps`] instance, operating on [`DeltaTable`] at given uri. diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index 61bb8443eb..cadf1f2eba 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -23,7 +23,7 @@ use std::collections::HashMap; use std::sync::Arc; use std::time::{SystemTime, UNIX_EPOCH}; -use super::writer::{PartitionWriter, PartitionWriterConfig}; +use super::writer::{DeltaWriter, WriterConfig}; use super::MAX_SUPPORTED_WRITER_VERSION; use super::{transaction::commit, CreateBuilder}; use crate::action::{Action, Add, DeltaOperation, Remove, SaveMode}; @@ -54,6 +54,16 @@ enum WriteError { #[error("A table already exists at: {0}")] AlreadyExists(String), + + #[error( + "Specified table partitioning does not match table partitioning: expected: {:?}, got: {:?}", + expected, + got + )] + PartitionColumnMismatch { + expected: Vec, + got: Vec, + }, } impl From for DeltaTableError { @@ -79,6 +89,8 @@ pub struct WriteBuilder { predicate: Option, /// Size above which we will write a buffered parquet file to disk. target_file_size: Option, + /// Number of records to be written in single batch to underlying writer + write_batch_size: Option, /// An object store to be used as backend for delta table object_store: Option>, /// Storage options used to create a new storage backend @@ -104,6 +116,7 @@ impl WriteBuilder { predicate: None, storage_options: None, target_file_size: None, + write_batch_size: None, object_store: None, batches: None, } @@ -153,9 +166,6 @@ impl WriteBuilder { /// Set options used to initialize storage backend /// /// Options may be passed in the HashMap or set as environment variables. - /// - /// [crate::builder::s3_storage_options] describes the available options for the AWS or S3-compliant backend. - /// [dynamodb_lock::DynamoDbLockClient] describes additional options for the AWS atomic rename client. pub fn with_storage_options(mut self, storage_options: HashMap) -> Self { self.storage_options = Some(storage_options); self @@ -172,6 +182,12 @@ impl WriteBuilder { self.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 + } } impl std::future::IntoFuture for WriteBuilder { @@ -231,6 +247,24 @@ impl std::future::IntoFuture for WriteBuilder { Err(err) => Err(err), }?; + // validate partition columns + let partition_columns = if let Ok(meta) = table.get_metadata() { + if let Some(ref partition_columns) = this.partition_columns { + if &meta.partition_columns != partition_columns { + Err(WriteError::PartitionColumnMismatch { + expected: table.get_metadata()?.partition_columns.clone(), + got: partition_columns.to_vec(), + }) + } else { + Ok(partition_columns.clone()) + } + } else { + Ok(meta.partition_columns.clone()) + } + } else { + Ok(this.partition_columns.unwrap_or_default()) + }?; + let plan = if let Some(plan) = this.input { Ok(plan) } else if let Some(batches) = this.batches { @@ -246,26 +280,25 @@ impl std::future::IntoFuture for WriteBuilder { "Updating table schema not yet implemented".to_string(), )); } - if let Some(cols) = this.partition_columns.as_ref() { - if cols != &meta.partition_columns { - return Err(DeltaTableError::Generic( - "Updating table partitions not yet implemented".to_string(), - )); - } - }; }; - let data = if let Some(cols) = this.partition_columns.as_ref() { + 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 divided = - divide_by_partition_values(schema.clone(), cols.clone(), &batch) - .unwrap(); + let divided = divide_by_partition_values( + schema.clone(), + partition_columns.clone(), + &batch, + ) + .unwrap(); for part in divided { - let key = PartitionPath::from_hashmap(cols, &part.partition_values) - .map_err(DeltaTableError::from)? - .into(); + let key = PartitionPath::from_hashmap( + &partition_columns, + &part.partition_values, + ) + .map_err(DeltaTableError::from)? + .into(); match partitions.get_mut(&key) { Some(part_batches) => { part_batches.push(part.record_batch); @@ -294,15 +327,14 @@ impl std::future::IntoFuture for WriteBuilder { let inner_plan = plan.clone(); let state = SessionContext::new(); let task_ctx = Arc::new(TaskContext::from(&state)); - let config = PartitionWriterConfig::try_new( + let config = WriterConfig::new( inner_plan.schema(), - HashMap::new(), - Vec::new(), + partition_columns.clone(), None, this.target_file_size, - None, - )?; - let mut writer = PartitionWriter::try_with_config(object_store.clone(), config)?; + this.write_batch_size, + ); + let mut writer = DeltaWriter::new(object_store.clone(), config); let mut stream = inner_plan.execute(i, task_ctx)?; let handle: tokio::task::JoinHandle>> = @@ -370,7 +402,11 @@ impl std::future::IntoFuture for WriteBuilder { // Finally, commit ... let operation = DeltaOperation::Write { mode: this.mode, - partition_by: this.partition_columns, + partition_by: if !partition_columns.is_empty() { + Some(partition_columns) + } else { + None + }, predicate: this.predicate, }; let _version = commit( diff --git a/rust/src/operations/writer.rs b/rust/src/operations/writer.rs index 520c54eb24..cab5a5d8f3 100644 --- a/rust/src/operations/writer.rs +++ b/rust/src/operations/writer.rs @@ -2,16 +2,22 @@ use std::collections::HashMap; use crate::action::Add; use crate::storage::ObjectStoreRef; +use crate::writer::record_batch::{divide_by_partition_values, PartitionResult}; use crate::writer::stats::{apply_null_counts, create_add, NullCounts}; -use crate::writer::utils::{PartitionPath, ShareableBuffer}; +use crate::writer::utils::{ + arrow_schema_without_partitions, record_batch_without_partitions, PartitionPath, + ShareableBuffer, +}; use crate::{crate_version, DeltaResult, DeltaTableError}; use arrow::datatypes::SchemaRef as ArrowSchemaRef; +use arrow::error::ArrowError; use arrow::record_batch::RecordBatch; use bytes::Bytes; use log::warn; use object_store::{path::Path, ObjectStore}; use parquet::arrow::ArrowWriter; +use parquet::basic::Compression; use parquet::file::properties::WriterProperties; // TODO databricks often suggests a file size of 100mb, should we set this default? @@ -35,6 +41,15 @@ enum WriteError { FileName { source: Box, }, + + #[error("Error handling Arrow data: {source}")] + Arrow { + #[from] + source: ArrowError, + }, + + #[error("Error partitioning record batch: {0}")] + Partitioning(String), } impl From for DeltaTableError { @@ -43,6 +58,7 @@ impl From for DeltaTableError { WriteError::SchemaMismatch { .. } => DeltaTableError::SchemaMismatch { msg: err.to_string(), }, + WriteError::Arrow { source } => DeltaTableError::Arrow { source }, _ => DeltaTableError::GenericError { source: Box::new(err), }, @@ -50,6 +66,179 @@ impl From for DeltaTableError { } } +pub(crate) struct WriterConfig { + /// Schema of the delta table + table_schema: ArrowSchemaRef, + /// Column names for columns the table is partitioned by + partition_columns: Vec, + /// Properties passed to underlying parquet writer + writer_properties: WriterProperties, + /// Size above which we will write a buffered parquet file to disk. + target_file_size: usize, + /// Row chunks passed to parquet writer. This and the internal parquet writer settings + /// determine how fine granular we can track / control the size of resulting files. + write_batch_size: usize, +} + +impl WriterConfig { + pub fn new( + table_schema: ArrowSchemaRef, + partition_columns: Vec, + writer_properties: Option, + target_file_size: Option, + write_batch_size: Option, + ) -> Self { + let writer_properties = writer_properties.unwrap_or_else(|| { + WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build() + }); + let target_file_size = target_file_size.unwrap_or(DEFAULT_TARGET_FILE_SIZE); + let write_batch_size = write_batch_size.unwrap_or(DEFAULT_WRITE_BATCH_SIZE); + + Self { + table_schema, + partition_columns, + writer_properties, + target_file_size, + write_batch_size, + } + } + + pub fn file_schema(&self) -> ArrowSchemaRef { + arrow_schema_without_partitions(&self.table_schema, &self.partition_columns) + } +} + +/// A parquet writer implementation tailored to the needs of writing data to a delta table. +pub(crate) struct DeltaWriter { + /// An object store pointing at Delta table root + object_store: ObjectStoreRef, + /// configuration for the writers + config: WriterConfig, + /// partition writers for individual partitions + partition_writers: HashMap, +} + +impl DeltaWriter { + pub fn new(object_store: ObjectStoreRef, config: WriterConfig) -> Self { + Self { + object_store, + config, + partition_writers: HashMap::new(), + } + } + + /// Creates a [`DeltaWriter`] to write data to provided Delta Table + // pub fn for_table( + // table: &DeltaTable, + // writer_properties: Option, + // ) -> DeltaResult { + // // Initialize an arrow schema ref from the delta table schema + // let metadata = table.get_metadata()?; + // let table_schema = Arc::new(>::try_from( + // &metadata.schema.clone(), + // )?); + // let partition_columns = metadata.partition_columns.clone(); + // + // // Initialize writer properties for the underlying arrow writer + // let writer_properties = writer_properties.unwrap_or_else(|| { + // WriterProperties::builder() + // .set_compression(Compression::SNAPPY) + // .build() + // }); + // + // // TODO get target file size form table meta data + // + // let config = WriterConfig::new( + // table_schema, + // partition_columns, + // Some(writer_properties), + // None, + // None, + // ); + // + // Ok(Self { + // object_store: table.storage.clone(), + // config, + // partition_writers: HashMap::new(), + // }) + // } + + fn divide_by_partition_values( + &mut self, + values: &RecordBatch, + ) -> DeltaResult> { + Ok(divide_by_partition_values( + self.config.file_schema(), + self.config.partition_columns.clone(), + values, + ) + .map_err(|err| WriteError::Partitioning(err.to_string()))?) + } + + /// Write a batch to the partition induced by the partition_values. The record batch is expected + /// to be pre-partitioned and only contain rows that belong into the same partition. + pub async fn write_partition( + &mut self, + record_batch: RecordBatch, + partition_values: &HashMap>, + ) -> DeltaResult<()> { + let partition_key = + PartitionPath::from_hashmap(&self.config.partition_columns, partition_values)? + .as_ref() + .into(); + + let record_batch = + record_batch_without_partitions(&record_batch, &self.config.partition_columns)?; + + match self.partition_writers.get_mut(&partition_key) { + Some(writer) => { + writer.write(&record_batch).await?; + } + None => { + let config = PartitionWriterConfig::try_new( + self.config.file_schema(), + partition_values.clone(), + self.config.partition_columns.clone(), + Some(self.config.writer_properties.clone()), + Some(self.config.target_file_size), + Some(self.config.write_batch_size), + )?; + let mut writer = + PartitionWriter::try_with_config(self.object_store.clone(), config)?; + writer.write(&record_batch).await?; + let _ = self.partition_writers.insert(partition_key, writer); + } + } + + Ok(()) + } + + /// Buffers record batches in-memory per partition up to appx. `target_file_size` for a partition. + /// Flushes data to storage once a full file can be written. + /// + /// The `close` method has to be invoked to write all data still buffered + /// and get the list of all written files. + pub async fn write(&mut self, batch: &RecordBatch) -> DeltaResult<()> { + for result in self.divide_by_partition_values(batch)? { + self.write_partition(result.record_batch, &result.partition_values) + .await?; + } + Ok(()) + } + + pub async fn close(mut self) -> DeltaResult> { + let writers = std::mem::take(&mut self.partition_writers); + let mut actions = Vec::new(); + for (_, writer) in writers { + let writer_actions = writer.close().await?; + actions.extend(writer_actions); + } + Ok(actions) + } +} + pub(crate) struct PartitionWriterConfig { /// Schema of the data written to disk file_schema: ArrowSchemaRef, @@ -138,6 +327,7 @@ impl PartitionWriter { fn next_data_path(&mut self) -> Path { let part = format!("{:0>5}", self.part_counter); self.part_counter += 1; + // TODO: what does c000 mean? // TODO handle file name for different compressions let file_name = format!("part-{}-{}-c000.snappy.parquet", part, self.writer_id); self.config.prefix.child(file_name) @@ -178,7 +368,7 @@ impl PartitionWriter { } async fn flush_arrow_writer(&mut self) -> DeltaResult<()> { - // replace counter / buffers and close the current writer + // replace counter / buffers adn close the current writer let (writer, buffer) = self.replace_arrow_buffer(vec![])?; let null_counts = std::mem::take(&mut self.null_counts); let metadata = writer.close()?; diff --git a/rust/src/time_utils.rs b/rust/src/time_utils.rs index 1df522622c..09aee331e4 100644 --- a/rust/src/time_utils.rs +++ b/rust/src/time_utils.rs @@ -1,9 +1,9 @@ //! Utility functions for converting time formats. -use parquet_format::TimeUnit; - #[cfg(feature = "arrow")] use arrow::temporal_conversions; +#[cfg(feature = "arrow")] +use parquet::basic::TimeUnit; // vendored from arrow-rs and arrow2 so we don't need to depend on arrow2 when the parquet2 feature // is enabled. @@ -95,7 +95,7 @@ pub fn timestamp_to_delta_stats_string(n: i64, time_unit: &TimeUnit) -> String { #[cfg(test)] mod tests { use super::*; - use parquet_format::{MicroSeconds, MilliSeconds, NanoSeconds, TimeUnit}; + use parquet::format::{MicroSeconds, MilliSeconds, NanoSeconds, TimeUnit}; #[test] fn test_timestamp_to_delta_stats_string() { diff --git a/rust/src/writer/stats.rs b/rust/src/writer/stats.rs index cea89ad683..f4c6043706 100644 --- a/rust/src/writer/stats.rs +++ b/rust/src/writer/stats.rs @@ -11,12 +11,10 @@ use arrow::{ }, buffer::MutableBuffer, }; -use parquet::{ - errors::ParquetError, - file::{metadata::RowGroupMetaData, statistics::Statistics}, - schema::types::{ColumnDescriptor, SchemaDescriptor}, -}; -use parquet_format::FileMetaData; +use parquet::errors::ParquetError; +use parquet::file::{metadata::RowGroupMetaData, statistics::Statistics}; +use parquet::format::FileMetaData; +use parquet::schema::types::{ColumnDescriptor, SchemaDescriptor}; use serde_json::{Number, Value}; use std::collections::HashMap; use std::sync::Arc; diff --git a/rust/tests/datafusion_test.rs b/rust/tests/datafusion_test.rs index ee1e60d406..a539467544 100644 --- a/rust/tests/datafusion_test.rs +++ b/rust/tests/datafusion_test.rs @@ -67,7 +67,7 @@ async fn prepare_table( .unwrap(); for batch in batches { - table = DeltaOps::from(table) + table = DeltaOps(table) .write(vec![batch]) .with_save_mode(save_mode.clone()) .await From 57d643b17655c2a1b38b940e42dfbe2c1b83ebd7 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Wed, 19 Oct 2022 21:42:36 +0200 Subject: [PATCH 06/19] fix: migrate arrow-pyarrow conversions --- Cargo.lock | 4 +++ python/Cargo.toml | 1 + python/src/lib.rs | 51 +++++++++++++++----------- python/src/schema.rs | 81 +++++++++++++++++++++++------------------- rust/src/time_utils.rs | 2 +- 5 files changed, 81 insertions(+), 58 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 317a1facf2..b2e4f9da7b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -160,6 +160,9 @@ name = "arrow-schema" version = "24.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "86d1fef01f25e1452c86fa6887f078de8e0aaeeb828370feab205944cfc30e27" +dependencies = [ + "serde", +] [[package]] name = "async-stream" @@ -808,6 +811,7 @@ dependencies = [ name = "deltalake-python" version = "0.6.1" dependencies = [ + "arrow-schema", "chrono", "deltalake", "env_logger 0.9.1", diff --git a/python/Cargo.toml b/python/Cargo.toml index 39344f54fb..6624f4fe47 100644 --- a/python/Cargo.toml +++ b/python/Cargo.toml @@ -17,6 +17,7 @@ crate-type = ["cdylib"] name = "deltalake._internal" [dependencies] +arrow-schema = { version = "24", features = ["serde"] } chrono = "0" env_logger = "0" futures = "0.3" diff --git a/python/src/lib.rs b/python/src/lib.rs index 7a7fc657ad..bbe86d21e6 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -4,6 +4,7 @@ mod filesystem; mod schema; mod utils; +use arrow::pyarrow::PyArrowType; use chrono::{DateTime, FixedOffset, Utc}; use deltalake::action::{ self, Action, ColumnCountStat, ColumnValueStat, DeltaOperation, SaveMode, Stats, @@ -269,8 +270,7 @@ impl RawDeltaTable { .map_err(PyDeltaTableError::from_raw)?; serde_json::to_string( &>::try_from(schema) - .map_err(PyDeltaTableError::from_arrow)? - .to_json(), + .map_err(PyDeltaTableError::from_arrow)?, ) .map_err(|_| PyDeltaTableError::new_err("Got invalid table schema")) } @@ -285,7 +285,7 @@ impl RawDeltaTable { &mut self, py: Python<'py>, partition_filters: Option>, - schema: ArrowSchema, + schema: PyArrowType, ) -> PyResult)>> { let path_set = match partition_filters { Some(filters) => Some(HashSet::<_>::from_iter( @@ -316,10 +316,12 @@ impl RawDeltaTable { add_actions: Vec, mode: &str, partition_by: Vec, - schema: ArrowSchema, + schema: PyArrowType, ) -> PyResult<()> { let mode = save_mode_from_str(mode)?; - let schema: Schema = (&schema).try_into()?; + let schema: Schema = (&schema.0) + .try_into() + .map_err(|_| PyDeltaTableError::new_err("Failed to convert schema"))?; let existing_schema = self ._table @@ -417,7 +419,7 @@ fn json_value_to_py(value: &serde_json::Value, py: Python) -> PyObject { /// skipped during a scan. fn filestats_to_expression<'py>( py: Python<'py>, - schema: &ArrowSchema, + schema: &PyArrowType, partitions_values: &HashMap>, stats: Option, ) -> PyResult> { @@ -427,14 +429,19 @@ fn filestats_to_expression<'py>( let mut expressions: Vec> = Vec::new(); let cast_to_type = |column_name: &String, value: PyObject, schema: &ArrowSchema| { - let column_type = schema - .field_with_name(column_name) - .map_err(|_| { - PyDeltaTableError::new_err(format!("Column not found in schema: {}", column_name)) - })? - .data_type() - .clone() - .into_py(py); + let column_type = PyArrowType( + schema + .field_with_name(column_name) + .map_err(|_| { + PyDeltaTableError::new_err(format!( + "Column not found in schema: {}", + column_name + )) + })? + .data_type() + .clone(), + ) + .into_py(py); pa.call_method1("scalar", (value,))? .call_method1("cast", (column_type,)) }; @@ -442,7 +449,7 @@ fn filestats_to_expression<'py>( for (column, value) in partitions_values.iter() { if let Some(value) = value { // value is a string, but needs to be parsed into appropriate type - let converted_value = cast_to_type(column, value.into_py(py), schema)?; + let converted_value = cast_to_type(column, value.into_py(py), &schema.0)?; expressions.push( field .call1((column,))? @@ -458,7 +465,7 @@ fn filestats_to_expression<'py>( // Blocked on https://issues.apache.org/jira/browse/ARROW-11259 _ => None, }) { - let maybe_minimum = cast_to_type(&col_name, minimum, schema); + let maybe_minimum = cast_to_type(&col_name, minimum, &schema.0); if let Ok(minimum) = maybe_minimum { expressions.push(field.call1((col_name,))?.call_method1("__ge__", (minimum,))); } @@ -468,7 +475,7 @@ fn filestats_to_expression<'py>( ColumnValueStat::Value(val) => Some((k.clone(), json_value_to_py(val, py))), _ => None, }) { - let maybe_maximum = cast_to_type(&col_name, maximum, schema); + let maybe_maximum = cast_to_type(&col_name, maximum, &schema.0); if let Ok(maximum) = maybe_maximum { expressions.push(field.call1((col_name,))?.call_method1("__le__", (maximum,))); } @@ -551,7 +558,7 @@ impl From<&PyAddAction> for action::Add { #[allow(clippy::too_many_arguments)] fn write_new_deltalake( table_uri: String, - schema: ArrowSchema, + schema: PyArrowType, add_actions: Vec, _mode: &str, partition_by: Vec, @@ -567,7 +574,9 @@ fn write_new_deltalake( name, description, None, // Format - (&schema).try_into()?, + (&schema.0) + .try_into() + .map_err(PyDeltaTableError::from_arrow)?, partition_by, configuration.unwrap_or_default(), ); @@ -610,10 +619,10 @@ impl PyDeltaDataChecker { } } - fn check_batch(&self, batch: RecordBatch) -> PyResult<()> { + fn check_batch(&self, batch: PyArrowType) -> PyResult<()> { self.rt.block_on(async { self.inner - .check_batch(&batch) + .check_batch(&batch.0) .await .map_err(PyDeltaTableError::from_raw) }) diff --git a/python/src/schema.rs b/python/src/schema.rs index be561d6bf4..3df81e6efa 100644 --- a/python/src/schema.rs +++ b/python/src/schema.rs @@ -4,6 +4,7 @@ use deltalake::arrow::datatypes::{ DataType as ArrowDataType, Field as ArrowField, Schema as ArrowSchema, }; use deltalake::arrow::error::ArrowError; +use deltalake::arrow::pyarrow::PyArrowType; use deltalake::schema::{ Schema, SchemaDataType, SchemaField, SchemaTypeArray, SchemaTypeMap, SchemaTypeStruct, }; @@ -211,11 +212,11 @@ impl PrimitiveType { /// /// :rtype: pyarrow.DataType #[pyo3(text_signature = "($self)")] - fn to_pyarrow(&self) -> PyResult { + fn to_pyarrow(&self) -> PyResult> { let inner_type = SchemaDataType::primitive(self.inner_type.clone()); - (&inner_type) - .try_into() - .map_err(|err: ArrowError| PyException::new_err(err.to_string())) + Ok(PyArrowType((&inner_type).try_into().map_err( + |err: ArrowError| PyException::new_err(err.to_string()), + )?)) } /// Create a PrimitiveType from a PyArrow type @@ -227,8 +228,8 @@ impl PrimitiveType { /// :rtype: PrimitiveType #[pyo3(text_signature = "(data_type)")] #[staticmethod] - fn from_pyarrow(data_type: ArrowDataType) -> PyResult { - let inner_type: SchemaDataType = (&data_type) + fn from_pyarrow(data_type: PyArrowType) -> PyResult { + let inner_type: SchemaDataType = (&data_type.0) .try_into() .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; @@ -374,10 +375,12 @@ impl ArrayType { /// /// :rtype: pyarrow.DataType #[pyo3(text_signature = "($self)")] - fn to_pyarrow(&self) -> PyResult { - (&SchemaDataType::array(self.inner_type.clone())) - .try_into() - .map_err(|err: ArrowError| PyException::new_err(err.to_string())) + fn to_pyarrow(&self) -> PyResult> { + Ok(PyArrowType( + (&SchemaDataType::array(self.inner_type.clone())) + .try_into() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?, + )) } /// Create an ArrayType from a pyarrow.ListType. @@ -389,8 +392,8 @@ impl ArrayType { /// :rtype: ArrayType #[staticmethod] #[pyo3(text_signature = "(data_type)")] - fn from_pyarrow(data_type: ArrowDataType) -> PyResult { - let inner_type: SchemaDataType = (&data_type) + fn from_pyarrow(data_type: PyArrowType) -> PyResult { + let inner_type: SchemaDataType = (&data_type.0) .try_into() .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; @@ -556,10 +559,12 @@ impl MapType { /// /// :rtype: pyarrow.MapType #[pyo3(text_signature = "($self)")] - fn to_pyarrow(&self) -> PyResult { - (&SchemaDataType::map(self.inner_type.clone())) - .try_into() - .map_err(|err: ArrowError| PyException::new_err(err.to_string())) + fn to_pyarrow(&self) -> PyResult> { + Ok(PyArrowType( + (&SchemaDataType::map(self.inner_type.clone())) + .try_into() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?, + )) } /// Create a MapType from a PyArrow MapType. @@ -571,8 +576,8 @@ impl MapType { /// :rtype: MapType #[staticmethod] #[pyo3(text_signature = "(data_type)")] - fn from_pyarrow(data_type: ArrowDataType) -> PyResult { - let inner_type: SchemaDataType = (&data_type) + fn from_pyarrow(data_type: PyArrowType) -> PyResult { + let inner_type: SchemaDataType = (&data_type.0) .try_into() .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; @@ -744,10 +749,10 @@ impl Field { /// /// :rtype: pyarrow.Field #[pyo3(text_signature = "($self)")] - fn to_pyarrow(&self) -> PyResult { - (&self.inner) - .try_into() - .map_err(|err: ArrowError| PyException::new_err(err.to_string())) + fn to_pyarrow(&self) -> PyResult> { + Ok(PyArrowType((&self.inner).try_into().map_err( + |err: ArrowError| PyException::new_err(err.to_string()), + )?)) } /// Create a Field from a PyArrow field @@ -759,9 +764,9 @@ impl Field { /// :rtype: Field #[staticmethod] #[pyo3(text_signature = "(field)")] - fn from_pyarrow(field: ArrowField) -> PyResult { + fn from_pyarrow(field: PyArrowType) -> PyResult { Ok(Self { - inner: SchemaField::try_from(&field) + inner: SchemaField::try_from(&field.0) .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?, }) } @@ -892,10 +897,12 @@ impl StructType { /// /// :rtype: pyarrow.StructType #[pyo3(text_signature = "($self)")] - fn to_pyarrow(&self) -> PyResult { - (&SchemaDataType::r#struct(self.inner_type.clone())) - .try_into() - .map_err(|err: ArrowError| PyException::new_err(err.to_string())) + fn to_pyarrow(&self) -> PyResult> { + Ok(PyArrowType( + (&SchemaDataType::r#struct(self.inner_type.clone())) + .try_into() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?, + )) } /// Create a new StructType from a PyArrow struct type. @@ -907,8 +914,8 @@ impl StructType { /// :rtype: StructType #[staticmethod] #[pyo3(text_signature = "(data_type)")] - fn from_pyarrow(data_type: ArrowDataType) -> PyResult { - let inner_type: SchemaDataType = (&data_type) + fn from_pyarrow(data_type: PyArrowType) -> PyResult { + let inner_type: SchemaDataType = (&data_type.0) .try_into() .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; @@ -1003,11 +1010,13 @@ impl PySchema { /// /// :rtype: pyarrow.Schema #[pyo3(text_signature = "($self)")] - fn to_pyarrow(self_: PyRef<'_, Self>) -> PyResult { + fn to_pyarrow(self_: PyRef<'_, Self>) -> PyResult> { let super_ = self_.as_ref(); - (&super_.inner_type.clone()) - .try_into() - .map_err(|err: ArrowError| PyException::new_err(err.to_string())) + Ok(PyArrowType( + (&super_.inner_type.clone()) + .try_into() + .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?, + )) } /// Create from a PyArrow schema @@ -1017,8 +1026,8 @@ impl PySchema { /// :rtype: Schema #[staticmethod] #[pyo3(text_signature = "(data_type)")] - fn from_pyarrow(data_type: ArrowSchema, py: Python) -> PyResult { - let inner_type: SchemaTypeStruct = (&data_type) + fn from_pyarrow(data_type: PyArrowType, py: Python) -> PyResult { + let inner_type: SchemaTypeStruct = (&data_type.0) .try_into() .map_err(|err: ArrowError| PyException::new_err(err.to_string()))?; diff --git a/rust/src/time_utils.rs b/rust/src/time_utils.rs index 09aee331e4..0d79107c05 100644 --- a/rust/src/time_utils.rs +++ b/rust/src/time_utils.rs @@ -2,7 +2,7 @@ #[cfg(feature = "arrow")] use arrow::temporal_conversions; -#[cfg(feature = "arrow")] + use parquet::basic::TimeUnit; // vendored from arrow-rs and arrow2 so we don't need to depend on arrow2 when the parquet2 feature From e42743d5e4c90995545747a8d11c94a7126a420e Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Wed, 19 Oct 2022 22:24:56 +0200 Subject: [PATCH 07/19] test: fix parquet2 build --- python/src/lib.rs | 2 +- rust/Cargo.toml | 2 +- rust/src/delta.rs | 1 + rust/src/time_utils.rs | 30 +++++++++++++++++++++++++++++- 4 files changed, 32 insertions(+), 3 deletions(-) diff --git a/python/src/lib.rs b/python/src/lib.rs index bbe86d21e6..e631a9a7c8 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -321,7 +321,7 @@ impl RawDeltaTable { let mode = save_mode_from_str(mode)?; let schema: Schema = (&schema.0) .try_into() - .map_err(|_| PyDeltaTableError::new_err("Failed to convert schema"))?; + .map_err(PyDeltaTableError::from_arrow)?; let existing_schema = self ._table diff --git a/rust/Cargo.toml b/rust/Cargo.toml index 9c3b513159..0543f464fe 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -75,7 +75,7 @@ utime = "0.3" glibc_version = "0" [features] -default = ["arrow", "parquet", "datafusion-ext"] +default = ["arrow", "parquet"] datafusion-ext = [ "datafusion", "datafusion-expr", diff --git a/rust/src/delta.rs b/rust/src/delta.rs index c4445f21d1..b2e89c65e7 100644 --- a/rust/src/delta.rs +++ b/rust/src/delta.rs @@ -95,6 +95,7 @@ pub enum DeltaTableError { #[cfg(feature = "parquet")] #[from] source: parquet::errors::ParquetError, + /// Parquet error details returned when reading the checkpoint failed. #[cfg(feature = "parquet2")] #[from] source: parquet2::error::Error, diff --git a/rust/src/time_utils.rs b/rust/src/time_utils.rs index 0d79107c05..e947ee4ef5 100644 --- a/rust/src/time_utils.rs +++ b/rust/src/time_utils.rs @@ -2,8 +2,10 @@ #[cfg(feature = "arrow")] use arrow::temporal_conversions; - +#[cfg(not(feature = "parquet2"))] use parquet::basic::TimeUnit; +#[cfg(feature = "parquet2")] +use parquet2::schema::types::TimeUnit; // vendored from arrow-rs and arrow2 so we don't need to depend on arrow2 when the parquet2 feature // is enabled. @@ -82,6 +84,7 @@ pub fn timestamp_micros_from_stats_string(s: &str) -> Result String { let dt = match time_unit { TimeUnit::MILLIS(_) => temporal_conversions::timestamp_ms_to_datetime(n), @@ -92,11 +95,25 @@ pub fn timestamp_to_delta_stats_string(n: i64, time_unit: &TimeUnit) -> String { format!("{}", dt.format("%Y-%m-%dT%H:%M:%S%.3fZ")) } +/// Convert the timestamp to a ISO-8601 style format suitable for JSON statistics. +#[cfg(feature = "parquet2")] +pub fn timestamp_to_delta_stats_string(n: i64, time_unit: &TimeUnit) -> String { + let dt = match time_unit { + TimeUnit::Milliseconds => temporal_conversions::timestamp_ms_to_datetime(n), + TimeUnit::Microseconds => temporal_conversions::timestamp_us_to_datetime(n), + TimeUnit::Nanoseconds => temporal_conversions::timestamp_ns_to_datetime(n), + }; + + format!("{}", dt.format("%Y-%m-%dT%H:%M:%S%.3fZ")) +} + #[cfg(test)] mod tests { use super::*; + #[cfg(not(feature = "parquet2"))] use parquet::format::{MicroSeconds, MilliSeconds, NanoSeconds, TimeUnit}; + #[cfg(not(feature = "parquet2"))] #[test] fn test_timestamp_to_delta_stats_string() { let s = @@ -114,6 +131,17 @@ mod tests { assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); } + #[cfg(feature = "parquet2")] + #[test] + fn test_timestamp_to_delta_stats_string() { + let s = timestamp_to_delta_stats_string(1628685199541, &TimeUnit::Milliseconds); + assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); + let s = timestamp_to_delta_stats_string(1628685199541000, &TimeUnit::Microseconds); + assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); + let s = timestamp_to_delta_stats_string(1628685199541000000, &TimeUnit::Nanoseconds); + assert_eq!("2021-08-11T12:33:19.541Z".to_string(), s); + } + #[test] fn test_timestamp_micros_from_stats_string() { let us = timestamp_micros_from_stats_string("2021-08-11T12:33:19.541Z").unwrap(); From 7cfcf5b9ce85803aa0203cc15f881a41281cf782 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Wed, 19 Oct 2022 22:46:49 +0200 Subject: [PATCH 08/19] try using tempfile to fix failing windows tests --- rust/src/operations/create.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 0ba8372fda..e0bf03428f 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -327,7 +327,7 @@ mod tests { #[tokio::test] async fn test_create_table_save_mode() { - let tmp_dir = tempdir::TempDir::new("").unwrap(); + let tmp_dir = tempfile::tempdir().unwrap(); let schema = get_delta_schema(); let table = CreateBuilder::new() From cb6e2140ad529f7d2aada42a46ec1941cccb0b4f Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 20 Oct 2022 00:13:53 +0200 Subject: [PATCH 09/19] fix: use list to determine if delta location --- rust/src/storage/mod.rs | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/rust/src/storage/mod.rs b/rust/src/storage/mod.rs index d1522ebe47..232e68a972 100644 --- a/rust/src/storage/mod.rs +++ b/rust/src/storage/mod.rs @@ -157,10 +157,16 @@ impl DeltaObjectStore { /// Check if the location is a delta table location pub async fn is_delta_table_location(&self) -> ObjectStoreResult { - match self.head(self.log_path()).await { - Ok(_) => Ok(true), - Err(ObjectStoreError::NotFound { .. }) => Ok(false), - Err(err) => Err(err), + // TODO We should really be using HEAD here, but this fails in windows tests + let mut stream = self.list(Some(self.log_path())).await?; + if let Some(res) = stream.next().await { + match res { + Ok(_) => Ok(true), + Err(ObjectStoreError::NotFound { .. }) => Ok(false), + Err(err) => Err(err), + } + } else { + Ok(false) } } } From 47ac52cf099656216a133aaa2d949a0fbc8dc96a Mon Sep 17 00:00:00 2001 From: Robert Pack <42610831+roeap@users.noreply.github.com> Date: Thu, 27 Oct 2022 11:20:33 +0200 Subject: [PATCH 10/19] Apply suggestions from code review Co-authored-by: Will Jones --- rust/src/operations/create.rs | 14 ++++++++------ rust/src/operations/load.rs | 13 ------------- rust/src/operations/write.rs | 2 +- 3 files changed, 9 insertions(+), 20 deletions(-) diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index e0bf03428f..02ce4b5802 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -18,16 +18,16 @@ use serde_json::Value; #[derive(thiserror::Error, Debug)] enum CreateError { - #[error("Location must be provided to create a table")] + #[error("Location must be provided to create a table.")] MissingLocation, - #[error("At least one column must be defined to create a table")] + #[error("At least one column must be defined to create a table.")] MissingSchema, #[error("Please configure table meta data via the CreateBuilder.")] MetadataSpecified, - #[error("Please configure table meta data via the CreateBuilder.")] + #[error("A Delta Lake table already exists at that location.")] TableAlreadyExists, #[error("SaveMode `append` is not allowed for create operation.")] @@ -42,7 +42,7 @@ impl From for DeltaTableError { } } -/// Build a new [DeltaTable] +/// Build an operation to create a new [DeltaTable] #[derive(Debug, Clone)] pub struct CreateBuilder { name: Option, @@ -50,7 +50,7 @@ pub struct CreateBuilder { mode: SaveMode, comment: Option, columns: Vec, - partition_columns: Option>, + partition_columns: Vec, properties: HashMap, storage_options: Option>, actions: Vec, @@ -150,6 +150,8 @@ impl CreateBuilder { /// /// [crate::builder::s3_storage_options] describes the available options for the AWS or S3-compliant backend. /// [dynamodb_lock::DynamoDbLockClient] describes additional options for the AWS atomic rename client. + /// + /// If an object store is also passed using `with_object_store()` these options will be ignored. pub fn with_storage_options(mut self, storage_options: HashMap) -> Self { self.storage_options = Some(storage_options); self @@ -167,7 +169,7 @@ impl CreateBuilder { self } - /// Consume self into uninitialized table with corresponding crate actions and operation meta + /// Consume self into uninitialized table with corresponding create actions and operation meta pub(crate) fn into_table_and_actions( self, ) -> DeltaResult<(DeltaTable, Vec, DeltaOperation)> { diff --git a/rust/src/operations/load.rs b/rust/src/operations/load.rs index 39a4780a7f..5a6158c0ac 100644 --- a/rust/src/operations/load.rs +++ b/rust/src/operations/load.rs @@ -10,19 +10,6 @@ use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; use futures::future::BoxFuture; -// #[derive(thiserror::Error, Debug)] -// enum LoadError { -// Task, -// } -// -// impl From for DeltaTableError { -// fn from(err: LoadError) -> Self { -// DeltaTableError::GenericError { -// source: Box::new(err), -// } -// } -// } - #[derive(Debug, Clone)] pub struct LoadBuilder { location: Option, diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index cadf1f2eba..2f567a878f 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -84,7 +84,7 @@ pub struct WriteBuilder { /// SaveMode defines how to treat data already written to table location mode: SaveMode, /// Column names for table partitioning - partition_columns: Option>, + partition_columns: Vec, /// When using `Overwrite` mode, replace data that matches a predicate predicate: Option, /// Size above which we will write a buffered parquet file to disk. From a3361c1c71ca792c108b59dcc09085dd6a929927 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 28 Oct 2022 21:46:57 +0200 Subject: [PATCH 11/19] revert option partition columns --- rust/src/operations/create.rs | 2 +- rust/src/operations/write.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 02ce4b5802..5e8f82417d 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -50,7 +50,7 @@ pub struct CreateBuilder { mode: SaveMode, comment: Option, columns: Vec, - partition_columns: Vec, + partition_columns: Option>, properties: HashMap, storage_options: Option>, actions: Vec, diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index 2f567a878f..cadf1f2eba 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -84,7 +84,7 @@ pub struct WriteBuilder { /// SaveMode defines how to treat data already written to table location mode: SaveMode, /// Column names for table partitioning - partition_columns: Vec, + partition_columns: Option>, /// When using `Overwrite` mode, replace data that matches a predicate predicate: Option, /// Size above which we will write a buffered parquet file to disk. From 9f3727e76bc31ed0e6ba3df9d9efb5e95e434b1e Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Thu, 3 Nov 2022 19:28:20 +0100 Subject: [PATCH 12/19] PR comments --- rust/src/operations/create.rs | 3 ++- rust/src/operations/write.rs | 2 -- rust/src/operations/writer.rs | 39 ++--------------------------------- 3 files changed, 4 insertions(+), 40 deletions(-) diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 5e8f82417d..407441570f 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -296,7 +296,8 @@ mod tests { .with_save_mode(SaveMode::Ignore) .await .unwrap(); - assert_eq!(table.version(), 0) + assert_eq!(table.version(), 0); + assert_eq!(table.get_metadata().unwrap().schema, table_schema) } #[tokio::test] diff --git a/rust/src/operations/write.rs b/rust/src/operations/write.rs index cadf1f2eba..c18a1428ff 100644 --- a/rust/src/operations/write.rs +++ b/rust/src/operations/write.rs @@ -14,8 +14,6 @@ //! //! In combination with `Overwrite`, a `replaceWhere` option can be used to transactionally //! replace data that matches a predicate. -//! -//! # Example // https://github.com/delta-io/delta/blob/master/core/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala diff --git a/rust/src/operations/writer.rs b/rust/src/operations/writer.rs index cab5a5d8f3..f49a00d9b5 100644 --- a/rust/src/operations/writer.rs +++ b/rust/src/operations/writer.rs @@ -129,42 +129,6 @@ impl DeltaWriter { } } - /// Creates a [`DeltaWriter`] to write data to provided Delta Table - // pub fn for_table( - // table: &DeltaTable, - // writer_properties: Option, - // ) -> DeltaResult { - // // Initialize an arrow schema ref from the delta table schema - // let metadata = table.get_metadata()?; - // let table_schema = Arc::new(>::try_from( - // &metadata.schema.clone(), - // )?); - // let partition_columns = metadata.partition_columns.clone(); - // - // // Initialize writer properties for the underlying arrow writer - // let writer_properties = writer_properties.unwrap_or_else(|| { - // WriterProperties::builder() - // .set_compression(Compression::SNAPPY) - // .build() - // }); - // - // // TODO get target file size form table meta data - // - // let config = WriterConfig::new( - // table_schema, - // partition_columns, - // Some(writer_properties), - // None, - // None, - // ); - // - // Ok(Self { - // object_store: table.storage.clone(), - // config, - // partition_writers: HashMap::new(), - // }) - // } - fn divide_by_partition_values( &mut self, values: &RecordBatch, @@ -179,6 +143,7 @@ impl DeltaWriter { /// Write a batch to the partition induced by the partition_values. The record batch is expected /// to be pre-partitioned and only contain rows that belong into the same partition. + /// However, it should still contain the partition columns. pub async fn write_partition( &mut self, record_batch: RecordBatch, @@ -414,7 +379,7 @@ impl PartitionWriter { for offset in (0..max_offset).step_by(self.config.write_batch_size) { let length = usize::min(self.config.write_batch_size, max_offset - offset); self.write_batch(&batch.slice(offset, length))?; - // flush currently buffered data to disc once we meet ot exceed the tart file size. + // flush currently buffered data to disk once we meet or exceed the target file size. if self.buffer.len() >= self.config.target_file_size { log::debug!("Writing file with size {:?} to disk.", self.buffer.len()); self.flush_arrow_writer().await?; From 1294fee8d4704adacaf82ad18b3631cae0682ef1 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 4 Nov 2022 21:51:24 +0100 Subject: [PATCH 13/19] chore: fix some 1.65 clippy errors --- rust/src/checkpoints.rs | 4 ++-- rust/src/writer/stats.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/rust/src/checkpoints.rs b/rust/src/checkpoints.rs index be41867dfd..12fa3e032d 100644 --- a/rust/src/checkpoints.rs +++ b/rust/src/checkpoints.rs @@ -146,7 +146,7 @@ async fn create_checkpoint_for( debug!("Writing checkpoint to {:?}.", checkpoint_path); storage.put(&checkpoint_path, parquet_bytes).await?; - let last_checkpoint_content: Value = serde_json::to_value(&checkpoint)?; + let last_checkpoint_content: Value = serde_json::to_value(checkpoint)?; let last_checkpoint_content = bytes::Bytes::from(serde_json::to_vec(&last_checkpoint_content)?); debug!("Writing _last_checkpoint to {:?}.", last_checkpoint_path); @@ -359,7 +359,7 @@ fn parquet_bytes_from_state(state: &DeltaTableState) -> Result, - nest_level: i32, + #[allow(clippy::only_used_in_recursion)] nest_level: i32, ) { let fields = match array.data_type() { DataType::Struct(fields) => fields, From ce37e4f4de583012f70d466a400bf433a008d37c Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Fri, 4 Nov 2022 22:05:43 +0100 Subject: [PATCH 14/19] chore: more clippy --- dynamodb_lock/src/lib.rs | 2 +- rust/src/writer/stats.rs | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dynamodb_lock/src/lib.rs b/dynamodb_lock/src/lib.rs index 2495404300..bef745ab02 100644 --- a/dynamodb_lock/src/lib.rs +++ b/dynamodb_lock/src/lib.rs @@ -486,7 +486,7 @@ impl DynamoDbLockClient { PARTITION_KEY_NAME.to_string() => attr(self.opts.partition_key_value.clone()), OWNER_NAME.to_string() => attr(&self.opts.owner_name), RECORD_VERSION_NUMBER.to_string() => attr(&rvn), - LEASE_DURATION.to_string() => attr(&self.opts.lease_duration), + LEASE_DURATION.to_string() => attr(self.opts.lease_duration), }; if let Some(d) = data { diff --git a/rust/src/writer/stats.rs b/rust/src/writer/stats.rs index e80e8eb805..b36376450c 100644 --- a/rust/src/writer/stats.rs +++ b/rust/src/writer/stats.rs @@ -29,7 +29,8 @@ pub type MinAndMaxValues = ( pub(crate) fn apply_null_counts( array: &StructArray, null_counts: &mut HashMap, - #[allow(clippy::only_used_in_recursion)] nest_level: i32, + // NOTE clippy wants the underscore + _nest_level: i32, ) { let fields = match array.data_type() { DataType::Struct(fields) => fields, @@ -52,7 +53,7 @@ pub(crate) fn apply_null_counts( match col_struct { ColumnCountStat::Column(map) => { - apply_null_counts(as_struct_array(column), map, nest_level + 1); + apply_null_counts(as_struct_array(column), map, _nest_level + 1); } _ => unreachable!(), } From 04d7584ed7d0240330fe0ec7ed91cbdef79f98c9 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 6 Nov 2022 20:51:17 +0100 Subject: [PATCH 15/19] chore: remove unstable option --- Cargo.toml | 3 --- 1 file changed, 3 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 6aee43c6bb..8c9f633ce0 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -8,6 +8,3 @@ members = [ "dynamodb_lock", ] exclude = ["proofs", "delta-inspect"] - -[profile.dev] -split-debuginfo = "unpacked" From 21814600a692db0809f15405072ffe01481206dd Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 6 Nov 2022 21:42:42 +0100 Subject: [PATCH 16/19] fix: fail on create with overwrite --- rust/src/operations/create.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 407441570f..1a8adb4d1d 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -265,9 +265,7 @@ impl std::future::IntoFuture for CreateBuilder { return Ok(table); } SaveMode::Overwrite => { - let curr_files = - flatten_list_stream(table.object_store().as_ref(), None).await?; - table.object_store().delete_batch(&curr_files).await?; + todo!("Overwriting on create not yet implemented. Use 'write' operation instead.") } } } From 6b30e97cf10f366e4378008a89c7b7d579127a96 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Sun, 6 Nov 2022 22:37:18 +0100 Subject: [PATCH 17/19] fix: unused import --- rust/src/operations/create.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index 1a8adb4d1d..fe32a61d55 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -10,7 +10,7 @@ use super::{MAX_SUPPORTED_READER_VERSION, MAX_SUPPORTED_WRITER_VERSION}; use crate::action::{Action, DeltaOperation, MetaData, Protocol, SaveMode}; use crate::builder::StorageUrl; use crate::schema::{SchemaDataType, SchemaField, SchemaTypeStruct}; -use crate::storage::{utils::flatten_list_stream, DeltaObjectStore}; +use crate::storage::DeltaObjectStore; use crate::{DeltaResult, DeltaTable, DeltaTableError}; use futures::future::BoxFuture; From 200e1d7411f96a3cf0419726b73b6ad52d321070 Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Tue, 15 Nov 2022 23:58:58 +0100 Subject: [PATCH 18/19] fix: omit create overwrite test Signed-off-by: Robert Pack --- Cargo.lock | 295 +++++++++++++++++----------- rust/src/action/parquet_read/mod.rs | 23 ++- rust/src/delta_datafusion.rs | 8 +- rust/src/operations/create.rs | 14 +- 4 files changed, 203 insertions(+), 137 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f50fd02dda..3dbd761012 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -21,9 +21,9 @@ dependencies = [ [[package]] name = "ahash" -version = "0.8.0" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57e6e951cfbb2db8de1828d49073a113a29fd7117b1596caa781a258c7e38d72" +checksum = "bf6ccdb167abbf410dcb915cabd428929d7f6a04980b54a11f26a39f1c7f7107" dependencies = [ "cfg-if", "const-random", @@ -95,7 +95,7 @@ version = "24.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d68391300d5237f6725f0f869ae7cb65d45fcf8a6d18f6ceecd328fb803bef93" dependencies = [ - "ahash 0.8.0", + "ahash 0.8.2", "arrow-array", "arrow-buffer", "arrow-data", @@ -124,7 +124,7 @@ version = "24.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f0bb00c5862b5eea683812083c495bef01a9a5149da46ad2f4c0e4aa8800f64d" dependencies = [ - "ahash 0.8.0", + "ahash 0.8.2", "arrow-buffer", "arrow-data", "arrow-schema", @@ -215,9 +215,9 @@ checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" [[package]] name = "base64" -version = "0.13.0" +version = "0.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "904dfeac50f3cdaba28fc6f57fdcddb75f49ed61346676a78c4ffe55877802fd" +checksum = "9e1b586273c5702936fe7b7d6896644d8be71e6314cfe09d3167c95f712589e8" [[package]] name = "bitflags" @@ -227,9 +227,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "blake2" -version = "0.10.4" +version = "0.10.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9cf849ee05b2ee5fba5e36f97ff8ec2533916700fc0758d40d92136a42f3388" +checksum = "b12e5fd123190ce1c2e559308a94c9bacad77907d4c6005d9e58fe1a0689e55e" dependencies = [ "digest 0.10.5", ] @@ -301,9 +301,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.11.0" +version = "3.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1ad822118d20d2c234f427000d5acc36eabe1e29a348c89b63dd60b13f28e5d" +checksum = "572f695136211188308f16ad2ca5c851a712c464060ae6974944458eb83880ba" [[package]] name = "byteorder" @@ -325,9 +325,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.0.73" +version = "1.0.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fff2a6927b3bb87f9595d67196a70493f627687a71d87a0d692242c33f58c11" +checksum = "76a284da2e6fe2092f2353e51713435363112dfd60030e22add80be333fb928f" dependencies = [ "jobserver", ] @@ -340,9 +340,9 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "chrono" -version = "0.4.22" +version = "0.4.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bfd4d1b31faaa3a89d7934dbded3111da0d2ef28e3ebccdb4f0179f5929d1ef1" +checksum = "16b0a3d9ed01224b22057780a37bb8c5dbfe1be8ba48678e7bf57ec4b385411f" dependencies = [ "iana-time-zone", "js-sys", @@ -383,9 +383,9 @@ dependencies = [ [[package]] name = "clap" -version = "3.2.22" +version = "3.2.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86447ad904c7fb335a790c9d7fe3d0d971dc523b8ccd1561a520de9a85302750" +checksum = "71655c45cb9845d3270c9d6df84ebe72b4dad3c2ba3f7023ad47c144e4e473a5" dependencies = [ "bitflags", "clap_lex", @@ -414,9 +414,9 @@ dependencies = [ [[package]] name = "comfy-table" -version = "6.1.0" +version = "6.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85914173c2f558d61613bfbbf1911f14e630895087a7ed2fafc0f5319e1536e7" +checksum = "1090f39f45786ec6dc6286f8ea9c75d0a7ef0a0d3cda674cef0c3af7b307fbc2" dependencies = [ "strum", "strum_macros", @@ -425,9 +425,9 @@ dependencies = [ [[package]] name = "const-random" -version = "0.1.13" +version = "0.1.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f590d95d011aa80b063ffe3253422ed5aa462af4e9867d43ce8337562bac77c4" +checksum = "368a7a772ead6ce7e1de82bfb04c485f3db8ec744f72925af5735e29a22cc18e" dependencies = [ "const-random-macro", "proc-macro-hack", @@ -435,12 +435,12 @@ dependencies = [ [[package]] name = "const-random-macro" -version = "0.1.13" +version = "0.1.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "615f6e27d000a2bffbc7f2f6a8669179378fa27ee4d0a509e985dfc0a7defb40" +checksum = "9d7d6ab3c3a2282db210df5f02c4dab6e0a7057af0fb7ebd4070f30fe05c0ddb" dependencies = [ "getrandom", - "lazy_static", + "once_cell", "proc-macro-hack", "tiny-keccak", ] @@ -614,9 +614,9 @@ dependencies = [ [[package]] name = "ctor" -version = "0.1.23" +version = "0.1.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cdffe87e1d521a10f9696f833fe502293ea446d7f256c06128293a4119bdf4cb" +checksum = "6d2301688392eb071b0bf1a37be05c469d3cc4dbbd95df672fe28ab021e6a096" dependencies = [ "quote", "syn", @@ -624,9 +624,9 @@ dependencies = [ [[package]] name = "cxx" -version = "1.0.78" +version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19f39818dcfc97d45b03953c1292efc4e80954e1583c4aa770bac1383e2310a4" +checksum = "97abf9f0eca9e52b7f81b945524e76710e6cb2366aead23b7d4fbf72e281f888" dependencies = [ "cc", "cxxbridge-flags", @@ -636,9 +636,9 @@ dependencies = [ [[package]] name = "cxx-build" -version = "1.0.78" +version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e580d70777c116df50c390d1211993f62d40302881e54d4b79727acb83d0199" +checksum = "7cc32cc5fea1d894b77d269ddb9f192110069a8a9c1f1d441195fba90553dea3" dependencies = [ "cc", "codespan-reporting", @@ -651,15 +651,15 @@ dependencies = [ [[package]] name = "cxxbridge-flags" -version = "1.0.78" +version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56a46460b88d1cec95112c8c363f0e2c39afdb237f60583b0b36343bf627ea9c" +checksum = "8ca220e4794c934dc6b1207c3b42856ad4c302f2df1712e9f8d2eec5afaacf1f" [[package]] name = "cxxbridge-macro" -version = "1.0.78" +version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "747b608fecf06b0d72d440f27acc99288207324b793be2c17991839f3d4995ea" +checksum = "b846f081361125bfc8dc9d3940c84e1fd83ba54bbca7b17cd29483c828be0704" dependencies = [ "proc-macro2", "quote", @@ -685,7 +685,7 @@ version = "13.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a2bdec06a3db088da76fc28cb0877b8b5438ca6b6025e04d975bace0fd85df19" dependencies = [ - "ahash 0.8.0", + "ahash 0.8.2", "arrow", "async-trait", "bytes", @@ -704,7 +704,7 @@ dependencies = [ "log", "num_cpus", "object_store", - "ordered-float 3.2.0", + "ordered-float 3.4.0", "parking_lot", "parquet", "paste", @@ -727,7 +727,7 @@ checksum = "506eab038bf2d39ac02c22be30b019873ca01f887148b939d309a0e9523f4515" dependencies = [ "arrow", "object_store", - "ordered-float 3.2.0", + "ordered-float 3.4.0", "parquet", "sqlparser", ] @@ -738,7 +738,7 @@ version = "13.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b3d2810e369c735d69479e27fe8410e97a76ed07484aa9b3ad7c039efa504257" dependencies = [ - "ahash 0.8.0", + "ahash 0.8.2", "arrow", "datafusion-common", "sqlparser", @@ -766,7 +766,7 @@ version = "13.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e9bf3b7ae861d351a85174fd4fddb29d249950b2f23671318971960452b4b9ab" dependencies = [ - "ahash 0.8.0", + "ahash 0.8.2", "arrow", "blake2", "blake3", @@ -777,7 +777,7 @@ dependencies = [ "hashbrown", "lazy_static", "md-5 0.10.5", - "ordered-float 3.2.0", + "ordered-float 3.4.0", "paste", "rand 0.8.5", "regex", @@ -867,7 +867,7 @@ dependencies = [ "arrow-schema", "chrono", "deltalake", - "env_logger 0.9.1", + "env_logger 0.9.3", "futures", "lazy_static", "pyo3", @@ -990,9 +990,9 @@ dependencies = [ [[package]] name = "env_logger" -version = "0.9.1" +version = "0.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c90bf5f19754d10198ccb95b70664fc925bd1fc090a0fd9a6ebc54acc8cd6272" +checksum = "a12e6657c4c97ebab115a42dcee77225f7f482cdd841cf7088c657a42e9e00e7" dependencies = [ "atty", "humantime 2.1.0", @@ -1201,9 +1201,9 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.7" +version = "0.2.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4eb1a864a501629691edf6c15a593b7a51eebaa1e8468e9ddc623de7c9b58ec6" +checksum = "c05aeb6a22b8f62540c194aac980f2115af067bfe15a0734d7277a768d396b31" dependencies = [ "cfg-if", "libc", @@ -1234,9 +1234,9 @@ checksum = "9b919933a397b79c37e33b77bb2aa3dc8eb6e165ad809e58ff75bc7db2e34574" [[package]] name = "h2" -version = "0.3.14" +version = "0.3.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ca32592cf21ac7ccab1825cd87f6c9b3d9022c44d086172ed0966bec8af30be" +checksum = "5f9f29bc9dda355256b2916cf526ab02ce0aeaaaf2bad60d65ef3f12f11dd0f4" dependencies = [ "bytes", "fnv", @@ -1358,9 +1358,9 @@ checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" [[package]] name = "hyper" -version = "0.14.20" +version = "0.14.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02c929dc5c39e335a03c405292728118860721b10190d98c2a0f0efd5baafbac" +checksum = "034711faac9d2166cb1baf1a2fb0b60b1f277f8492fd72176c17f3515e1abd3c" dependencies = [ "bytes", "futures-channel", @@ -1382,9 +1382,9 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.23.0" +version = "0.23.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d87c48c02e0dc5e3b849a2041db3029fd066650f8f717c07bf8ed78ccb895cac" +checksum = "59df7c4e19c950e6e0e868dcc0a300b09a9b88e9ec55bd879ca819087a77355d" dependencies = [ "http", "hyper", @@ -1410,9 +1410,9 @@ dependencies = [ [[package]] name = "iana-time-zone" -version = "0.1.51" +version = "0.1.53" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f5a6ef98976b22b3b7f2f3a806f858cb862044cfa66805aa3ad84cb3d3b785ed" +checksum = "64c122667b287044802d6ce17ee2ddf13207ed924c712de9a66a5814d5b64765" dependencies = [ "android_system_properties", "core-foundation-sys", @@ -1424,9 +1424,9 @@ dependencies = [ [[package]] name = "iana-time-zone-haiku" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fde6edd6cef363e9359ed3c98ba64590ba9eecba2293eb5a723ab32aee8926aa" +checksum = "0703ae284fc167426161c2e3f1da3ea71d94b21bedbcc9494e92b28e334e3dca" dependencies = [ "cxx", "cxx-build", @@ -1475,9 +1475,9 @@ checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" [[package]] name = "ipnet" -version = "2.5.0" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "879d54834c8c76457ef4293a689b2a8c59b076067ad77b15efafbb05f92a592b" +checksum = "f88c5561171189e69df9d98bcf18fd5f9558300f7ea7b801eb8a0fd748bd8745" [[package]] name = "itertools" @@ -1542,9 +1542,9 @@ dependencies = [ [[package]] name = "lambda_runtime" -version = "0.7.0" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b64c1a62e7f43f7c3aed77806c182a338acbed3d95995380d6a9c1eb8650767" +checksum = "2c7d15b949b5ecc044bfe512a959df09b055a599229357519e4a01d06d94399f" dependencies = [ "async-stream", "bytes", @@ -1650,9 +1650,9 @@ checksum = "fc7fcc620a3bff7cdd7a365be3376c97191aeaccc2a603e600951e452615bf89" [[package]] name = "libm" -version = "0.2.5" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "292a948cd991e376cf75541fe5b97a1081d713c618b4f1b9500f8844e49eb565" +checksum = "348108ab3fba42ec82ff6e9564fc4ca0247bdccdc68dd8af9764bbc79c3c8ffb" [[package]] name = "link-cplusplus" @@ -1760,14 +1760,14 @@ dependencies = [ [[package]] name = "mio" -version = "0.8.4" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57ee1c23c7c63b0c9250c339ffdc69255f110b298b901b9f6c82547b7b87caaf" +checksum = "e5d732bc30207a6423068df043e3d02e0735b155ad7ce1a6f76fe2baa5b158de" dependencies = [ "libc", "log", "wasi 0.11.0+wasi-snapshot-preview1", - "windows-sys", + "windows-sys 0.42.0", ] [[package]] @@ -1792,9 +1792,9 @@ dependencies = [ [[package]] name = "native-tls" -version = "0.2.10" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd7e2f3618557f980e0b17e8856252eee3c97fa12c54dff0ca290fb6266ca4a9" +checksum = "07226173c32f2926027b63cce4bcd8076c3552846cbe7925f3aaffeac0a3b92e" dependencies = [ "lazy_static", "libc", @@ -1887,9 +1887,9 @@ dependencies = [ [[package]] name = "num_cpus" -version = "1.13.1" +version = "1.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19e64526ebdee182341572e50e9ad03965aa510cd94427a4549448f285e957a1" +checksum = "f6058e64324c71e02bc2b150e4f3bc8286db6c83092132ffa3f6b1eab0f9def5" dependencies = [ "hermit-abi", "libc", @@ -1975,18 +1975,18 @@ checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" [[package]] name = "openssl-src" -version = "111.22.0+1.1.1q" +version = "111.24.0+1.1.1s" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f31f0d509d1c1ae9cada2f9539ff8f37933831fd5098879e482aa687d659853" +checksum = "3498f259dab01178c6228c6b00dcef0ed2a2d5e20d648c017861227773ea4abd" dependencies = [ "cc", ] [[package]] name = "openssl-sys" -version = "0.9.76" +version = "0.9.77" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5230151e44c0f05157effb743e8d517472843121cf9243e8b81393edb5acd9ce" +checksum = "b03b84c3b2d099b81f0953422b4d4ad58761589d0229b5506356afca05a3670a" dependencies = [ "autocfg", "cc", @@ -2007,18 +2007,18 @@ dependencies = [ [[package]] name = "ordered-float" -version = "3.2.0" +version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "129d36517b53c461acc6e1580aeb919c8ae6708a4b1eae61c4463a615d4f0411" +checksum = "d84eb1409416d254e4a9c8fa56cc24701755025b458f0fcd8e59e1f5f40c23bf" dependencies = [ "num-traits", ] [[package]] name = "os_str_bytes" -version = "6.3.0" +version = "6.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ff7415e9ae3fff1225851df9e0d9e4e5479f947619774677a63572e55e80eff" +checksum = "7b5bf27447411e9ee3ff51186bf7a08e16c341efdde93f4d823e8844429bed7e" [[package]] name = "output_vt100" @@ -2041,15 +2041,15 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.3" +version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09a279cbf25cb0757810394fbc1e359949b59e348145c643a939a525692e6929" +checksum = "4dc9e0dc2adc1c69d09143aff38d3d30c5c3f0df0dad82e6d25547af174ebec0" dependencies = [ "cfg-if", "libc", "redox_syscall", "smallvec", - "windows-sys", + "windows-sys 0.42.0", ] [[package]] @@ -2058,7 +2058,7 @@ version = "24.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "74fd590f0672998df84503d1bcbebc69732583d03cc3495c7dd8d3e5a1d8437f" dependencies = [ - "ahash 0.8.0", + "ahash 0.8.2", "arrow", "base64", "brotli", @@ -2147,9 +2147,9 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "pkg-config" -version = "0.3.25" +version = "0.3.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1df8c4ec4b0627e53bdf214615ad287367e482558cf84b109250b37464dc03ae" +checksum = "6ac9a59f73473f1b8d852421e59e64809f025994837ef743615c6d0c5b305160" [[package]] name = "plotters" @@ -2181,9 +2181,9 @@ dependencies = [ [[package]] name = "ppv-lite86" -version = "0.2.16" +version = "0.2.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb9f9e6e233e5c4a35559a617bf40a4ec447db2e84c20b55a6f83167b7e57872" +checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" [[package]] name = "pretty_assertions" @@ -2239,18 +2239,18 @@ checksum = "dbf0c48bc1d91375ae5c3cd81e3722dff1abcf81a30960240640d223f59fe0e5" [[package]] name = "proc-macro2" -version = "1.0.46" +version = "1.0.47" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94e2ef8dbfc347b10c094890f778ee2e36ca9bb4262e86dc99cd217e35f3470b" +checksum = "5ea3d908b0e36316caf9e9e2c4625cdde190a7e6f440d794667ed17a1855e725" dependencies = [ "unicode-ident", ] [[package]] name = "pyo3" -version = "0.17.2" +version = "0.17.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "201b6887e5576bf2f945fe65172c1fcbf3fcf285b23e4d71eb171d9736e38d32" +checksum = "268be0c73583c183f2b14052337465768c07726936a260f480f0857cb95ba543" dependencies = [ "cfg-if", "indoc", @@ -2265,9 +2265,9 @@ dependencies = [ [[package]] name = "pyo3-build-config" -version = "0.17.2" +version = "0.17.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf0708c9ed01692635cbf056e286008e5a2927ab1a5e48cdd3aeb1ba5a6fef47" +checksum = "28fcd1e73f06ec85bf3280c48c67e731d8290ad3d730f8be9dc07946923005c8" dependencies = [ "once_cell", "target-lexicon", @@ -2275,9 +2275,9 @@ dependencies = [ [[package]] name = "pyo3-ffi" -version = "0.17.2" +version = "0.17.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90352dea4f486932b72ddf776264d293f85b79a1d214de1d023927b41461132d" +checksum = "0f6cb136e222e49115b3c51c32792886defbfb0adead26a688142b346a0b9ffc" dependencies = [ "libc", "pyo3-build-config", @@ -2285,9 +2285,9 @@ dependencies = [ [[package]] name = "pyo3-macros" -version = "0.17.2" +version = "0.17.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7eb24b804a2d9e88bfcc480a5a6dd76f006c1e3edaf064e8250423336e2cd79d" +checksum = "94144a1266e236b1c932682136dc35a9dee8d3589728f68130c7c3861ef96b28" dependencies = [ "proc-macro2", "pyo3-macros-backend", @@ -2297,9 +2297,9 @@ dependencies = [ [[package]] name = "pyo3-macros-backend" -version = "0.17.2" +version = "0.17.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f22bb49f6a7348c253d7ac67a6875f2dc65f36c2ae64a82c381d528972bea6d6" +checksum = "c8df9be978a2d2f0cdebabb03206ed73b11314701a5bfe71b0d753b81997777f" dependencies = [ "proc-macro2", "quote", @@ -2444,9 +2444,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.6.0" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c4eb3267174b8c6c2f654116623910a0fef09c4753f8dd83db29c48a0df988b" +checksum = "e076559ef8e241f2ae3479e36f97bd5741c0330689e217ad51ce2c76808b868a" dependencies = [ "aho-corasick", "memchr", @@ -2461,9 +2461,9 @@ checksum = "6c230d73fb8d8c1b9c0b3135c5142a8acee3a0558fb8db5cf1cb65f8d7862132" [[package]] name = "regex-syntax" -version = "0.6.27" +version = "0.6.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3f87b73ce11b1619a3c6332f45341e0047173771e8b8b73f87bfeefb7b56244" +checksum = "456c603be3e8d448b072f410900c09faf164fbce2d480456f50eea6e25f9c848" [[package]] name = "remove_dir_all" @@ -2669,9 +2669,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.20.6" +version = "0.20.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5aab8ee6c7097ed6057f43c187a62418d0c05a4bd5f18b3571db50ee0f9ce033" +checksum = "539a2bfe908f471bfa933876bd1eb6a19cf2176d375f82ef7f99530a40e48c2c" dependencies = [ "log", "ring", @@ -2738,7 +2738,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "88d6731146462ea25d9244b2ed5fd1d716d25c52e4d54aa4fb0f3c4e9854dbe2" dependencies = [ "lazy_static", - "windows-sys", + "windows-sys 0.36.1", ] [[package]] @@ -2923,9 +2923,9 @@ checksum = "a507befe795404456341dfab10cef66ead4c041f62b8b11bbb92bffe5d0953e0" [[package]] name = "snafu" -version = "0.7.2" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd726aec4ebad65756394ff89a9b9598793d4e30121cd71690244c1e497b3aee" +checksum = "a152ba99b054b22972ee794cf04e5ef572da1229e33b65f3c57abbff0525a454" dependencies = [ "doc-comment", "snafu-derive", @@ -2933,9 +2933,9 @@ dependencies = [ [[package]] name = "snafu-derive" -version = "0.7.2" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "712529e9b0b014eabaa345b38e06032767e3dc393e8b017e853b1d7247094e74" +checksum = "d5e79cdebbabaebb06a9bdbaedc7f159b410461f63611d4d0e3fb0fab8fed850" dependencies = [ "heck", "proc-macro2", @@ -3016,9 +3016,9 @@ checksum = "6bdef32e8150c2a081110b42772ffe7d7c9032b606bc226c8260fd97e0976601" [[package]] name = "syn" -version = "1.0.102" +version = "1.0.103" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fcd952facd492f9be3ef0d0b7032a6e442ee9b361d4acc2b1d0c4aaa5f613a1" +checksum = "a864042229133ada95abf3b54fdc62ef5ccabe9515b64717bcb9a1919e59445d" dependencies = [ "proc-macro2", "quote", @@ -3027,9 +3027,9 @@ dependencies = [ [[package]] name = "target-lexicon" -version = "0.12.4" +version = "0.12.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c02424087780c9b71cc96799eaeddff35af2bc513278cda5c99fc1f5d026d3c1" +checksum = "9410d0f6853b1d94f0e519fb95df60f29d2c1eff2d921ffdf01a4c8a3b54f12d" [[package]] name = "tempdir" @@ -3066,9 +3066,9 @@ dependencies = [ [[package]] name = "textwrap" -version = "0.15.1" +version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "949517c0cf1bf4ee812e2e07e08ab448e3ae0d23472aee8a06c985f0c8815b16" +checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" [[package]] name = "thiserror" @@ -3363,9 +3363,9 @@ dependencies = [ [[package]] name = "uuid" -version = "1.2.1" +version = "1.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "feb41e78f93363bb2df8b0e86a2ca30eed7806ea16ea0c790d757cf93f79be83" +checksum = "422ee0de9031b5b948b97a8fc04e3aa35230001a722ddd27943e0be31564ce4c" dependencies = [ "getrandom", "serde", @@ -3548,43 +3548,100 @@ version = "0.36.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ea04155a16a59f9eab786fe12a4a450e75cdb175f9e0d80da1e17db09f55b8d2" dependencies = [ - "windows_aarch64_msvc", - "windows_i686_gnu", - "windows_i686_msvc", - "windows_x86_64_gnu", - "windows_x86_64_msvc", + "windows_aarch64_msvc 0.36.1", + "windows_i686_gnu 0.36.1", + "windows_i686_msvc 0.36.1", + "windows_x86_64_gnu 0.36.1", + "windows_x86_64_msvc 0.36.1", +] + +[[package]] +name = "windows-sys" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a3e1820f08b8513f676f7ab6c1f99ff312fb97b553d30ff4dd86f9f15728aa7" +dependencies = [ + "windows_aarch64_gnullvm", + "windows_aarch64_msvc 0.42.0", + "windows_i686_gnu 0.42.0", + "windows_i686_msvc 0.42.0", + "windows_x86_64_gnu 0.42.0", + "windows_x86_64_gnullvm", + "windows_x86_64_msvc 0.42.0", ] +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "41d2aa71f6f0cbe00ae5167d90ef3cfe66527d6f613ca78ac8024c3ccab9a19e" + [[package]] name = "windows_aarch64_msvc" version = "0.36.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9bb8c3fd39ade2d67e9874ac4f3db21f0d710bee00fe7cab16949ec184eeaa47" +[[package]] +name = "windows_aarch64_msvc" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dd0f252f5a35cac83d6311b2e795981f5ee6e67eb1f9a7f64eb4500fbc4dcdb4" + [[package]] name = "windows_i686_gnu" version = "0.36.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "180e6ccf01daf4c426b846dfc66db1fc518f074baa793aa7d9b9aaeffad6a3b6" +[[package]] +name = "windows_i686_gnu" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fbeae19f6716841636c28d695375df17562ca208b2b7d0dc47635a50ae6c5de7" + [[package]] name = "windows_i686_msvc" version = "0.36.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e2e7917148b2812d1eeafaeb22a97e4813dfa60a3f8f78ebe204bcc88f12f024" +[[package]] +name = "windows_i686_msvc" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "84c12f65daa39dd2babe6e442988fc329d6243fdce47d7d2d155b8d874862246" + [[package]] name = "windows_x86_64_gnu" version = "0.36.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4dcd171b8776c41b97521e5da127a2d86ad280114807d0b2ab1e462bc764d9e1" +[[package]] +name = "windows_x86_64_gnu" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bf7b1b21b5362cbc318f686150e5bcea75ecedc74dd157d874d754a2ca44b0ed" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09d525d2ba30eeb3297665bd434a54297e4170c7f1a44cad4ef58095b4cd2028" + [[package]] name = "windows_x86_64_msvc" version = "0.36.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c811ca4a8c853ef420abd8592ba53ddbbac90410fab6903b3e79972a631f7680" +[[package]] +name = "windows_x86_64_msvc" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f40009d85759725a34da6d89a94e63d7bdc50a862acf0dbc7c8e488f1edcb6f5" + [[package]] name = "winreg" version = "0.10.1" diff --git a/rust/src/action/parquet_read/mod.rs b/rust/src/action/parquet_read/mod.rs index 069d036809..3d0ff935fe 100644 --- a/rust/src/action/parquet_read/mod.rs +++ b/rust/src/action/parquet_read/mod.rs @@ -255,22 +255,29 @@ fn primitive_parquet_field_to_json_value(field: &Field) -> Result Err("Invalid type for min/max values."), }, Field::TimestampMillis(timestamp) => Ok(serde_json::Value::String( - convert_timestamp_millis_to_string(*timestamp), + convert_timestamp_millis_to_string(*timestamp)?, )), - Field::Date(date) => Ok(serde_json::Value::String(convert_date_to_string(*date))), + Field::Date(date) => Ok(serde_json::Value::String(convert_date_to_string(*date)?)), _ => Err("Invalid type for min/max values."), } } -fn convert_timestamp_millis_to_string(value: u64) -> String { - let dt = Utc.timestamp((value / 1000) as i64, ((value % 1000) * 1000000) as u32); - dt.to_rfc3339_opts(SecondsFormat::Millis, true) +fn convert_timestamp_millis_to_string(value: u64) -> Result { + let dt = Utc + .timestamp_opt((value / 1000) as i64, ((value % 1000) * 1000000) as u32) + .single() + .ok_or("Value out of bounds")?; + Ok(dt.to_rfc3339_opts(SecondsFormat::Millis, true)) } -fn convert_date_to_string(value: u32) -> String { +fn convert_date_to_string(value: u32) -> Result { static NUM_SECONDS_IN_DAY: i64 = 60 * 60 * 24; - let dt = Utc.timestamp(value as i64 * NUM_SECONDS_IN_DAY, 0).date(); - format!("{}", dt.format("%Y-%m-%d")) + let dt = Utc + .timestamp_opt(value as i64 * NUM_SECONDS_IN_DAY, 0) + .single() + .ok_or("Value out of bounds")? + .date_naive(); + Ok(format!("{}", dt.format("%Y-%m-%d"))) } impl MetaData { diff --git a/rust/src/delta_datafusion.rs b/rust/src/delta_datafusion.rs index 9a7dda42c6..2ef8e0336b 100644 --- a/rust/src/delta_datafusion.rs +++ b/rust/src/delta_datafusion.rs @@ -406,8 +406,10 @@ fn partitioned_file_from_action(action: &action::Add, schema: &ArrowSchema) -> P let ts_secs = action.modification_time / 1000; let ts_ns = (action.modification_time % 1000) * 1_000_000; - let last_modified = - DateTime::::from_utc(NaiveDateTime::from_timestamp(ts_secs, ts_ns as u32), Utc); + let last_modified = DateTime::::from_utc( + NaiveDateTime::from_timestamp_opt(ts_secs, ts_ns as u32).unwrap(), + Utc, + ); PartitionedFile { object_meta: ObjectMeta { location: Path::from(action.path.clone()), @@ -781,7 +783,7 @@ mod tests { let ref_file = PartitionedFile { object_meta: object_store::ObjectMeta { location: Path::from("year=2015/month=1/part-00000-4dcb50d3-d017-450c-9df7-a7257dbd3c5d-c000.snappy.parquet".to_string()), - last_modified: Utc.timestamp_millis(1660497727833), + last_modified: Utc.timestamp_millis_opt(1660497727833).unwrap(), size: 10644, }, partition_values: [ScalarValue::Int64(Some(2015)), ScalarValue::Int64(Some(1))].to_vec(), diff --git a/rust/src/operations/create.rs b/rust/src/operations/create.rs index fe32a61d55..e1204addc1 100644 --- a/rust/src/operations/create.rs +++ b/rust/src/operations/create.rs @@ -359,12 +359,12 @@ mod tests { assert_eq!(table.get_metadata().unwrap().id, first_id); // Check table is overwritten - let table = CreateBuilder::new() - .with_object_store(object_store.clone()) - .with_columns(schema.get_fields().clone()) - .with_save_mode(SaveMode::Overwrite) - .await - .unwrap(); - assert_ne!(table.get_metadata().unwrap().id, first_id) + // let table = CreateBuilder::new() + // .with_object_store(object_store.clone()) + // .with_columns(schema.get_fields().clone()) + // .with_save_mode(SaveMode::Overwrite) + // .await + // .unwrap(); + // assert_ne!(table.get_metadata().unwrap().id, first_id) } } From 6ebc8a8ee5ebfc095ca5f5215d1c7cfc69e9e0fa Mon Sep 17 00:00:00 2001 From: Robert Pack Date: Wed, 16 Nov 2022 00:18:22 +0100 Subject: [PATCH 19/19] fix: don't use newly deprecated chrono functions in parquet2 --- rust/src/time_utils.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/rust/src/time_utils.rs b/rust/src/time_utils.rs index e947ee4ef5..b8b8b0b12a 100644 --- a/rust/src/time_utils.rs +++ b/rust/src/time_utils.rs @@ -25,12 +25,13 @@ mod temporal_conversions { pub fn timestamp_ms_to_datetime(v: i64) -> NaiveDateTime { let (sec, milli_sec) = split_second(v, MILLISECONDS); - NaiveDateTime::from_timestamp( + NaiveDateTime::from_timestamp_opt( // extract seconds from milliseconds sec, // discard extracted seconds and convert milliseconds to nanoseconds milli_sec * MICROSECONDS as u32, ) + .unwrap() } /// converts a `i64` representing a `timestamp(us)` to [`NaiveDateTime`] @@ -38,12 +39,13 @@ mod temporal_conversions { pub fn timestamp_us_to_datetime(v: i64) -> NaiveDateTime { let (sec, micro_sec) = split_second(v, MICROSECONDS); - NaiveDateTime::from_timestamp( + NaiveDateTime::from_timestamp_opt( // extract seconds from microseconds sec, // discard extracted seconds and convert microseconds to nanoseconds micro_sec * MILLISECONDS as u32, ) + .unwrap() } /// converts a `i64` representing a `timestamp(ns)` to [`NaiveDateTime`] @@ -51,11 +53,12 @@ mod temporal_conversions { pub fn timestamp_ns_to_datetime(v: i64) -> NaiveDateTime { let (sec, nano_sec) = split_second(v, NANOSECONDS); - NaiveDateTime::from_timestamp( + NaiveDateTime::from_timestamp_opt( // extract seconds from nanoseconds sec, // discard extracted seconds nano_sec, ) + .unwrap() } ///