diff --git a/Cargo.toml b/Cargo.toml index 0b3862bd1f..f7dc8b4296 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -20,9 +20,11 @@ debug = "line-tables-only" [workspace.dependencies] # arrow arrow = { version = "47" } +arrow-arith = { version = "47" } arrow-array = { version = "47" } arrow-buffer = { version = "47" } arrow-cast = { version = "47" } +arrow-json = { version = "47" } arrow-ord = { version = "47" } arrow-row = { version = "47" } arrow-schema = { version = "47" } diff --git a/crates/deltalake-core/Cargo.toml b/crates/deltalake-core/Cargo.toml index 9fa259fa39..e14f8f6db4 100644 --- a/crates/deltalake-core/Cargo.toml +++ b/crates/deltalake-core/Cargo.toml @@ -20,9 +20,11 @@ features = ["azure", "datafusion", "gcs", "glue", "hdfs", "json", "python", "s3" [dependencies] # arrow arrow = { workspace = true, optional = true } +arrow-arith = { workspace = true, optional = true } arrow-array = { workspace = true, optional = true } arrow-buffer = { workspace = true, optional = true } arrow-cast = { workspace = true, optional = true } +arrow-json = { workspace = true, optional = true } arrow-ord = { workspace = true, optional = true } arrow-row = { workspace = true, optional = true } arrow-schema = { workspace = true, optional = true, features = ["serde"] } @@ -110,7 +112,6 @@ reqwest = { version = "0.11.18", default-features = false, features = [ # Datafusion dashmap = { version = "5", optional = true } - sqlparser = { version = "0.38", optional = true } # NOTE dependencies only for integration tests @@ -130,13 +131,16 @@ tempfile = "3" tokio = { version = "1", features = ["macros", "rt-multi-thread"] } utime = "0.3" hyper = { version = "0.14", features = ["server"] } +criterion = "0.5" [features] azure = ["object_store/azure"] arrow = [ "dep:arrow", + "arrow-arith", "arrow-array", "arrow-cast", + "arrow-json", "arrow-ord", "arrow-row", "arrow-schema", diff --git a/crates/deltalake-core/benches/read_checkpoint.rs b/crates/deltalake-core/benches/read_checkpoint.rs index 2ecbee661b..0db72c3e17 100644 --- a/crates/deltalake-core/benches/read_checkpoint.rs +++ b/crates/deltalake-core/benches/read_checkpoint.rs @@ -1,6 +1,6 @@ use criterion::{criterion_group, criterion_main, Criterion}; -use deltalake::table::state::DeltaTableState; -use deltalake::DeltaTableConfig; +use deltalake_core::table::state::DeltaTableState; +use deltalake_core::DeltaTableConfig; use std::fs::File; use std::io::Read; diff --git a/crates/deltalake-core/src/delta_datafusion/expr.rs b/crates/deltalake-core/src/delta_datafusion/expr.rs index e451484183..8a2c675add 100644 --- a/crates/deltalake-core/src/delta_datafusion/expr.rs +++ b/crates/deltalake-core/src/delta_datafusion/expr.rs @@ -516,7 +516,7 @@ mod test { &arrow_schema::DataType::Utf8, &table .state - .input_schema() + .arrow_schema(false) .unwrap() .as_ref() .to_owned() diff --git a/crates/deltalake-core/src/delta_datafusion/mod.rs b/crates/deltalake-core/src/delta_datafusion/mod.rs index 38bf135739..6ebc8d62e3 100644 --- a/crates/deltalake-core/src/delta_datafusion/mod.rs +++ b/crates/deltalake-core/src/delta_datafusion/mod.rs @@ -21,12 +21,10 @@ //! ``` use std::any::Any; -use std::collections::{HashMap, HashSet}; -use std::convert::TryFrom; -use std::fmt::{self, Debug}; +use std::collections::HashMap; +use std::fmt::Debug; use std::sync::Arc; -use arrow::array::ArrayRef; use arrow::compute::{cast_with_options, CastOptions}; use arrow::datatypes::DataType; use arrow::datatypes::{DataType as ArrowDataType, Schema as ArrowSchema, SchemaRef, TimeUnit}; @@ -36,28 +34,18 @@ use arrow_array::types::UInt16Type; use arrow_array::{DictionaryArray, StringArray}; use arrow_schema::Field; use async_trait::async_trait; -use chrono::{NaiveDateTime, TimeZone, Utc}; -use datafusion::datasource::file_format::{parquet::ParquetFormat, FileFormat}; -use datafusion::datasource::physical_plan::{ - wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileScanConfig, -}; use datafusion::datasource::provider::TableProviderFactory; -use datafusion::datasource::{listing::PartitionedFile, MemTable, TableProvider, TableType}; +use datafusion::datasource::{MemTable, TableProvider, TableType}; use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::optimizer::utils::conjunction; -use datafusion::physical_expr::PhysicalSortExpr; -use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::limit::LocalLimitExec; -use datafusion::physical_plan::{ - ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, - SendableRecordBatchStream, Statistics, -}; +use datafusion::physical_plan::{ExecutionPlan, Statistics}; use datafusion_common::scalar::ScalarValue; use datafusion_common::tree_node::{TreeNode, TreeNodeVisitor, VisitRecursion}; -use datafusion_common::{Column, DataFusionError, Result as DataFusionResult, ToDFSchema}; +use datafusion_common::{DataFusionError, Result as DataFusionResult, ToDFSchema}; use datafusion_expr::expr::{ScalarFunction, ScalarUDF}; use datafusion_expr::logical_plan::CreateExternalTable; use datafusion_expr::{col, Expr, Extension, LogicalPlan, TableProviderFilterPushDown, Volatility}; @@ -65,21 +53,24 @@ use datafusion_physical_expr::execution_props::ExecutionProps; use datafusion_physical_expr::{create_physical_expr, PhysicalExpr}; use datafusion_proto::logical_plan::LogicalExtensionCodec; use datafusion_proto::physical_plan::PhysicalExtensionCodec; -use object_store::ObjectMeta; use serde::{Deserialize, Serialize}; use url::Url; +pub(crate) use self::scan::DeltaScanBuilder; + +use self::pruning::DatafusionExt; +use self::scan::{DeltaScan, DeltaScanConfig, DeltaScanConfigBuilder}; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::{Add, DataType as DeltaDataType, Invariant, PrimitiveType}; +use crate::kernel::{Add, Invariant}; use crate::logstore::LogStoreRef; -use crate::protocol::{self}; -use crate::table::builder::ensure_table_uri; use crate::table::state::DeltaTableState; use crate::{open_table, open_table_with_storage_options, DeltaTable}; const PATH_COLUMN: &str = "__delta_rs_path"; pub mod expr; +pub mod pruning; +pub mod scan; impl From for DataFusionError { fn from(err: DeltaTableError) -> Self { @@ -105,256 +96,6 @@ impl From for DeltaTableError { } } -impl DeltaTableState { - /// Return statistics for Datafusion Table - pub fn datafusion_table_statistics(&self) -> Statistics { - let stats = self - .files() - .iter() - .try_fold( - Statistics { - num_rows: Some(0), - total_byte_size: Some(0), - column_statistics: Some(vec![ - ColumnStatistics { - null_count: Some(0), - max_value: None, - min_value: None, - distinct_count: None - }; - self.schema().unwrap().fields().len() - ]), - is_exact: true, - }, - |acc, action| { - let new_stats = action - .get_stats() - .unwrap_or_else(|_| Some(protocol::Stats::default()))?; - Some(Statistics { - num_rows: acc - .num_rows - .map(|rows| rows + new_stats.num_records as usize), - total_byte_size: acc - .total_byte_size - .map(|total_size| total_size + action.size as usize), - column_statistics: acc.column_statistics.map(|col_stats| { - self.schema() - .unwrap() - .fields() - .iter() - .zip(col_stats) - .map(|(field, stats)| { - let null_count = new_stats - .null_count - .get(field.name()) - .and_then(|x| { - let null_count_acc = stats.null_count?; - let null_count = x.as_value()? as usize; - Some(null_count_acc + null_count) - }) - .or(stats.null_count); - - let max_value = new_stats - .max_values - .get(field.name()) - .and_then(|x| { - let old_stats = stats.clone(); - let max_value = to_scalar_value(x.as_value()?); - - match (max_value, old_stats.max_value) { - (Some(max_value), Some(old_max_value)) => { - if left_larger_than_right( - old_max_value.clone(), - max_value.clone(), - )? { - Some(old_max_value) - } else { - Some(max_value) - } - } - (Some(max_value), None) => Some(max_value), - (None, old) => old, - } - }) - .or_else(|| stats.max_value.clone()); - - let min_value = new_stats - .min_values - .get(field.name()) - .and_then(|x| { - let old_stats = stats.clone(); - let min_value = to_scalar_value(x.as_value()?); - - match (min_value, old_stats.min_value) { - (Some(min_value), Some(old_min_value)) => { - if left_larger_than_right( - min_value.clone(), - old_min_value.clone(), - )? { - Some(old_min_value) - } else { - Some(min_value) - } - } - (Some(min_value), None) => Some(min_value), - (None, old) => old, - } - }) - .or_else(|| stats.min_value.clone()); - - ColumnStatistics { - null_count, - max_value, - min_value, - distinct_count: None, // TODO: distinct - } - }) - .collect() - }), - is_exact: true, - }) - }, - ) - .unwrap_or_default(); - - // Convert column max/min scalar values to correct types based on arrow types. - Statistics { - is_exact: true, - num_rows: stats.num_rows, - total_byte_size: stats.total_byte_size, - column_statistics: stats.column_statistics.map(|col_stats| { - let fields = self.schema().unwrap().fields(); - col_stats - .iter() - .zip(fields) - .map(|(col_states, field)| { - let dt = self - .arrow_schema() - .unwrap() - .field_with_name(field.name()) - .unwrap() - .data_type() - .clone(); - ColumnStatistics { - null_count: col_states.null_count, - max_value: col_states - .max_value - .as_ref() - .and_then(|scalar| correct_scalar_value_type(scalar.clone(), &dt)), - min_value: col_states - .min_value - .as_ref() - .and_then(|scalar| correct_scalar_value_type(scalar.clone(), &dt)), - distinct_count: col_states.distinct_count, - } - }) - .collect() - }), - } - } -} - -// TODO: Collapse with operations/transaction/state.rs method of same name -fn get_prune_stats(table: &DeltaTable, column: &Column, get_max: bool) -> Option { - let field = table - .get_schema() - .ok() - .map(|s| s.field_with_name(&column.name).ok())??; - - // See issue 1214. Binary type does not support natural order which is required for Datafusion to prune - if let DeltaDataType::Primitive(PrimitiveType::Binary) = &field.data_type() { - return None; - } - - let data_type = field.data_type().try_into().ok()?; - let partition_columns = &table.get_metadata().ok()?.partition_columns; - - let values = table.get_state().files().iter().map(|add| { - if partition_columns.contains(&column.name) { - let value = add.partition_values.get(&column.name).unwrap(); - let value = match value { - Some(v) => serde_json::Value::String(v.to_string()), - None => serde_json::Value::Null, - }; - to_correct_scalar_value(&value, &data_type).unwrap_or( - get_null_of_arrow_type(&data_type).expect("Could not determine null type"), - ) - } else if let Ok(Some(statistics)) = add.get_stats() { - let values = if get_max { - statistics.max_values - } else { - statistics.min_values - }; - - values - .get(&column.name) - .and_then(|f| to_correct_scalar_value(f.as_value()?, &data_type)) - .unwrap_or( - get_null_of_arrow_type(&data_type).expect("Could not determine null type"), - ) - } else { - // No statistics available - get_null_of_arrow_type(&data_type).expect("Could not determine null type") - } - }); - ScalarValue::iter_to_array(values).ok() -} - -impl PruningStatistics for DeltaTable { - /// return the minimum values for the named column, if known. - /// Note: the returned array must contain `num_containers()` rows - fn min_values(&self, column: &Column) -> Option { - get_prune_stats(self, column, false) - } - - /// return the maximum values for the named column, if known. - /// Note: the returned array must contain `num_containers()` rows. - fn max_values(&self, column: &Column) -> Option { - get_prune_stats(self, column, true) - } - - /// return the number of containers (e.g. row groups) being - /// pruned with these statistics - fn num_containers(&self) -> usize { - self.get_state().files().len() - } - - /// return the number of null values for the named column as an - /// `Option`. - /// - /// Note: the returned array must contain `num_containers()` rows. - fn null_counts(&self, column: &Column) -> Option { - let partition_columns = &self.get_metadata().ok()?.partition_columns; - - let values = self.get_state().files().iter().map(|add| { - if let Ok(Some(statistics)) = add.get_stats() { - if partition_columns.contains(&column.name) { - let value = add.partition_values.get(&column.name).unwrap(); - match value { - Some(_) => ScalarValue::UInt64(Some(0)), - None => ScalarValue::UInt64(Some(statistics.num_records as u64)), - } - } else { - statistics - .null_count - .get(&column.name) - .map(|f| ScalarValue::UInt64(f.as_value().map(|val| val as u64))) - .unwrap_or(ScalarValue::UInt64(None)) - } - } else if partition_columns.contains(&column.name) { - let value = add.partition_values.get(&column.name).unwrap(); - match value { - Some(_) => ScalarValue::UInt64(Some(0)), - None => ScalarValue::UInt64(None), - } - } else { - ScalarValue::UInt64(None) - } - }); - ScalarValue::iter_to_array(values).ok() - } -} - // each delta table must register a specific object store, since paths are internally // handled relative to the table root. pub(crate) fn register_store(store: LogStoreRef, env: Arc) { @@ -364,10 +105,10 @@ pub(crate) fn register_store(store: LogStoreRef, env: Arc) { } pub(crate) fn logical_schema( - snapshot: &DeltaTableState, + snapshot: &dyn DatafusionExt, scan_config: &DeltaScanConfig, ) -> DeltaResult { - let input_schema = snapshot.input_schema()?; + let input_schema = snapshot.arrow_schema(false)?; let mut fields = Vec::new(); for field in input_schema.fields.iter() { fields.push(field.to_owned()); @@ -384,277 +125,6 @@ pub(crate) fn logical_schema( Ok(Arc::new(ArrowSchema::new(fields))) } -#[derive(Debug, Clone, Default)] -/// Used to specify if additional metadata columns are exposed to the user -pub struct DeltaScanConfigBuilder { - /// Include the source path for each record. The name of this column is determine by `file_column_name` - include_file_column: bool, - /// Column name that contains the source path. - /// - /// If include_file_column is true and the name is None then it will be auto-generated - /// Otherwise the user provided name will be used - file_column_name: Option, -} - -impl DeltaScanConfigBuilder { - /// Construct a new instance of `DeltaScanConfigBuilder` - pub fn new() -> Self { - Self::default() - } - - /// Indicate that a column containing a records file path is included. - /// Column name is generated and can be determined once this Config is built - pub fn with_file_column(mut self, include: bool) -> Self { - self.include_file_column = include; - self.file_column_name = None; - self - } - - /// Indicate that a column containing a records file path is included and column name is user defined. - pub fn with_file_column_name(mut self, name: &S) -> Self { - self.file_column_name = Some(name.to_string()); - self.include_file_column = true; - self - } - - /// Build a DeltaScanConfig and ensure no column name conflicts occur during downstream processing - pub fn build(&self, snapshot: &DeltaTableState) -> DeltaResult { - let input_schema = snapshot.input_schema()?; - let mut file_column_name = None; - let mut column_names: HashSet<&String> = HashSet::new(); - for field in input_schema.fields.iter() { - column_names.insert(field.name()); - } - - if self.include_file_column { - match &self.file_column_name { - Some(name) => { - if column_names.contains(name) { - return Err(DeltaTableError::Generic(format!( - "Unable to add file path column since column with name {} exits", - name - ))); - } - - file_column_name = Some(name.to_owned()) - } - None => { - let prefix = PATH_COLUMN; - let mut idx = 0; - let mut name = prefix.to_owned(); - - while column_names.contains(&name) { - idx += 1; - name = format!("{}_{}", prefix, idx); - } - - file_column_name = Some(name); - } - } - } - - Ok(DeltaScanConfig { file_column_name }) - } -} - -#[derive(Debug, Clone, Default, Serialize, Deserialize)] -/// Include additional metadata columns during a [`DeltaScan`] -pub struct DeltaScanConfig { - /// Include the source path for each record - pub file_column_name: Option, -} - -#[derive(Debug)] -pub(crate) struct DeltaScanBuilder<'a> { - snapshot: &'a DeltaTableState, - log_store: LogStoreRef, - filter: Option, - state: &'a SessionState, - projection: Option<&'a Vec>, - limit: Option, - files: Option<&'a [Add]>, - config: DeltaScanConfig, - schema: Option, -} - -impl<'a> DeltaScanBuilder<'a> { - pub fn new( - snapshot: &'a DeltaTableState, - log_store: LogStoreRef, - state: &'a SessionState, - ) -> Self { - DeltaScanBuilder { - snapshot, - log_store, - filter: None, - state, - files: None, - projection: None, - limit: None, - config: DeltaScanConfig::default(), - schema: None, - } - } - - pub fn with_filter(mut self, filter: Option) -> Self { - self.filter = filter; - self - } - - pub fn with_files(mut self, files: &'a [Add]) -> Self { - self.files = Some(files); - self - } - - pub fn with_projection(mut self, projection: Option<&'a Vec>) -> Self { - self.projection = projection; - self - } - - pub fn with_limit(mut self, limit: Option) -> Self { - self.limit = limit; - self - } - - pub fn with_scan_config(mut self, config: DeltaScanConfig) -> Self { - self.config = config; - self - } - - pub fn with_schema(mut self, schema: SchemaRef) -> Self { - self.schema = Some(schema); - self - } - - pub async fn build(self) -> DeltaResult { - let config = self.config; - let schema = match self.schema { - Some(schema) => schema, - None => { - self.snapshot - .physical_arrow_schema(self.log_store.object_store()) - .await? - } - }; - let logical_schema = logical_schema(self.snapshot, &config)?; - - let logical_schema = if let Some(used_columns) = self.projection { - let mut fields = vec![]; - for idx in used_columns { - fields.push(logical_schema.field(*idx).to_owned()); - } - Arc::new(ArrowSchema::new(fields)) - } else { - logical_schema - }; - - let logical_filter = self - .filter - .map(|expr| logical_expr_to_physical_expr(&expr, &logical_schema)); - - // Perform Pruning of files to scan - let files = match self.files { - Some(files) => files.to_owned(), - None => { - if let Some(predicate) = &logical_filter { - let pruning_predicate = - PruningPredicate::try_new(predicate.clone(), logical_schema.clone())?; - let files_to_prune = pruning_predicate.prune(self.snapshot)?; - self.snapshot - .files() - .iter() - .zip(files_to_prune.into_iter()) - .filter_map( - |(action, keep)| { - if keep { - Some(action.to_owned()) - } else { - None - } - }, - ) - .collect() - } else { - self.snapshot.files().to_owned() - } - } - }; - - // TODO we group files together by their partition values. If the table is partitioned - // and partitions are somewhat evenly distributed, probably not the worst choice ... - // However we may want to do some additional balancing in case we are far off from the above. - let mut file_groups: HashMap, Vec> = HashMap::new(); - - let table_partition_cols = &self - .snapshot - .current_metadata() - .ok_or(DeltaTableError::NoMetadata)? - .partition_columns; - - for action in files.iter() { - let mut part = partitioned_file_from_action(action, table_partition_cols, &schema); - - if config.file_column_name.is_some() { - part.partition_values - .push(wrap_partition_value_in_dict(ScalarValue::Utf8(Some( - action.path.clone(), - )))); - } - - file_groups - .entry(part.partition_values.clone()) - .or_default() - .push(part); - } - - let file_schema = Arc::new(ArrowSchema::new( - schema - .fields() - .iter() - .filter(|f| !table_partition_cols.contains(f.name())) - .cloned() - .collect::>(), - )); - - let mut table_partition_cols = table_partition_cols - .iter() - .map(|c| Ok((c.to_owned(), schema.field_with_name(c)?.data_type().clone()))) - .collect::, ArrowError>>()?; - - if let Some(file_column_name) = &config.file_column_name { - table_partition_cols.push(( - file_column_name.clone(), - wrap_partition_type_in_dict(DataType::Utf8), - )); - } - - let scan = ParquetFormat::new() - .create_physical_plan( - self.state, - FileScanConfig { - object_store_url: self.log_store.object_store_url(), - file_schema, - file_groups: file_groups.into_values().collect(), - statistics: self.snapshot.datafusion_table_statistics(), - projection: self.projection.cloned(), - limit: self.limit, - table_partition_cols, - output_ordering: vec![], - infinite_source: false, - }, - logical_filter.as_ref(), - ) - .await?; - - Ok(DeltaScan { - table_uri: ensure_table_uri(self.log_store.root_uri())?.as_str().into(), - parquet_scan: scan, - config, - logical_schema, - }) - } -} - #[async_trait] impl TableProvider for DeltaTable { fn as_any(&self) -> &dyn Any { @@ -662,7 +132,7 @@ impl TableProvider for DeltaTable { } fn schema(&self) -> Arc { - self.state.arrow_schema().unwrap() + self.state.arrow_schema(true).unwrap() } fn table_type(&self) -> TableType { @@ -788,74 +258,6 @@ impl TableProvider for DeltaTableProvider { } } -// TODO: this will likely also need to perform column mapping later when we support reader protocol v2 -/// A wrapper for parquet scans -#[derive(Debug)] -pub struct DeltaScan { - /// The URL of the ObjectStore root - pub table_uri: String, - /// Column that contains an index that maps to the original metadata Add - pub config: DeltaScanConfig, - /// The parquet scan to wrap - pub parquet_scan: Arc, - /// The schema of the table to be used when evaluating expressions - pub logical_schema: Arc, -} - -#[derive(Debug, Serialize, Deserialize)] -struct DeltaScanWire { - pub table_uri: String, - pub config: DeltaScanConfig, - pub logical_schema: Arc, -} - -impl DisplayAs for DeltaScan { - fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> std::fmt::Result { - write!(f, "DeltaScan") - } -} - -impl ExecutionPlan for DeltaScan { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.parquet_scan.schema() - } - - fn output_partitioning(&self) -> Partitioning { - self.parquet_scan.output_partitioning() - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.parquet_scan.output_ordering() - } - - fn children(&self) -> Vec> { - vec![self.parquet_scan.clone()] - } - - fn with_new_children( - self: Arc, - children: Vec>, - ) -> DataFusionResult> { - ExecutionPlan::with_new_children(self.parquet_scan.clone(), children) - } - - fn execute( - &self, - partition: usize, - context: Arc, - ) -> DataFusionResult { - self.parquet_scan.execute(partition, context) - } - - fn statistics(&self) -> Statistics { - self.parquet_scan.statistics() - } -} - pub(crate) fn get_null_of_arrow_type(t: &ArrowDataType) -> DeltaResult { match t { ArrowDataType::Null => Ok(ScalarValue::Null), @@ -917,50 +319,6 @@ pub(crate) fn get_null_of_arrow_type(t: &ArrowDataType) -> DeltaResult PartitionedFile { - let partition_values = partition_columns - .iter() - .map(|part| { - action - .partition_values - .get(part) - .map(|val| { - schema - .field_with_name(part) - .map(|field| match val { - Some(value) => to_correct_scalar_value( - &serde_json::Value::String(value.to_string()), - field.data_type(), - ) - .unwrap_or(ScalarValue::Null), - None => get_null_of_arrow_type(field.data_type()) - .unwrap_or(ScalarValue::Null), - }) - .unwrap_or(ScalarValue::Null) - }) - .unwrap_or(ScalarValue::Null) - }) - .collect::>(); - - let ts_secs = action.modification_time / 1000; - let ts_ns = (action.modification_time % 1000) * 1_000_000; - let last_modified = - Utc.from_utc_datetime(&NaiveDateTime::from_timestamp_opt(ts_secs, ts_ns as u32).unwrap()); - PartitionedFile { - object_meta: ObjectMeta { - last_modified, - ..action.try_into().unwrap() - }, - partition_values, - range: None, - extensions: None, - } -} - fn to_scalar_value(stat_val: &serde_json::Value) -> Option { match stat_val { serde_json::Value::Bool(val) => Some(ScalarValue::from(*val)), @@ -1031,83 +389,6 @@ pub(crate) fn to_correct_scalar_value( } } -fn correct_scalar_value_type(value: ScalarValue, field_dt: &ArrowDataType) -> Option { - match field_dt { - ArrowDataType::Int64 => { - let raw_value = i64::try_from(value).ok()?; - Some(ScalarValue::from(raw_value)) - } - ArrowDataType::Int32 => { - let raw_value = i64::try_from(value).ok()? as i32; - Some(ScalarValue::from(raw_value)) - } - ArrowDataType::Int16 => { - let raw_value = i64::try_from(value).ok()? as i16; - Some(ScalarValue::from(raw_value)) - } - ArrowDataType::Int8 => { - let raw_value = i64::try_from(value).ok()? as i8; - Some(ScalarValue::from(raw_value)) - } - ArrowDataType::Float32 => { - let raw_value = f64::try_from(value).ok()? as f32; - Some(ScalarValue::from(raw_value)) - } - ArrowDataType::Float64 => { - let raw_value = f64::try_from(value).ok()?; - Some(ScalarValue::from(raw_value)) - } - ArrowDataType::Utf8 => match value { - ScalarValue::Utf8(val) => Some(ScalarValue::Utf8(val)), - _ => None, - }, - ArrowDataType::LargeUtf8 => match value { - ScalarValue::Utf8(val) => Some(ScalarValue::LargeUtf8(val)), - _ => None, - }, - ArrowDataType::Boolean => { - let raw_value = bool::try_from(value).ok()?; - Some(ScalarValue::from(raw_value)) - } - ArrowDataType::Decimal128(_, _) => { - let raw_value = f64::try_from(value).ok()?; - Some(ScalarValue::from(raw_value)) - } - ArrowDataType::Decimal256(_, _) => { - let raw_value = f64::try_from(value).ok()?; - Some(ScalarValue::from(raw_value)) - } - ArrowDataType::Date32 => { - let raw_value = i64::try_from(value).ok()? as i32; - Some(ScalarValue::Date32(Some(raw_value))) - } - ArrowDataType::Date64 => { - let raw_value = i64::try_from(value).ok()?; - Some(ScalarValue::Date64(Some(raw_value))) - } - ArrowDataType::Timestamp(TimeUnit::Nanosecond, None) => { - let raw_value = i64::try_from(value).ok()?; - Some(ScalarValue::TimestampNanosecond(Some(raw_value), None)) - } - ArrowDataType::Timestamp(TimeUnit::Microsecond, None) => { - let raw_value = i64::try_from(value).ok()?; - Some(ScalarValue::TimestampMicrosecond(Some(raw_value), None)) - } - ArrowDataType::Timestamp(TimeUnit::Millisecond, None) => { - let raw_value = i64::try_from(value).ok()?; - Some(ScalarValue::TimestampMillisecond(Some(raw_value), None)) - } - _ => { - log::error!( - "Scalar value of arrow type unimplemented for {:?} and {:?}", - value, - field_dt - ); - None - } - } -} - fn left_larger_than_right(left: ScalarValue, right: ScalarValue) -> Option { match (&left, &right) { (ScalarValue::Float64(Some(l)), ScalarValue::Float64(Some(r))) => Some(l > r), @@ -1206,6 +487,13 @@ impl DeltaDataChecker { } } +#[derive(Debug, Serialize, Deserialize)] +struct DeltaScanWire { + pub table_uri: String, + pub config: DeltaScanConfig, + pub logical_schema: Arc, +} + /// A codec for deltalake physical plans #[derive(Debug)] pub struct DeltaPhysicalCodec {} @@ -1626,12 +914,10 @@ mod tests { use crate::writer::test_utils::get_delta_schema; use arrow::array::StructArray; use arrow::datatypes::{DataType, Field, Schema}; - use chrono::{TimeZone, Utc}; use datafusion::assert_batches_sorted_eq; use datafusion::physical_plan::empty::EmptyExec; use datafusion_proto::physical_plan::AsExecutionPlan; use datafusion_proto::protobuf; - use object_store::path::Path; use serde_json::json; use std::ops::Deref; @@ -1781,46 +1067,6 @@ mod tests { } } - #[test] - fn test_partitioned_file_from_action() { - let mut partition_values = std::collections::HashMap::new(); - partition_values.insert("month".to_string(), Some("1".to_string())); - partition_values.insert("year".to_string(), Some("2015".to_string())); - let action = Add { - path: "year=2015/month=1/part-00000-4dcb50d3-d017-450c-9df7-a7257dbd3c5d-c000.snappy.parquet".to_string(), - size: 10644, - partition_values, - modification_time: 1660497727833, - partition_values_parsed: None, - data_change: true, - stats: None, - deletion_vector: None, - stats_parsed: None, - tags: None, - base_row_id: None, - default_row_commit_version: None, - }; - let schema = ArrowSchema::new(vec![ - Field::new("year", ArrowDataType::Int64, true), - Field::new("month", ArrowDataType::Int64, true), - ]); - - let part_columns = vec!["year".to_string(), "month".to_string()]; - let file = partitioned_file_from_action(&action, &part_columns, &schema); - 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_opt(1660497727833).unwrap(), - size: 10644, - e_tag: None - }, - partition_values: [ScalarValue::Int64(Some(2015)), ScalarValue::Int64(Some(1))].to_vec(), - range: None, - extensions: None, - }; - assert_eq!(file.partition_values, ref_file.partition_values) - } - #[tokio::test] async fn test_enforce_invariants() { let schema = Arc::new(Schema::new(vec![ diff --git a/crates/deltalake-core/src/delta_datafusion/pruning.rs b/crates/deltalake-core/src/delta_datafusion/pruning.rs new file mode 100644 index 0000000000..161bbe8df5 --- /dev/null +++ b/crates/deltalake-core/src/delta_datafusion/pruning.rs @@ -0,0 +1,684 @@ +//! Pruning statistics for Datafusion +use std::convert::TryFrom; +use std::sync::Arc; + +use arrow_array::ArrayRef; +use arrow_schema::{ + DataType as ArrowDataType, Field as ArrowField, Schema as ArrowSchema, + SchemaRef as ArrowSchemaRef, TimeUnit, +}; +use datafusion::datasource::physical_plan::wrap_partition_type_in_dict; +use datafusion::execution::context::SessionState; +use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; +use datafusion::physical_plan::{ColumnStatistics, Statistics}; +use datafusion_common::scalar::ScalarValue; +use datafusion_common::{Column, DFSchema}; +use datafusion_expr::Expr; +use object_store::ObjectStore; +use parquet::arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStreamBuilder}; + +use super::expr::parse_predicate_expression; +use super::{ + get_null_of_arrow_type, left_larger_than_right, logical_expr_to_physical_expr, + to_correct_scalar_value, to_scalar_value, +}; +use crate::errors::DeltaResult; +use crate::kernel::snapshot::Snapshot; +use crate::kernel::Add; +use crate::protocol::Stats; +use crate::table::state::DeltaTableState; + +/// Extension trait to implement datafusion helpers for DeltaTable +#[async_trait::async_trait] +pub trait DatafusionExt: Send + Sync + 'static { + /// Get the table schema as an [`ArrowSchemaRef`] + fn arrow_schema(&self, wrap_partitions: bool) -> DeltaResult; + + /// Return statistics for Datafusion Table + fn datafusion_table_statistics(&self) -> Statistics; + + /// Parse an expression string into a datafusion [`Expr`] + fn parse_predicate_expression(&self, expr: &str, df_state: &SessionState) -> DeltaResult { + let schema = DFSchema::try_from(self.arrow_schema(true)?.as_ref().to_owned())?; + parse_predicate_expression(&schema, expr, df_state) + } +} + +/// Get the physical table schema. +/// +/// This will construct a schema derived from the parquet schema of the latest data file, +/// and fields for partition columns from the schema defined in table meta data. +pub async fn physical_arrow_schema( + snapshot: &T, + object_store: Arc, +) -> DeltaResult { + if let Some(add) = snapshot.files()?.max_by_key(|obj| obj.modification_time) { + let file_meta = add.try_into()?; + let file_reader = ParquetObjectReader::new(object_store, file_meta); + let file_schema = ParquetRecordBatchStreamBuilder::new(file_reader) + .await? + .build()? + .schema() + .clone(); + + let table_schema = Arc::new(ArrowSchema::new( + snapshot + .arrow_schema(true)? + .fields + .clone() + .into_iter() + .map(|field| { + // field is an &Arc + let owned_field: ArrowField = field.as_ref().clone(); + file_schema + .field_with_name(field.name()) + // yielded with &Field + .cloned() + .unwrap_or(owned_field) + }) + .collect::>(), + )); + + Ok(table_schema) + } else { + snapshot.arrow_schema(true) + } +} + +impl DatafusionExt for S { + fn arrow_schema(&self, wrap_partitions: bool) -> DeltaResult { + let meta = self.metadata()?; + let fields = meta + .schema()? + .fields() + .iter() + .filter(|f| !meta.partition_columns.contains(&f.name().to_string())) + .map(|f| f.try_into()) + .chain( + meta.schema()? + .fields() + .iter() + .filter(|f| meta.partition_columns.contains(&f.name().to_string())) + .map(|f| { + let field = ArrowField::try_from(f)?; + let corrected = if wrap_partitions { + match field.data_type() { + // Only dictionary-encode types that may be large + // // https://github.com/apache/arrow-datafusion/pull/5545 + ArrowDataType::Utf8 + | ArrowDataType::LargeUtf8 + | ArrowDataType::Binary + | ArrowDataType::LargeBinary => { + wrap_partition_type_in_dict(field.data_type().clone()) + } + _ => field.data_type().clone(), + } + } else { + field.data_type().clone() + }; + Ok(field.with_data_type(corrected)) + }), + ) + .collect::, _>>()?; + + Ok(Arc::new(ArrowSchema::new(fields))) + } + + /// Return statistics for Datafusion Table + fn datafusion_table_statistics(&self) -> Statistics { + let stats = self + .files() + .unwrap() + .try_fold( + Statistics { + num_rows: Some(0), + total_byte_size: Some(0), + column_statistics: Some(vec![ + ColumnStatistics { + null_count: Some(0), + max_value: None, + min_value: None, + distinct_count: None + }; + self.schema().unwrap().fields().len() + ]), + is_exact: true, + }, + |acc, action| { + let new_stats = action + .get_stats() + .unwrap_or_else(|_| Some(Stats::default()))?; + Some(Statistics { + num_rows: acc + .num_rows + .map(|rows| rows + new_stats.num_records as usize), + total_byte_size: acc + .total_byte_size + .map(|total_size| total_size + action.size as usize), + column_statistics: acc.column_statistics.map(|col_stats| { + self.schema() + .unwrap() + .fields() + .iter() + .zip(col_stats) + .map(|(field, stats)| { + let null_count = new_stats + .null_count + .get(field.name()) + .and_then(|x| { + let null_count_acc = stats.null_count?; + let null_count = x.as_value()? as usize; + Some(null_count_acc + null_count) + }) + .or(stats.null_count); + + let max_value = new_stats + .max_values + .get(field.name()) + .and_then(|x| { + let old_stats = stats.clone(); + let max_value = to_scalar_value(x.as_value()?); + + match (max_value, old_stats.max_value) { + (Some(max_value), Some(old_max_value)) => { + if left_larger_than_right( + old_max_value.clone(), + max_value.clone(), + )? { + Some(old_max_value) + } else { + Some(max_value) + } + } + (Some(max_value), None) => Some(max_value), + (None, old) => old, + } + }) + .or_else(|| stats.max_value.clone()); + + let min_value = new_stats + .min_values + .get(field.name()) + .and_then(|x| { + let old_stats = stats.clone(); + let min_value = to_scalar_value(x.as_value()?); + + match (min_value, old_stats.min_value) { + (Some(min_value), Some(old_min_value)) => { + if left_larger_than_right( + min_value.clone(), + old_min_value.clone(), + )? { + Some(old_min_value) + } else { + Some(min_value) + } + } + (Some(min_value), None) => Some(min_value), + (None, old) => old, + } + }) + .or_else(|| stats.min_value.clone()); + + ColumnStatistics { + null_count, + max_value, + min_value, + distinct_count: None, // TODO: distinct + } + }) + .collect() + }), + is_exact: true, + }) + }, + ) + .unwrap_or_default(); + + // Convert column max/min scalar values to correct types based on arrow types. + Statistics { + is_exact: true, + num_rows: stats.num_rows, + total_byte_size: stats.total_byte_size, + column_statistics: stats.column_statistics.map(|col_stats| { + let fields = self.schema().unwrap().fields(); + col_stats + .iter() + .zip(fields) + .map(|(col_states, field)| { + let dt = self + .arrow_schema(true) + .unwrap() + .field_with_name(field.name()) + .unwrap() + .data_type() + .clone(); + ColumnStatistics { + null_count: col_states.null_count, + max_value: col_states + .max_value + .as_ref() + .and_then(|scalar| correct_scalar_value_type(scalar.clone(), &dt)), + min_value: col_states + .min_value + .as_ref() + .and_then(|scalar| correct_scalar_value_type(scalar.clone(), &dt)), + distinct_count: col_states.distinct_count, + } + }) + .collect() + }), + } + } +} + +impl DeltaTableState { + /// Return statistics for Datafusion Table + pub fn datafusion_table_statistics(&self) -> Statistics { + let stats = self + .files() + .iter() + .try_fold( + Statistics { + num_rows: Some(0), + total_byte_size: Some(0), + column_statistics: Some(vec![ + ColumnStatistics { + null_count: Some(0), + max_value: None, + min_value: None, + distinct_count: None + }; + self.schema().unwrap().fields().len() + ]), + is_exact: true, + }, + |acc, action| { + let new_stats = action + .get_stats() + .unwrap_or_else(|_| Some(Stats::default()))?; + Some(Statistics { + num_rows: acc + .num_rows + .map(|rows| rows + new_stats.num_records as usize), + total_byte_size: acc + .total_byte_size + .map(|total_size| total_size + action.size as usize), + column_statistics: acc.column_statistics.map(|col_stats| { + self.schema() + .unwrap() + .fields() + .iter() + .zip(col_stats) + .map(|(field, stats)| { + let null_count = new_stats + .null_count + .get(field.name()) + .and_then(|x| { + let null_count_acc = stats.null_count?; + let null_count = x.as_value()? as usize; + Some(null_count_acc + null_count) + }) + .or(stats.null_count); + + let max_value = new_stats + .max_values + .get(field.name()) + .and_then(|x| { + let old_stats = stats.clone(); + let max_value = to_scalar_value(x.as_value()?); + + match (max_value, old_stats.max_value) { + (Some(max_value), Some(old_max_value)) => { + if left_larger_than_right( + old_max_value.clone(), + max_value.clone(), + )? { + Some(old_max_value) + } else { + Some(max_value) + } + } + (Some(max_value), None) => Some(max_value), + (None, old) => old, + } + }) + .or_else(|| stats.max_value.clone()); + + let min_value = new_stats + .min_values + .get(field.name()) + .and_then(|x| { + let old_stats = stats.clone(); + let min_value = to_scalar_value(x.as_value()?); + + match (min_value, old_stats.min_value) { + (Some(min_value), Some(old_min_value)) => { + if left_larger_than_right( + min_value.clone(), + old_min_value.clone(), + )? { + Some(old_min_value) + } else { + Some(min_value) + } + } + (Some(min_value), None) => Some(min_value), + (None, old) => old, + } + }) + .or_else(|| stats.min_value.clone()); + + ColumnStatistics { + null_count, + max_value, + min_value, + distinct_count: None, // TODO: distinct + } + }) + .collect() + }), + is_exact: true, + }) + }, + ) + .unwrap_or_default(); + + // Convert column max/min scalar values to correct types based on arrow types. + Statistics { + is_exact: true, + num_rows: stats.num_rows, + total_byte_size: stats.total_byte_size, + column_statistics: stats.column_statistics.map(|col_stats| { + let fields = self.schema().unwrap().fields(); + col_stats + .iter() + .zip(fields) + .map(|(col_states, field)| { + let dt = self + .arrow_schema(true) + .unwrap() + .field_with_name(field.name()) + .unwrap() + .data_type() + .clone(); + ColumnStatistics { + null_count: col_states.null_count, + max_value: col_states + .max_value + .as_ref() + .and_then(|scalar| correct_scalar_value_type(scalar.clone(), &dt)), + min_value: col_states + .min_value + .as_ref() + .and_then(|scalar| correct_scalar_value_type(scalar.clone(), &dt)), + distinct_count: col_states.distinct_count, + } + }) + .collect() + }), + } + } +} + +fn correct_scalar_value_type(value: ScalarValue, field_dt: &ArrowDataType) -> Option { + match field_dt { + ArrowDataType::Int64 => { + let raw_value = i64::try_from(value).ok()?; + Some(ScalarValue::from(raw_value)) + } + ArrowDataType::Int32 => { + let raw_value = i64::try_from(value).ok()? as i32; + Some(ScalarValue::from(raw_value)) + } + ArrowDataType::Int16 => { + let raw_value = i64::try_from(value).ok()? as i16; + Some(ScalarValue::from(raw_value)) + } + ArrowDataType::Int8 => { + let raw_value = i64::try_from(value).ok()? as i8; + Some(ScalarValue::from(raw_value)) + } + ArrowDataType::Float32 => { + let raw_value = f64::try_from(value).ok()? as f32; + Some(ScalarValue::from(raw_value)) + } + ArrowDataType::Float64 => { + let raw_value = f64::try_from(value).ok()?; + Some(ScalarValue::from(raw_value)) + } + ArrowDataType::Utf8 => match value { + ScalarValue::Utf8(val) => Some(ScalarValue::Utf8(val)), + _ => None, + }, + ArrowDataType::LargeUtf8 => match value { + ScalarValue::Utf8(val) => Some(ScalarValue::LargeUtf8(val)), + _ => None, + }, + ArrowDataType::Boolean => { + let raw_value = bool::try_from(value).ok()?; + Some(ScalarValue::from(raw_value)) + } + ArrowDataType::Decimal128(_, _) => { + let raw_value = f64::try_from(value).ok()?; + Some(ScalarValue::from(raw_value)) + } + ArrowDataType::Decimal256(_, _) => { + let raw_value = f64::try_from(value).ok()?; + Some(ScalarValue::from(raw_value)) + } + ArrowDataType::Date32 => { + let raw_value = i64::try_from(value).ok()? as i32; + Some(ScalarValue::Date32(Some(raw_value))) + } + ArrowDataType::Date64 => { + let raw_value = i64::try_from(value).ok()?; + Some(ScalarValue::Date64(Some(raw_value))) + } + ArrowDataType::Timestamp(TimeUnit::Nanosecond, None) => { + let raw_value = i64::try_from(value).ok()?; + Some(ScalarValue::TimestampNanosecond(Some(raw_value), None)) + } + ArrowDataType::Timestamp(TimeUnit::Microsecond, None) => { + let raw_value = i64::try_from(value).ok()?; + Some(ScalarValue::TimestampMicrosecond(Some(raw_value), None)) + } + ArrowDataType::Timestamp(TimeUnit::Millisecond, None) => { + let raw_value = i64::try_from(value).ok()?; + Some(ScalarValue::TimestampMillisecond(Some(raw_value), None)) + } + _ => { + log::error!( + "Scalar value of arrow type unimplemented for {:?} and {:?}", + value, + field_dt + ); + None + } + } +} + +/// Container for [`Add`] actions to implement helper functions for pruning +pub struct AddContainer<'a> { + inner: &'a Vec, + partition_columns: &'a Vec, + schema: ArrowSchemaRef, +} + +impl<'a> AddContainer<'a> { + /// Create a new instance of [`AddContainer`] + pub fn new( + adds: &'a Vec, + partition_columns: &'a Vec, + schema: ArrowSchemaRef, + ) -> Self { + Self { + inner: adds, + partition_columns, + schema, + } + } + + /// get column statistics + pub fn get_prune_stats(&self, column: &Column, get_max: bool) -> Option { + let (_, field) = self.schema.column_with_name(&column.name)?; + + // See issue 1214. Binary type does not support natural order which is required for Datafusion to prune + if field.data_type() == &ArrowDataType::Binary { + return None; + } + + let data_type = field.data_type(); + + let values = self.inner.iter().map(|add| { + if self.partition_columns.contains(&column.name) { + let value = add.partition_values.get(&column.name).unwrap(); + let value = match value { + Some(v) => serde_json::Value::String(v.to_string()), + None => serde_json::Value::Null, + }; + to_correct_scalar_value(&value, data_type).unwrap_or( + get_null_of_arrow_type(data_type).expect("Could not determine null type"), + ) + } else if let Ok(Some(statistics)) = add.get_stats() { + let values = if get_max { + statistics.max_values + } else { + statistics.min_values + }; + + values + .get(&column.name) + .and_then(|f| to_correct_scalar_value(f.as_value()?, data_type)) + .unwrap_or( + get_null_of_arrow_type(data_type).expect("Could not determine null type"), + ) + } else { + get_null_of_arrow_type(data_type).expect("Could not determine null type") + } + }); + ScalarValue::iter_to_array(values).ok() + } + + /// Get an iterator of add actions / files, that MAY contain data matching the predicate. + /// + /// Expressions are evaluated for file statistics, essentially column-wise min max bounds, + /// so evaluating expressions is inexact. However, excluded files are guaranteed (for a correct log) + /// to not contain matches by the predicate expression. + pub fn predicate_matches(&self, predicate: Expr) -> DeltaResult> { + let expr = logical_expr_to_physical_expr(&predicate, &self.schema); + let pruning_predicate = PruningPredicate::try_new(expr, self.schema.clone())?; + Ok(self + .inner + .iter() + .zip(pruning_predicate.prune(self)?) + .filter_map( + |(action, keep_file)| { + if keep_file { + Some(action) + } else { + None + } + }, + )) + } +} + +impl<'a> PruningStatistics for AddContainer<'a> { + /// return the minimum values for the named column, if known. + /// Note: the returned array must contain `num_containers()` rows + fn min_values(&self, column: &Column) -> Option { + self.get_prune_stats(column, false) + } + + /// return the maximum values for the named column, if known. + /// Note: the returned array must contain `num_containers()` rows. + fn max_values(&self, column: &Column) -> Option { + self.get_prune_stats(column, true) + } + + /// return the number of containers (e.g. row groups) being + /// pruned with these statistics + fn num_containers(&self) -> usize { + self.inner.len() + } + + /// return the number of null values for the named column as an + /// `Option`. + /// + /// Note: the returned array must contain `num_containers()` rows. + fn null_counts(&self, column: &Column) -> Option { + let values = self.inner.iter().map(|add| { + if let Ok(Some(statistics)) = add.get_stats() { + if self.partition_columns.contains(&column.name) { + let value = add.partition_values.get(&column.name).unwrap(); + match value { + Some(_) => ScalarValue::UInt64(Some(0)), + None => ScalarValue::UInt64(Some(statistics.num_records as u64)), + } + } else { + statistics + .null_count + .get(&column.name) + .map(|f| ScalarValue::UInt64(f.as_value().map(|val| val as u64))) + .unwrap_or(ScalarValue::UInt64(None)) + } + } else if self.partition_columns.contains(&column.name) { + let value = add.partition_values.get(&column.name).unwrap(); + match value { + Some(_) => ScalarValue::UInt64(Some(0)), + None => ScalarValue::UInt64(None), + } + } else { + ScalarValue::UInt64(None) + } + }); + ScalarValue::iter_to_array(values).ok() + } +} + +impl PruningStatistics for DeltaTableState { + /// return the minimum values for the named column, if known. + /// Note: the returned array must contain `num_containers()` rows + fn min_values(&self, column: &Column) -> Option { + let partition_columns = &self.current_metadata()?.partition_columns; + let container = AddContainer::new( + self.files(), + partition_columns, + self.arrow_schema(true).ok()?, + ); + container.min_values(column) + } + + /// return the maximum values for the named column, if known. + /// Note: the returned array must contain `num_containers()` rows. + fn max_values(&self, column: &Column) -> Option { + let partition_columns = &self.current_metadata()?.partition_columns; + let container = AddContainer::new( + self.files(), + partition_columns, + self.arrow_schema(true).ok()?, + ); + container.max_values(column) + } + + /// return the number of containers (e.g. row groups) being + /// pruned with these statistics + fn num_containers(&self) -> usize { + self.files().len() + } + + /// return the number of null values for the named column as an + /// `Option`. + /// + /// Note: the returned array must contain `num_containers()` rows. + fn null_counts(&self, column: &Column) -> Option { + let partition_columns = &self.current_metadata()?.partition_columns; + let container = AddContainer::new( + self.files(), + partition_columns, + self.arrow_schema(true).ok()?, + ); + container.null_counts(column) + } +} diff --git a/crates/deltalake-core/src/delta_datafusion/scan.rs b/crates/deltalake-core/src/delta_datafusion/scan.rs new file mode 100644 index 0000000000..a6f82ca035 --- /dev/null +++ b/crates/deltalake-core/src/delta_datafusion/scan.rs @@ -0,0 +1,458 @@ +//! This module contains the implementation of the DeltaTable scan operator for datafusion. + +use std::any::Any; +use std::collections::{HashMap, HashSet}; +use std::fmt::{self, Debug}; +use std::sync::Arc; + +use arrow::datatypes::DataType; +use arrow::datatypes::{Schema as ArrowSchema, SchemaRef}; +use arrow::error::ArrowError; +use chrono::{NaiveDateTime, TimeZone, Utc}; +use datafusion::datasource::file_format::{parquet::ParquetFormat, FileFormat}; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::physical_plan::{ + wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileScanConfig, +}; +use datafusion::execution::context::{SessionState, TaskContext}; +use datafusion::physical_expr::PhysicalSortExpr; +use datafusion::physical_optimizer::pruning::PruningPredicate; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, + Statistics, +}; +use datafusion_common::scalar::ScalarValue; +use datafusion_common::Result as DataFusionResult; +use datafusion_expr::Expr; +use object_store::ObjectMeta; +use serde::{Deserialize, Serialize}; + +use super::pruning::physical_arrow_schema; +use super::{get_null_of_arrow_type, to_correct_scalar_value}; +use super::{logical_expr_to_physical_expr, logical_schema, PATH_COLUMN}; +use crate::errors::{DeltaResult, DeltaTableError}; +use crate::kernel::Add; +use crate::logstore::LogStoreRef; +use crate::table::builder::ensure_table_uri; +use crate::table::state::DeltaTableState; + +#[derive(Debug, Clone, Default)] +/// Used to specify if additional metadata columns are exposed to the user +pub struct DeltaScanConfigBuilder { + /// Include the source path for each record. The name of this column is determine by `file_column_name` + pub(super) include_file_column: bool, + /// Column name that contains the source path. + /// + /// If include_file_column is true and the name is None then it will be auto-generated + /// Otherwise the user provided name will be used + pub(super) file_column_name: Option, +} + +impl DeltaScanConfigBuilder { + /// Construct a new instance of `DeltaScanConfigBuilder` + pub fn new() -> Self { + Self::default() + } + + /// Indicate that a column containing a records file path is included. + /// Column name is generated and can be determined once this Config is built + pub fn with_file_column(mut self, include: bool) -> Self { + self.include_file_column = include; + self.file_column_name = None; + self + } + + /// Indicate that a column containing a records file path is included and column name is user defined. + pub fn with_file_column_name(mut self, name: &S) -> Self { + self.file_column_name = Some(name.to_string()); + self.include_file_column = true; + self + } + + /// Build a DeltaScanConfig and ensure no column name conflicts occur during downstream processing + pub fn build(&self, snapshot: &DeltaTableState) -> DeltaResult { + let input_schema = snapshot.arrow_schema(false)?; + let mut file_column_name = None; + let mut column_names: HashSet<&String> = HashSet::new(); + for field in input_schema.fields.iter() { + column_names.insert(field.name()); + } + + if self.include_file_column { + match &self.file_column_name { + Some(name) => { + if column_names.contains(name) { + return Err(DeltaTableError::Generic(format!( + "Unable to add file path column since column with name {} exits", + name + ))); + } + + file_column_name = Some(name.to_owned()) + } + None => { + let prefix = PATH_COLUMN; + let mut idx = 0; + let mut name = prefix.to_owned(); + + while column_names.contains(&name) { + idx += 1; + name = format!("{}_{}", prefix, idx); + } + + file_column_name = Some(name); + } + } + } + + Ok(DeltaScanConfig { file_column_name }) + } +} + +#[derive(Debug, Clone, Default, Serialize, Deserialize)] +/// Include additional metadata columns during a [`DeltaScan`] +pub struct DeltaScanConfig { + /// Include the source path for each record + pub file_column_name: Option, +} + +#[derive(Debug)] +pub(crate) struct DeltaScanBuilder<'a> { + snapshot: &'a DeltaTableState, + log_store: LogStoreRef, + filter: Option, + state: &'a SessionState, + projection: Option<&'a Vec>, + limit: Option, + files: Option<&'a [Add]>, + config: DeltaScanConfig, + schema: Option, +} + +impl<'a> DeltaScanBuilder<'a> { + pub fn new( + snapshot: &'a DeltaTableState, + log_store: LogStoreRef, + state: &'a SessionState, + ) -> Self { + DeltaScanBuilder { + snapshot, + log_store, + filter: None, + state, + files: None, + projection: None, + limit: None, + config: DeltaScanConfig::default(), + schema: None, + } + } + + pub fn with_filter(mut self, filter: Option) -> Self { + self.filter = filter; + self + } + + pub fn with_files(mut self, files: &'a [Add]) -> Self { + self.files = Some(files); + self + } + + pub fn with_projection(mut self, projection: Option<&'a Vec>) -> Self { + self.projection = projection; + self + } + + pub fn with_limit(mut self, limit: Option) -> Self { + self.limit = limit; + self + } + + pub fn with_scan_config(mut self, config: DeltaScanConfig) -> Self { + self.config = config; + self + } + + pub fn with_schema(mut self, schema: SchemaRef) -> Self { + self.schema = Some(schema); + self + } + + pub async fn build(self) -> DeltaResult { + let config = self.config; + let schema = match self.schema { + Some(schema) => schema, + None => physical_arrow_schema(self.snapshot, self.log_store.object_store()).await?, + }; + let logical_schema = logical_schema(self.snapshot, &config)?; + + let logical_schema = if let Some(used_columns) = self.projection { + let mut fields = vec![]; + for idx in used_columns { + fields.push(logical_schema.field(*idx).to_owned()); + } + Arc::new(ArrowSchema::new(fields)) + } else { + logical_schema + }; + + let logical_filter = self + .filter + .map(|expr| logical_expr_to_physical_expr(&expr, &logical_schema)); + + // Perform Pruning of files to scan + let files = match self.files { + Some(files) => files.to_owned(), + None => { + if let Some(predicate) = &logical_filter { + let pruning_predicate = + PruningPredicate::try_new(predicate.clone(), logical_schema.clone())?; + let files_to_prune = pruning_predicate.prune(self.snapshot)?; + self.snapshot + .files() + .iter() + .zip(files_to_prune.into_iter()) + .filter_map( + |(action, keep)| { + if keep { + Some(action.to_owned()) + } else { + None + } + }, + ) + .collect() + } else { + self.snapshot.files().to_owned() + } + } + }; + + // TODO we group files together by their partition values. If the table is partitioned + // and partitions are somewhat evenly distributed, probably not the worst choice ... + // However we may want to do some additional balancing in case we are far off from the above. + let mut file_groups: HashMap, Vec> = HashMap::new(); + + let table_partition_cols = &self + .snapshot + .current_metadata() + .ok_or(DeltaTableError::NoMetadata)? + .partition_columns; + + for action in files.iter() { + let mut part = partitioned_file_from_action(action, table_partition_cols, &schema); + + if config.file_column_name.is_some() { + part.partition_values + .push(wrap_partition_value_in_dict(ScalarValue::Utf8(Some( + action.path.clone(), + )))); + } + + file_groups + .entry(part.partition_values.clone()) + .or_default() + .push(part); + } + + let file_schema = Arc::new(ArrowSchema::new( + schema + .fields() + .iter() + .filter(|f| !table_partition_cols.contains(f.name())) + .cloned() + .collect::>(), + )); + + let mut table_partition_cols = table_partition_cols + .iter() + .map(|c| Ok((c.to_owned(), schema.field_with_name(c)?.data_type().clone()))) + .collect::, ArrowError>>()?; + + if let Some(file_column_name) = &config.file_column_name { + table_partition_cols.push(( + file_column_name.clone(), + wrap_partition_type_in_dict(DataType::Utf8), + )); + } + + let scan = ParquetFormat::new() + .create_physical_plan( + self.state, + FileScanConfig { + object_store_url: self.log_store.object_store_url(), + file_schema, + file_groups: file_groups.into_values().collect(), + statistics: self.snapshot.datafusion_table_statistics(), + projection: self.projection.cloned(), + limit: self.limit, + table_partition_cols, + output_ordering: vec![], + infinite_source: false, + }, + logical_filter.as_ref(), + ) + .await?; + + Ok(DeltaScan { + table_uri: ensure_table_uri(self.log_store.root_uri())?.as_str().into(), + parquet_scan: scan, + config, + logical_schema, + }) + } +} + +// TODO: this will likely also need to perform column mapping later when we support reader protocol v2 +/// A wrapper for parquet scans +#[derive(Debug)] +pub struct DeltaScan { + /// The URL of the ObjectStore root + pub table_uri: String, + /// Column that contains an index that maps to the original metadata Add + pub config: DeltaScanConfig, + /// The parquet scan to wrap + pub parquet_scan: Arc, + /// The schema of the table to be used when evaluating expressions + pub logical_schema: Arc, +} + +impl DisplayAs for DeltaScan { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> std::fmt::Result { + write!(f, "DeltaScan") + } +} + +impl ExecutionPlan for DeltaScan { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.parquet_scan.schema() + } + + fn output_partitioning(&self) -> Partitioning { + self.parquet_scan.output_partitioning() + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + self.parquet_scan.output_ordering() + } + + fn children(&self) -> Vec> { + vec![self.parquet_scan.clone()] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> DataFusionResult> { + ExecutionPlan::with_new_children(self.parquet_scan.clone(), children) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> DataFusionResult { + self.parquet_scan.execute(partition, context) + } + + fn statistics(&self) -> Statistics { + self.parquet_scan.statistics() + } +} + +fn partitioned_file_from_action( + action: &Add, + partition_columns: &[String], + schema: &ArrowSchema, +) -> PartitionedFile { + let partition_values = partition_columns + .iter() + .map(|part| { + action + .partition_values + .get(part) + .map(|val| { + schema + .field_with_name(part) + .map(|field| match val { + Some(value) => to_correct_scalar_value( + &serde_json::Value::String(value.to_string()), + field.data_type(), + ) + .unwrap_or(ScalarValue::Null), + None => get_null_of_arrow_type(field.data_type()) + .unwrap_or(ScalarValue::Null), + }) + .unwrap_or(ScalarValue::Null) + }) + .unwrap_or(ScalarValue::Null) + }) + .collect::>(); + + let ts_secs = action.modification_time / 1000; + let ts_ns = (action.modification_time % 1000) * 1_000_000; + let last_modified = + Utc.from_utc_datetime(&NaiveDateTime::from_timestamp_opt(ts_secs, ts_ns as u32).unwrap()); + PartitionedFile { + object_meta: ObjectMeta { + last_modified, + ..action.try_into().unwrap() + }, + partition_values, + range: None, + extensions: None, + } +} + +#[cfg(test)] +mod tests { + use arrow::datatypes::{DataType as ArrowDataType, Field}; + use chrono::{TimeZone, Utc}; + use object_store::path::Path; + + use super::*; + + #[test] + fn test_partitioned_file_from_action() { + let mut partition_values = std::collections::HashMap::new(); + partition_values.insert("month".to_string(), Some("1".to_string())); + partition_values.insert("year".to_string(), Some("2015".to_string())); + let action = Add { + path: "year=2015/month=1/part-00000-4dcb50d3-d017-450c-9df7-a7257dbd3c5d-c000.snappy.parquet".to_string(), + size: 10644, + partition_values, + modification_time: 1660497727833, + partition_values_parsed: None, + data_change: true, + stats: None, + deletion_vector: None, + stats_parsed: None, + tags: None, + base_row_id: None, + default_row_commit_version: None, + }; + let schema = ArrowSchema::new(vec![ + Field::new("year", ArrowDataType::Int64, true), + Field::new("month", ArrowDataType::Int64, true), + ]); + + let part_columns = vec!["year".to_string(), "month".to_string()]; + let file = partitioned_file_from_action(&action, &part_columns, &schema); + 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_opt(1660497727833).unwrap(), + size: 10644, + e_tag: None + }, + partition_values: [ScalarValue::Int64(Some(2015)), ScalarValue::Int64(Some(1))].to_vec(), + range: None, + extensions: None, + }; + assert_eq!(file.partition_values, ref_file.partition_values) + } +} diff --git a/crates/deltalake-core/src/kernel/actions/mod.rs b/crates/deltalake-core/src/kernel/actions/mod.rs index 865c9d3cd9..637d520c41 100644 --- a/crates/deltalake-core/src/kernel/actions/mod.rs +++ b/crates/deltalake-core/src/kernel/actions/mod.rs @@ -7,9 +7,7 @@ use std::collections::HashMap; use serde::{Deserialize, Serialize}; -#[cfg(all(feature = "arrow", feature = "parquet"))] -pub(crate) mod arrow; -// pub(crate) mod schemas; +pub(crate) mod schemas; mod serde_path; pub(crate) mod types; @@ -32,10 +30,12 @@ pub enum ActionType { Protocol, /// modify the data in a table by removing individual logical files Remove, - /// The Row ID high-water mark tracks the largest ID that has been assigned to a row in the table. - RowIdHighWaterMark, /// Transactional information Txn, + /// Checkpoint metadata + CheckpointMetadata, + /// Sidecar + Sidecar, } #[derive(Debug, PartialEq, Eq, Clone, Deserialize, Serialize)] diff --git a/crates/deltalake-core/src/kernel/actions/schemas.rs b/crates/deltalake-core/src/kernel/actions/schemas.rs index 0cc870318f..ad3e3ccbad 100644 --- a/crates/deltalake-core/src/kernel/actions/schemas.rs +++ b/crates/deltalake-core/src/kernel/actions/schemas.rs @@ -1,255 +1,262 @@ -use std::sync::Arc; +//! Schema definitions for action types -use arrow_schema::{DataType, Field, Fields, Schema}; +use lazy_static::lazy_static; use super::ActionType; +use crate::kernel::schema::{ArrayType, DataType, MapType, StructField, StructType}; -impl ActionType { - /// Returns the root field for the action type - pub fn field(&self) -> Field { - match self { - Self::Add => get_root("add", self.fields()), - Self::Cdc => get_root("cdc", self.fields()), - Self::CommitInfo => get_root("commitInfo", self.fields()), - Self::DomainMetadata => get_root("domainMetadata", self.fields()), - Self::Metadata => get_root("metaData", self.fields()), - Self::Protocol => get_root("protocol", self.fields()), - Self::Remove => get_root("remove", self.fields()), - Self::RowIdHighWaterMark => get_root("rowIdHighWaterMark", self.fields()), - Self::Txn => get_root("txn", self.fields()), - } - } - - /// Returns the child fields for the action type - pub fn fields(&self) -> Vec { - match self { - Self::Add => add_fields(), - Self::Cdc => cdc_fields(), - Self::CommitInfo => commit_info_fields(), - Self::DomainMetadata => domain_metadata_fields(), - Self::Metadata => metadata_fields(), - Self::Protocol => protocol_fields(), - Self::Remove => remove_fields(), - Self::RowIdHighWaterMark => watermark_fields(), - Self::Txn => txn_fields(), - } - } -} - -/// Returns the schema for the delta log -pub fn get_log_schema() -> Schema { - Schema { - fields: Fields::from_iter([ - ActionType::Add.field(), - ActionType::Cdc.field(), - ActionType::CommitInfo.field(), - ActionType::DomainMetadata.field(), - ActionType::Metadata.field(), - ActionType::Protocol.field(), - ActionType::Remove.field(), - ActionType::RowIdHighWaterMark.field(), - ActionType::Txn.field(), - ]), - metadata: Default::default(), - } -} - -fn get_root(name: &str, fields: Vec) -> Field { - Field::new(name, DataType::Struct(Fields::from_iter(fields)), true) -} - -fn add_fields() -> Vec { - Vec::from_iter([ - Field::new("path", DataType::Utf8, false), - Field::new("size", DataType::Int64, false), - Field::new("modificationTime", DataType::Int64, false), - Field::new("dataChange", DataType::Boolean, false), - Field::new("stats", DataType::Utf8, true), - Field::new( - "partitionValues", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - Field::new( - "tags", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - Field::new( - "deletionVector", - DataType::Struct(Fields::from(vec![ - Field::new("storageType", DataType::Utf8, false), - Field::new("pathOrInlineDv", DataType::Utf8, false), - Field::new("offset", DataType::Int32, true), - Field::new("sizeInBytes", DataType::Int32, false), - Field::new("cardinality", DataType::Int64, false), - ])), - true, - ), - Field::new("baseRowId", DataType::Int64, true), - Field::new("defaultRowCommitVersion", DataType::Int64, true), - ]) -} - -fn cdc_fields() -> Vec { - Vec::from_iter([ - Field::new("path", DataType::Utf8, true), - Field::new( - "partitionValues", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - Field::new("size", DataType::Int64, true), - Field::new("dataChange", DataType::Boolean, true), - Field::new( - "tags", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - ]) -} - -fn remove_fields() -> Vec { - Vec::from_iter([ - Field::new("path", DataType::Utf8, true), - Field::new("deletionTimestamp", DataType::Int64, true), - Field::new("dataChange", DataType::Boolean, true), - Field::new("extendedFileMetadata", DataType::Boolean, true), - Field::new("size", DataType::Int64, true), - Field::new( - "partitionValues", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - Field::new( - "tags", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - ]) -} - -fn metadata_fields() -> Vec { - Vec::from_iter([ - Field::new("id", DataType::Utf8, false), - Field::new("name", DataType::Utf8, true), - Field::new("description", DataType::Utf8, true), - Field::new( - "format", - DataType::Struct(Fields::from_iter([ - Field::new("provider", DataType::Utf8, true), - Field::new( - "options", - DataType::Map( - Arc::new(Field::new( - "key_value", - DataType::Struct(Fields::from_iter([ - Field::new("key", DataType::Utf8, false), - Field::new("value", DataType::Utf8, true), - ])), - false, - )), - false, +lazy_static! { + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#change-metadata + static ref METADATA_FIELD: StructField = StructField::new( + "metaData", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("id", DataType::string(), false), + StructField::new("name", DataType::string(), true), + StructField::new("description", DataType::string(), true), + StructField::new( + "format", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("provider", DataType::string(), false), + StructField::new( + "configuration", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), + true, + ))), + true, ), - false, - ), - ])), - false, - ), - Field::new("schemaString", DataType::Utf8, false), - Field::new("createdTime", DataType::Int64, true), - Field::new( - "partitionColumns", - DataType::List(Arc::new(Field::new("element", DataType::Utf8, false))), - false, - ), - Field::new( - "configuration", - DataType::Map( - Arc::new(Field::new( - "key_value", - DataType::Struct(Fields::from_iter([ - Field::new("key", DataType::Utf8, false), - Field::new("value", DataType::Utf8, true), - ])), - false, - )), + ]))), false, ), - true, - ), - ]) + StructField::new("schemaString", DataType::string(), false), + StructField::new( + "partitionColumns", + DataType::Array(Box::new(ArrayType::new(DataType::string(), false))), + false, + ), + StructField::new("createdTime", DataType::long(), true), + StructField::new( + "configuration", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), + true, + ))), + false, + ), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#protocol-evolution + static ref PROTOCOL_FIELD: StructField = StructField::new( + "protocol", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("minReaderVersion", DataType::integer(), false), + StructField::new("minWriterVersion", DataType::integer(), false), + StructField::new( + "readerFeatures", + DataType::Array(Box::new(ArrayType::new(DataType::string(), false))), + true, + ), + StructField::new( + "writerFeatures", + DataType::Array(Box::new(ArrayType::new(DataType::string(), false))), + true, + ), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#commit-provenance-information + static ref COMMIT_INFO_FIELD: StructField = StructField::new( + "commitInfo", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("timestamp", DataType::timestamp(), false), + StructField::new("operation", DataType::string(), false), + StructField::new("isolationLevel", DataType::string(), true), + StructField::new("isBlindAppend", DataType::boolean(), true), + StructField::new("txnId", DataType::string(), true), + StructField::new("readVersion", DataType::long(), true), + StructField::new( + "operationParameters", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), + true, + ))), + true, + ), + StructField::new( + "operationMetrics", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), + true, + ))), + true, + ), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-file-and-remove-file + static ref ADD_FIELD: StructField = StructField::new( + "add", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("path", DataType::string(), false), + partition_values_field(), + StructField::new("size", DataType::long(), false), + StructField::new("modificationTime", DataType::timestamp(), false), + StructField::new("dataChange", DataType::boolean(), false), + StructField::new("stats", DataType::string(), true), + tags_field(), + deletion_vector_field(), + StructField::new("baseRowId", DataType::long(), true), + StructField::new("defaultRowCommitVersion", DataType::long(), true), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-file-and-remove-file + static ref REMOVE_FIELD: StructField = StructField::new( + "remove", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("path", DataType::string(), false), + StructField::new("deletionTimestamp", DataType::timestamp(), true), + StructField::new("dataChange", DataType::boolean(), false), + StructField::new("extendedFileMetadata", DataType::boolean(), true), + partition_values_field(), + StructField::new("size", DataType::long(), true), + StructField::new("stats", DataType::string(), true), + tags_field(), + deletion_vector_field(), + StructField::new("baseRowId", DataType::long(), true), + StructField::new("defaultRowCommitVersion", DataType::long(), true), + ]))), + true, + ); + static ref REMOVE_FIELD_CHECKPOINT: StructField = StructField::new( + "remove", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("path", DataType::string(), false), + StructField::new("deletionTimestamp", DataType::timestamp(), true), + StructField::new("dataChange", DataType::boolean(), false), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-cdc-file + static ref CDC_FIELD: StructField = StructField::new( + "cdc", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("path", DataType::string(), false), + partition_values_field(), + StructField::new("size", DataType::long(), false), + StructField::new("dataChange", DataType::boolean(), false), + tags_field(), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#transaction-identifiers + static ref TXN_FIELD: StructField = StructField::new( + "txn", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("appId", DataType::string(), false), + StructField::new("version", DataType::long(), false), + StructField::new("lastUpdated", DataType::timestamp(), true), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#domain-metadata + static ref DOMAIN_METADATA_FIELD: StructField = StructField::new( + "domainMetadata", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("domain", DataType::string(), false), + StructField::new( + "configuration", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), + true, + ))), + false, + ), + StructField::new("removed", DataType::boolean(), false), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#checkpoint-metadata + static ref CHECKPOINT_METADATA_FIELD: StructField = StructField::new( + "checkpointMetadata", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("flavor", DataType::string(), false), + tags_field(), + ]))), + true, + ); + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#sidecar-file-information + static ref SIDECAR_FIELD: StructField = StructField::new( + "sidecar", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("path", DataType::string(), false), + StructField::new("sizeInBytes", DataType::long(), false), + StructField::new("modificationTime", DataType::timestamp(), false), + StructField::new("type", DataType::string(), false), + tags_field(), + ]))), + true, + ); } -fn protocol_fields() -> Vec { - Vec::from_iter([ - Field::new("minReaderVersion", DataType::Int32, false), - Field::new("minWriterVersion", DataType::Int32, false), - Field::new( - "readerFeatures", - DataType::List(Arc::new(Field::new("element", DataType::Utf8, false))), +fn tags_field() -> StructField { + StructField::new( + "tags", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), true, - ), - Field::new( - "writerFeatures", - DataType::List(Arc::new(Field::new("element", DataType::Utf8, false))), - true, - ), - ]) -} - -fn txn_fields() -> Vec { - Vec::from_iter([ - Field::new("appId", DataType::Utf8, true), - Field::new("version", DataType::Int64, true), - Field::new("lastUpdated", DataType::Int64, true), - ]) -} - -fn watermark_fields() -> Vec { - Vec::from_iter([Field::new("highWaterMark", DataType::Int64, true)]) + ))), + true, + ) } -fn commit_info_fields() -> Vec { - Vec::from_iter([ - Field::new("timestamp", DataType::Int64, true), - Field::new("operation", DataType::Utf8, true), - Field::new("isolationLevel", DataType::Utf8, true), - Field::new("isBlindAppend", DataType::Boolean, true), - Field::new("txnId", DataType::Utf8, true), - Field::new("readVersion", DataType::Int32, true), - Field::new( - "operationParameters", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - Field::new( - "operationMetrics", - DataType::Map(Arc::new(get_map_field()), false), +fn partition_values_field() -> StructField { + StructField::new( + "partitionValues", + DataType::Map(Box::new(MapType::new( + DataType::string(), + DataType::string(), true, - ), - ]) + ))), + false, + ) } -fn domain_metadata_fields() -> Vec { - Vec::from_iter([ - Field::new("domain", DataType::Utf8, true), - Field::new( - "configuration", - DataType::Map(Arc::new(get_map_field()), false), - true, - ), - Field::new("removed", DataType::Boolean, true), - ]) +fn deletion_vector_field() -> StructField { + StructField::new( + "deletionVector", + DataType::Struct(Box::new(StructType::new(vec![ + StructField::new("storageType", DataType::string(), false), + StructField::new("pathOrInlineDv", DataType::string(), false), + StructField::new("offset", DataType::integer(), true), + StructField::new("sizeInBytes", DataType::integer(), false), + StructField::new("cardinality", DataType::long(), false), + ]))), + true, + ) } -fn get_map_field() -> Field { - Field::new( - "key_value", - DataType::Struct(Fields::from_iter([ - Field::new("key", DataType::Utf8, false), - Field::new("value", DataType::Utf8, true), - ])), - false, - ) +impl ActionType { + /// Returns the type of the corresponding field in the delta log schema + pub fn schema_field(&self) -> &StructField { + match self { + Self::Metadata => &METADATA_FIELD, + Self::Protocol => &PROTOCOL_FIELD, + Self::CommitInfo => &COMMIT_INFO_FIELD, + Self::Add => &ADD_FIELD, + Self::Remove => &REMOVE_FIELD, + Self::Cdc => &CDC_FIELD, + Self::Txn => &TXN_FIELD, + Self::DomainMetadata => &DOMAIN_METADATA_FIELD, + Self::CheckpointMetadata => &CHECKPOINT_METADATA_FIELD, + Self::Sidecar => &SIDECAR_FIELD, + } + } } diff --git a/crates/deltalake-core/src/kernel/actions/types.rs b/crates/deltalake-core/src/kernel/actions/types.rs index e8060f145e..aa60823e4a 100644 --- a/crates/deltalake-core/src/kernel/actions/types.rs +++ b/crates/deltalake-core/src/kernel/actions/types.rs @@ -1,4 +1,5 @@ use std::collections::{HashMap, HashSet}; +use std::fmt; use std::str::FromStr; // use std::io::{Cursor, Read}; // use std::sync::Arc; @@ -173,7 +174,7 @@ pub enum ReaderFeatures { /// Mapping of one column to another ColumnMapping, /// Deletion vectors for merge, update, delete - DeleteionVecotrs, + DeletionVectors, /// timestamps without timezone support #[serde(alias = "timestampNtz")] TimestampWithoutTimezone, @@ -184,26 +185,13 @@ pub enum ReaderFeatures { Other(String), } -#[allow(clippy::from_over_into)] -impl Into for ReaderFeatures { - fn into(self) -> usize { - match self { - ReaderFeatures::Other(_) => 0, - ReaderFeatures::ColumnMapping => 2, - ReaderFeatures::DeleteionVecotrs - | ReaderFeatures::TimestampWithoutTimezone - | ReaderFeatures::V2Checkpoint => 3, - } - } -} - #[cfg(all(not(feature = "parquet2"), feature = "parquet"))] impl From<&parquet::record::Field> for ReaderFeatures { fn from(value: &parquet::record::Field) -> Self { match value { parquet::record::Field::Str(feature) => match feature.as_str() { "columnMapping" => ReaderFeatures::ColumnMapping, - "deletionVectors" => ReaderFeatures::DeleteionVecotrs, + "deletionVectors" => ReaderFeatures::DeletionVectors, "timestampNtz" => ReaderFeatures::TimestampWithoutTimezone, "v2Checkpoint" => ReaderFeatures::V2Checkpoint, f => ReaderFeatures::Other(f.to_string()), @@ -215,9 +203,15 @@ impl From<&parquet::record::Field> for ReaderFeatures { impl From for ReaderFeatures { fn from(value: String) -> Self { - match value.as_str() { + value.as_str().into() + } +} + +impl From<&str> for ReaderFeatures { + fn from(value: &str) -> Self { + match value { "columnMapping" => ReaderFeatures::ColumnMapping, - "deletionVectors" => ReaderFeatures::DeleteionVecotrs, + "deletionVectors" => ReaderFeatures::DeletionVectors, "timestampNtz" => ReaderFeatures::TimestampWithoutTimezone, "v2Checkpoint" => ReaderFeatures::V2Checkpoint, f => ReaderFeatures::Other(f.to_string()), @@ -225,6 +219,24 @@ impl From for ReaderFeatures { } } +impl AsRef for ReaderFeatures { + fn as_ref(&self) -> &str { + match self { + ReaderFeatures::ColumnMapping => "columnMapping", + ReaderFeatures::DeletionVectors => "deletionVectors", + ReaderFeatures::TimestampWithoutTimezone => "timestampNtz", + ReaderFeatures::V2Checkpoint => "v2Checkpoint", + ReaderFeatures::Other(f) => f, + } + } +} + +impl fmt::Display for ReaderFeatures { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.as_ref()) + } +} + /// Features table writers can support as well as let users know /// what is supported #[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq, Hash)] @@ -245,7 +257,7 @@ pub enum WriterFeatures { /// ID Columns IdentityColumns, /// Deletion vectors for merge, update, delete - DeleteionVecotrs, + DeletionVectors, /// Row tracking on tables RowTracking, /// timestamps without timezone support @@ -262,29 +274,15 @@ pub enum WriterFeatures { Other(String), } -#[allow(clippy::from_over_into)] -impl Into for WriterFeatures { - fn into(self) -> usize { - match self { - WriterFeatures::Other(_) => 0, - WriterFeatures::AppendOnly | WriterFeatures::Invariants => 2, - WriterFeatures::CheckConstraints => 3, - WriterFeatures::ChangeDataFeed | WriterFeatures::GeneratedColumns => 4, - WriterFeatures::ColumnMapping => 5, - WriterFeatures::IdentityColumns - | WriterFeatures::DeleteionVecotrs - | WriterFeatures::RowTracking - | WriterFeatures::TimestampWithoutTimezone - | WriterFeatures::DomainMetadata - | WriterFeatures::V2Checkpoint - | WriterFeatures::IcebergCompatV1 => 7, - } +impl From for WriterFeatures { + fn from(value: String) -> Self { + value.as_str().into() } } -impl From for WriterFeatures { - fn from(value: String) -> Self { - match value.as_str() { +impl From<&str> for WriterFeatures { + fn from(value: &str) -> Self { + match value { "appendOnly" => WriterFeatures::AppendOnly, "invariants" => WriterFeatures::Invariants, "checkConstraints" => WriterFeatures::CheckConstraints, @@ -292,7 +290,7 @@ impl From for WriterFeatures { "generatedColumns" => WriterFeatures::GeneratedColumns, "columnMapping" => WriterFeatures::ColumnMapping, "identityColumns" => WriterFeatures::IdentityColumns, - "deletionVectors" => WriterFeatures::DeleteionVecotrs, + "deletionVectors" => WriterFeatures::DeletionVectors, "rowTracking" => WriterFeatures::RowTracking, "timestampNtz" => WriterFeatures::TimestampWithoutTimezone, "domainMetadata" => WriterFeatures::DomainMetadata, @@ -303,6 +301,33 @@ impl From for WriterFeatures { } } +impl AsRef for WriterFeatures { + fn as_ref(&self) -> &str { + match self { + WriterFeatures::AppendOnly => "appendOnly", + WriterFeatures::Invariants => "invariants", + WriterFeatures::CheckConstraints => "checkConstraints", + WriterFeatures::ChangeDataFeed => "changeDataFeed", + WriterFeatures::GeneratedColumns => "generatedColumns", + WriterFeatures::ColumnMapping => "columnMapping", + WriterFeatures::IdentityColumns => "identityColumns", + WriterFeatures::DeletionVectors => "deletionVectors", + WriterFeatures::RowTracking => "rowTracking", + WriterFeatures::TimestampWithoutTimezone => "timestampNtz", + WriterFeatures::DomainMetadata => "domainMetadata", + WriterFeatures::V2Checkpoint => "v2Checkpoint", + WriterFeatures::IcebergCompatV1 => "icebergCompatV1", + WriterFeatures::Other(f) => f, + } + } +} + +impl fmt::Display for WriterFeatures { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.as_ref()) + } +} + #[cfg(all(not(feature = "parquet2"), feature = "parquet"))] impl From<&parquet::record::Field> for WriterFeatures { fn from(value: &parquet::record::Field) -> Self { @@ -315,7 +340,7 @@ impl From<&parquet::record::Field> for WriterFeatures { "generatedColumns" => WriterFeatures::GeneratedColumns, "columnMapping" => WriterFeatures::ColumnMapping, "identityColumns" => WriterFeatures::IdentityColumns, - "deletionVectors" => WriterFeatures::DeleteionVecotrs, + "deletionVectors" => WriterFeatures::DeletionVectors, "rowTracking" => WriterFeatures::RowTracking, "timestampNtz" => WriterFeatures::TimestampWithoutTimezone, "domainMetadata" => WriterFeatures::DomainMetadata, @@ -375,7 +400,7 @@ impl AsRef for StorageType { impl ToString for StorageType { fn to_string(&self) -> String { - self.as_ref().to_string() + self.as_ref().into() } } @@ -404,6 +429,7 @@ pub struct DeletionVectorDescriptor { /// Start of the data for this DV in number of bytes from the beginning of the file it is stored in. /// Always None (absent in JSON) when `storageType = 'i'`. + #[serde(skip_serializing_if = "Option::is_none")] pub offset: Option, /// Size of the serialized DV in bytes (raw data size, i.e. before base85 encoding, if inline). @@ -616,9 +642,11 @@ pub struct Remove { pub data_change: bool, /// The time this logical file was created, as milliseconds since the epoch. + #[serde(skip_serializing_if = "Option::is_none")] pub deletion_timestamp: Option, /// When true the fields `partition_values`, `size`, and `tags` are present + #[serde(skip_serializing_if = "Option::is_none")] pub extended_file_metadata: Option, /// A map from partition column to value for this logical file. @@ -640,9 +668,11 @@ pub struct Remove { /// Default generated Row ID of the first row in the file. The default generated Row IDs /// of the other rows in the file can be reconstructed by adding the physical index of the /// row within the file to the base Row ID + #[serde(skip_serializing_if = "Option::is_none")] pub base_row_id: Option, /// First commit version in which an add action with the same path was committed to the table. + #[serde(skip_serializing_if = "Option::is_none")] pub default_row_commit_version: Option, } @@ -661,13 +691,18 @@ pub struct AddCDCFile { /// absolute path to a CDC file #[serde(with = "serde_path")] pub path: String, + /// The size of this file in bytes pub size: i64, + /// A map from partition column to value for this file pub partition_values: HashMap>, + /// Should always be set to false because they do not change the underlying data of the table pub data_change: bool, + /// Map containing metadata about this file + #[serde(skip_serializing_if = "Option::is_none")] pub tags: Option>>, } @@ -678,9 +713,12 @@ pub struct AddCDCFile { pub struct Txn { /// A unique identifier for the application performing the transaction. pub app_id: String, + /// An application-specific numeric identifier for this transaction. pub version: i64, + /// The time when this transaction action was created in milliseconds since the Unix epoch. + #[serde(skip_serializing_if = "Option::is_none")] pub last_updated: Option, } @@ -693,30 +731,39 @@ pub struct CommitInfo { /// Timestamp in millis when the commit was created #[serde(skip_serializing_if = "Option::is_none")] pub timestamp: Option, + /// Id of the user invoking the commit #[serde(skip_serializing_if = "Option::is_none")] pub user_id: Option, + /// Name of the user invoking the commit #[serde(skip_serializing_if = "Option::is_none")] pub user_name: Option, + /// The operation performed during the #[serde(skip_serializing_if = "Option::is_none")] pub operation: Option, + /// Parameters used for table operation #[serde(skip_serializing_if = "Option::is_none")] pub operation_parameters: Option>, + /// Version of the table when the operation was started #[serde(skip_serializing_if = "Option::is_none")] pub read_version: Option, + /// The isolation level of the commit #[serde(skip_serializing_if = "Option::is_none")] pub isolation_level: Option, + /// TODO #[serde(skip_serializing_if = "Option::is_none")] pub is_blind_append: Option, + /// Delta engine which created the commit. #[serde(skip_serializing_if = "Option::is_none")] pub engine_info: Option, + /// Additional provenance information for the commit #[serde(flatten, default)] pub info: HashMap, @@ -728,12 +775,50 @@ pub struct CommitInfo { pub struct DomainMetadata { /// Identifier for this domain (system or user-provided) pub domain: String, + /// String containing configuration for the metadata domain pub configuration: String, + /// When `true` the action serves as a tombstone pub removed: bool, } +#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] +/// This action is only allowed in checkpoints following V2 spec. It describes the details about the checkpoint. +pub struct CheckpointMetadata { + /// The flavor of the V2 checkpoint. Allowed values: "flat". + pub flavor: String, + + /// Map containing any additional metadata about the v2 spec checkpoint. + #[serde(skip_serializing_if = "Option::is_none")] + pub tags: Option>>, +} + +/// The sidecar action references a sidecar file which provides some of the checkpoint's file actions. +/// This action is only allowed in checkpoints following V2 spec. +#[derive(Serialize, Deserialize, Debug, Clone, Default, PartialEq, Eq)] +#[serde(rename_all = "camelCase")] +pub struct Sidecar { + /// The name of the sidecar file (not a path). + /// The file must reside in the _delta_log/_sidecars directory. + pub file_name: String, + + /// The size of the sidecar file in bytes + pub size_in_bytes: i64, + + /// The time this sidecar file was created, as milliseconds since the epoch. + pub modification_time: i64, + + /// Type of sidecar. Valid values are: "fileaction". + /// This could be extended in future to allow different kinds of sidecars. + #[serde(rename = "type")] + pub sidecar_type: String, + + /// Map containing any additional metadata about the checkpoint sidecar file. + #[serde(skip_serializing_if = "Option::is_none")] + pub tags: Option>>, +} + #[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq)] /// The isolation level applied during transaction pub enum IsolationLevel { diff --git a/crates/deltalake-core/src/kernel/actions/checkpoint.rs b/crates/deltalake-core/src/kernel/arrow/checkpoint.rs similarity index 85% rename from crates/deltalake-core/src/kernel/actions/checkpoint.rs rename to crates/deltalake-core/src/kernel/arrow/checkpoint.rs index 59960f66b8..7ae8d5e87e 100644 --- a/crates/deltalake-core/src/kernel/actions/checkpoint.rs +++ b/crates/deltalake-core/src/kernel/arrow/checkpoint.rs @@ -1,23 +1,59 @@ -use std::collections::HashMap; +#![allow(dead_code)] +//! This module contains the logic to handle the delta log actions represented as recod batches + +use std::collections::{HashMap, HashSet}; +use std::io::Cursor; use std::str::FromStr; use arrow_array::{ BooleanArray, Int32Array, Int64Array, ListArray, MapArray, RecordBatch, StringArray, - StructArray, + StructArray, TimestampMicrosecondArray, }; +use arrow_json::ReaderBuilder; +use arrow_schema::SchemaRef as ArrowSchemaRef; +use arrow_select::concat::concat_batches; use either::Either; use fix_hidden_lifetime_bug::fix_hidden_lifetime_bug; + use itertools::izip; -use serde::{Deserialize, Serialize}; -use super::{error::Error, DeltaResult}; +use crate::kernel::error::{DeltaResult, Error}; +use crate::kernel::{ + Action, ActionType, Add, DeletionVectorDescriptor, Format, Metadata, Protocol, Remove, + StorageType, +}; -#[fix_hidden_lifetime_bug] -#[allow(dead_code)] -pub(crate) fn parse_actions<'a>( +/// Parse a JSON strings into a RecordBatch of delta actions +pub fn parse_json( + json_strings: StringArray, + output_schema: ArrowSchemaRef, +) -> DeltaResult { + // TODO concatenating to a single string is probably not needed if we use the + // lower level RawDecoder APIs + let data = json_strings + .into_iter() + .filter_map(|d| { + d.map(|dd| { + let mut data = dd.as_bytes().to_vec(); + data.extend("\n".as_bytes()); + data + }) + }) + .flatten() + .collect::>(); + + let batches = ReaderBuilder::new(output_schema.clone()) + .build(Cursor::new(data))? + .collect::, _>>()?; + + Ok(concat_batches(&output_schema, &batches)?) +} + +// #[fix_hidden_lifetime_bug] +pub(crate) fn parse_actions( batch: &RecordBatch, - types: impl IntoIterator, -) -> DeltaResult> { + types: Vec, +) -> DeltaResult + Send + '_> { Ok(types .into_iter() .filter_map(|action| parse_action(batch, action).ok()) @@ -27,8 +63,8 @@ pub(crate) fn parse_actions<'a>( #[fix_hidden_lifetime_bug] pub(crate) fn parse_action( batch: &RecordBatch, - action_type: &ActionType, -) -> DeltaResult> { + action_type: ActionType, +) -> DeltaResult + Send> { let column_name = match action_type { ActionType::Metadata => "metaData", ActionType::Protocol => "protocol", @@ -55,7 +91,9 @@ pub(crate) fn parse_action( } } -fn parse_action_metadata(arr: &StructArray) -> DeltaResult>> { +fn parse_action_metadata( + arr: &StructArray, +) -> DeltaResult + Send>> { let ids = cast_struct_column::(arr, "id")?; let schema_strings = cast_struct_column::(arr, "schemaString")?; let metadata = ids @@ -145,7 +183,9 @@ fn parse_action_metadata(arr: &StructArray) -> DeltaResult DeltaResult>> { +fn parse_action_protocol( + arr: &StructArray, +) -> DeltaResult + Send>> { let min_reader = cast_struct_column::(arr, "minReaderVersion")?; let min_writer = cast_struct_column::(arr, "minWriterVersion")?; let protocol = min_reader @@ -175,15 +215,15 @@ fn parse_action_protocol(arr: &StructArray) -> DeltaResult()? .iter() - .filter_map(|v| v.map(|inner| inner.to_owned())) - .collect::>(); + .filter_map(|v| v.map(|inner| inner.into())) + .collect::>(); Some(vals) } else { None } }) .flatten() - .collect::>() + .collect::>() }); protocol.writer_features = cast_struct_column::(arr, "writerFeatures") @@ -196,24 +236,27 @@ fn parse_action_protocol(arr: &StructArray) -> DeltaResult()? .iter() - .filter_map(|v| v.map(|inner| inner.to_string())) - .collect::>(); + .filter_map(|v| v.map(|inner| inner.into())) + .collect::>(); Some(vals) } else { None } }) .flatten() - .collect::>() + .collect::>() }); Ok(Box::new(std::iter::once(Action::Protocol(protocol)))) } -fn parse_actions_add(arr: &StructArray) -> DeltaResult + '_>> { +fn parse_actions_add( + arr: &StructArray, +) -> DeltaResult + Send + '_>> { let paths = cast_struct_column::(arr, "path")?; let sizes = cast_struct_column::(arr, "size")?; - let modification_times = cast_struct_column::(arr, "modificationTime")?; + let modification_times = + cast_struct_column::(arr, "modificationTime")?; let data_changes = cast_struct_column::(arr, "dataChange")?; let partition_values = cast_struct_column::(arr, "partitionValues")? .iter() @@ -309,12 +352,14 @@ fn parse_actions_add(arr: &StructArray) -> DeltaResult DeltaResult + '_>> { +fn parse_actions_remove( + arr: &StructArray, +) -> DeltaResult + Send + '_>> { let paths = cast_struct_column::(arr, "path")?; let data_changes = cast_struct_column::(arr, "dataChange")?; let deletion_timestamps = - if let Ok(ts) = cast_struct_column::(arr, "deletionTimestamp") { + if let Ok(ts) = cast_struct_column::(arr, "deletionTimestamp") { Either::Left(ts.into_iter()) } else { Either::Right(std::iter::repeat(None).take(data_changes.len())) @@ -423,7 +468,7 @@ fn parse_actions_remove(arr: &StructArray) -> DeltaResult DeltaResult> + '_> { +) -> DeltaResult> + Send + '_> { let storage_types = cast_struct_column::(arr, "storageType")?; let paths_or_inlines = cast_struct_column::(arr, "pathOrInlineDv")?; let sizes_in_bytes = cast_struct_column::(arr, "sizeInBytes")?; @@ -481,8 +526,13 @@ fn cast_struct_column(arr: &StructArray, name: impl AsRef) -> D } fn struct_array_to_map(arr: &StructArray) -> DeltaResult>> { - let keys = cast_struct_column::(arr, "key")?; - let values = cast_struct_column::(arr, "value")?; + if arr.fields().len() != 2 { + return Err(Error::UnexpectedColumnType( + "Error parsing map: expected struct array with 2 fields".into(), + )); + } + let keys = cast_struct_column::(arr, arr.fields()[0].name())?; + let values = cast_struct_column::(arr, arr.fields()[1].name())?; Ok(keys .into_iter() .zip(values) @@ -490,22 +540,15 @@ fn struct_array_to_map(arr: &StructArray) -> DeltaResult RecordBatch { - let store = Arc::new(LocalFileSystem::new()); - let handler = DefaultJsonHandler::new(store, Arc::new(TokioBackgroundExecutor::new())); - let json_strings: StringArray = vec![ r#"{"add":{"path":"part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet","partitionValues":{},"size":635,"modificationTime":1677811178336,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"value\":0},\"maxValues\":{\"value\":9},\"nullCount\":{\"value\":0},\"tightBounds\":true}","tags":{"INSERTION_TIME":"1677811178336000","MIN_INSERTION_TIME":"1677811178336000","MAX_INSERTION_TIME":"1677811178336000","OPTIMIZE_TARGET_SIZE":"268435456"}}}"#, r#"{"commitInfo":{"timestamp":1677811178585,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isolationLevel":"WriteSerializable","isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputRows":"10","numOutputBytes":"635"},"engineInfo":"Databricks-Runtime/","txnId":"a6a94671-55ef-450e-9546-b8465b9147de"}}"#, @@ -514,20 +557,20 @@ mod tests { ] .into(); let output_schema = Arc::new(get_log_schema()); - handler.parse_json(json_strings, output_schema).unwrap() + parse_json(json_strings, output_schema).unwrap() } #[test] fn test_parse_protocol() { let batch = action_batch(); - let action = parse_action(&batch, &ActionType::Protocol) + let action = parse_action(&batch, ActionType::Protocol) .unwrap() .collect::>(); let expected = Action::Protocol(Protocol { min_reader_version: 3, min_writer_version: 7, - reader_features: Some(vec!["deletionVectors".into()]), - writer_features: Some(vec!["deletionVectors".into()]), + reader_features: Some(vec![ReaderFeatures::DeletionVectors].into_iter().collect()), + writer_features: Some(vec![WriterFeatures::DeletionVectors].into_iter().collect()), }); assert_eq!(action[0], expected) } @@ -535,7 +578,7 @@ mod tests { #[test] fn test_parse_metadata() { let batch = action_batch(); - let action = parse_action(&batch, &ActionType::Metadata) + let action = parse_action(&batch, ActionType::Metadata) .unwrap() .collect::>(); let configuration = HashMap::from_iter([ @@ -566,9 +609,6 @@ mod tests { #[test] fn test_parse_add_partitioned() { - let store = Arc::new(LocalFileSystem::new()); - let handler = DefaultJsonHandler::new(store, Arc::new(TokioBackgroundExecutor::new())); - let json_strings: StringArray = vec![ r#"{"commitInfo":{"timestamp":1670892998177,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[\"c1\",\"c2\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"3","numOutputRows":"3","numOutputBytes":"1356"},"engineInfo":"Apache-Spark/3.3.1 Delta-Lake/2.2.0","txnId":"046a258f-45e3-4657-b0bf-abfb0f76681c"}}"#, r#"{"protocol":{"minReaderVersion":1,"minWriterVersion":2}}"#, @@ -579,9 +619,9 @@ mod tests { ] .into(); let output_schema = Arc::new(get_log_schema()); - let batch = handler.parse_json(json_strings, output_schema).unwrap(); + let batch = parse_json(json_strings, output_schema).unwrap(); - let actions = parse_action(&batch, &ActionType::Add) + let actions = parse_action(&batch, ActionType::Add) .unwrap() .collect::>(); println!("{:?}", actions) diff --git a/crates/deltalake-core/src/kernel/actions/arrow.rs b/crates/deltalake-core/src/kernel/arrow/mod.rs similarity index 98% rename from crates/deltalake-core/src/kernel/actions/arrow.rs rename to crates/deltalake-core/src/kernel/arrow/mod.rs index d292362604..c207503a8d 100644 --- a/crates/deltalake-core/src/kernel/actions/arrow.rs +++ b/crates/deltalake-core/src/kernel/arrow/mod.rs @@ -1,3 +1,5 @@ +//! Conversions between Delta and Arrow data types + use std::sync::Arc; use arrow_schema::{ @@ -6,7 +8,14 @@ use arrow_schema::{ }; use lazy_static::lazy_static; -use super::super::schema::{ArrayType, DataType, MapType, PrimitiveType, StructField, StructType}; +use super::schema::{ArrayType, DataType, MapType, PrimitiveType, StructField, StructType}; + +pub mod checkpoint; +pub mod schemas; +pub mod snapshot; + +const MAP_KEYS_NAME: &str = "keys"; +const MAP_VALUES_NAME: &str = "values"; impl TryFrom<&StructType> for ArrowSchema { type Error = ArrowError; @@ -64,9 +73,9 @@ impl TryFrom<&MapType> for ArrowField { "entries", ArrowDataType::Struct( vec![ - ArrowField::new("key", ArrowDataType::try_from(a.key_type())?, false), + ArrowField::new(MAP_KEYS_NAME, ArrowDataType::try_from(a.key_type())?, false), ArrowField::new( - "value", + MAP_VALUES_NAME, ArrowDataType::try_from(a.value_type())?, a.value_contains_null(), ), @@ -143,12 +152,12 @@ impl TryFrom<&DataType> for ArrowDataType { ArrowDataType::Struct( vec![ ArrowField::new( - "keys", + MAP_KEYS_NAME, >::try_from(m.key_type())?, false, ), ArrowField::new( - "values", + MAP_VALUES_NAME, >::try_from(m.value_type())?, m.value_contains_null(), ), diff --git a/crates/deltalake-core/src/kernel/arrow/schemas.rs b/crates/deltalake-core/src/kernel/arrow/schemas.rs new file mode 100644 index 0000000000..80a29e065e --- /dev/null +++ b/crates/deltalake-core/src/kernel/arrow/schemas.rs @@ -0,0 +1,63 @@ +//! Arrow schemas for the delta log + +use arrow_schema::{Field, Fields, Schema}; +use lazy_static::lazy_static; + +use super::super::ActionType; + +lazy_static! { + static ref ARROW_METADATA_FIELD: Field = + ActionType::Metadata.schema_field().try_into().unwrap(); + static ref ARROW_PROTOCOL_FIELD: Field = + ActionType::Protocol.schema_field().try_into().unwrap(); + static ref ARROW_COMMIT_INFO_FIELD: Field = + ActionType::CommitInfo.schema_field().try_into().unwrap(); + static ref ARROW_ADD_FIELD: Field = ActionType::Add.schema_field().try_into().unwrap(); + static ref ARROW_REMOVE_FIELD: Field = ActionType::Remove.schema_field().try_into().unwrap(); + static ref ARROW_CDC_FIELD: Field = ActionType::Cdc.schema_field().try_into().unwrap(); + static ref ARROW_TXN_FIELD: Field = ActionType::Txn.schema_field().try_into().unwrap(); + static ref ARROW_DOMAIN_METADATA_FIELD: Field = ActionType::DomainMetadata + .schema_field() + .try_into() + .unwrap(); + static ref ARROW_CHECKPOINT_METADATA_FIELD: Field = ActionType::CheckpointMetadata + .schema_field() + .try_into() + .unwrap(); + static ref ARROW_SIDECAR_FIELD: Field = ActionType::Sidecar.schema_field().try_into().unwrap(); +} + +impl ActionType { + /// Returns the root field for the action type + pub fn arrow_field(&self) -> &Field { + match self { + Self::Metadata => &ARROW_METADATA_FIELD, + Self::Protocol => &ARROW_PROTOCOL_FIELD, + Self::CommitInfo => &ARROW_COMMIT_INFO_FIELD, + Self::Add => &ARROW_ADD_FIELD, + Self::Remove => &ARROW_REMOVE_FIELD, + Self::Cdc => &ARROW_CDC_FIELD, + Self::Txn => &ARROW_TXN_FIELD, + Self::DomainMetadata => &ARROW_DOMAIN_METADATA_FIELD, + Self::CheckpointMetadata => &ARROW_CHECKPOINT_METADATA_FIELD, + Self::Sidecar => &ARROW_SIDECAR_FIELD, + } + } +} + +/// Returns the schema for the delta log +pub fn get_log_schema() -> Schema { + Schema { + fields: Fields::from_iter([ + ActionType::Add.arrow_field().clone(), + ActionType::Cdc.arrow_field().clone(), + ActionType::CommitInfo.arrow_field().clone(), + ActionType::DomainMetadata.arrow_field().clone(), + ActionType::Metadata.arrow_field().clone(), + ActionType::Protocol.arrow_field().clone(), + ActionType::Remove.arrow_field().clone(), + ActionType::Txn.arrow_field().clone(), + ]), + metadata: Default::default(), + } +} diff --git a/crates/deltalake-core/src/kernel/arrow/snapshot.rs b/crates/deltalake-core/src/kernel/arrow/snapshot.rs new file mode 100644 index 0000000000..7f1c17bb1a --- /dev/null +++ b/crates/deltalake-core/src/kernel/arrow/snapshot.rs @@ -0,0 +1,569 @@ +#![allow(dead_code)] +//! Snapshot of a Delta table. + +use std::cmp::Ordering; +use std::sync::Arc; + +use arrow_array::RecordBatch; +use arrow_json::reader::{Decoder, ReaderBuilder}; +use arrow_schema::ArrowError; +use arrow_select::concat::concat_batches; +use bytes::{Buf, Bytes}; +use futures::stream::{Stream, StreamExt, TryStreamExt}; +use object_store::path::Path; +use object_store::{Error as ObjectStoreError, ObjectStore}; +// use parquet::arrow::async_reader::ParquetObjectReader; +// use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; +use serde::{Deserialize, Serialize}; +use std::task::{ready, Poll}; + +use super::checkpoint::{parse_action, parse_actions}; +use super::schemas::get_log_schema; +use crate::kernel::error::{DeltaResult, Error}; +use crate::kernel::snapshot::Snapshot; +use crate::kernel::{Action, ActionType, Add, Metadata, Protocol, StructType}; +use crate::storage::path::{commit_version, is_checkpoint_file, is_commit_file, FileMeta, LogPath}; +use crate::table::config::TableConfig; + +/// A [`Snapshot`] that is dynamically typed. +pub type DynSnapshot = dyn Snapshot; + +#[derive(Debug)] +/// A [`Snapshot`] that is backed by an Arrow [`RecordBatch`]. +pub struct TableStateArrow { + version: i64, + actions: RecordBatch, + metadata: Metadata, + schema: StructType, + protocol: Protocol, +} + +impl TableStateArrow { + /// Create a new [`Snapshot`] from a [`RecordBatch`]. + pub fn try_new(version: i64, actions: RecordBatch) -> DeltaResult { + let metadata = parse_action(&actions, ActionType::Metadata)? + .next() + .and_then(|a| match a { + Action::Metadata(m) => Some(m), + _ => None, + }) + .ok_or(Error::Generic("expected metadata".into()))?; + let protocol = parse_action(&actions, ActionType::Protocol)? + .next() + .and_then(|a| match a { + Action::Protocol(p) => Some(p), + _ => None, + }) + .ok_or(Error::Generic("expected protocol".into()))?; + let schema = serde_json::from_str(&metadata.schema_string)?; + Ok(Self { + version, + actions, + metadata, + protocol, + schema, + }) + } + + /// Load a [`Snapshot`] from a given [`LogPath`]. + pub async fn load( + table_root: LogPath, + object_store: Arc, + version: Option, + ) -> DeltaResult { + let (log_segment, version) = + LogSegment::create(&table_root, object_store.as_ref(), version).await?; + Self::try_new(version, log_segment.load(object_store, None).await?) + } +} + +impl std::fmt::Display for TableStateArrow { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.actions.schema()) + } +} + +impl Snapshot for TableStateArrow { + fn version(&self) -> i64 { + self.version + } + + /// Table [`Metadata`] at this [`Snapshot`]'s version. + fn metadata(&self) -> DeltaResult { + Ok(self.metadata.clone()) + } + + /// Table [`Schema`](crate::kernel::schema::StructType) at this [`Snapshot`]'s version. + fn schema(&self) -> Option<&StructType> { + Some(&self.schema) + } + + /// Table [`Protocol`] at this [`Snapshot`]'s version. + fn protocol(&self) -> DeltaResult { + Ok(self.protocol.clone()) + } + + fn files(&self) -> DeltaResult + Send + '_>> { + Ok(Box::new( + parse_actions(&self.actions, vec![ActionType::Add])?.filter_map(|it| match it { + Action::Add(add) => Some(add), + _ => None, + }), + )) + } + + /// Well known table [configuration](crate::table::config::TableConfig). + fn table_config(&self) -> TableConfig<'_> { + TableConfig(&self.metadata.configuration) + } +} + +const LAST_CHECKPOINT_FILE_NAME: &str = "_last_checkpoint"; +const LOG_FOLDER_NAME: &str = "_delta_log"; +// const SIDECAR_FOLDER_NAME: &str = "_sidecars"; +// const CDC_FOLDER_NAME: &str = "_change_data"; + +#[derive(Debug)] +pub(crate) struct LogSegment { + log_root: LogPath, + /// Reverse order sorted commit files in the log segment + pub(crate) commit_files: Vec, + /// checkpoint files in the log segment. + pub(crate) checkpoint_files: Vec, +} + +impl LogSegment { + pub(crate) async fn create( + table_root: &LogPath, + object_store: &dyn ObjectStore, + version: Option, + ) -> DeltaResult<(Self, i64)> { + let log_url = table_root.child(LOG_FOLDER_NAME).unwrap(); + let log_path = match log_url { + LogPath::ObjectStore(log_path) => log_path, + LogPath::Url(_) => return Err(Error::Generic("Url handling not yet supported".into())), + }; + + // List relevant files from log + let (mut commit_files, checkpoint_files) = match ( + read_last_checkpoint(object_store, &log_path).await?, + version, + ) { + (Some(cp), None) => { + list_log_files_with_checkpoint(&cp, object_store, &log_path).await? + } + (Some(cp), Some(version)) if cp.version >= version => { + list_log_files_with_checkpoint(&cp, object_store, &log_path).await? + } + _ => list_log_files(object_store, &log_path, version).await?, + }; + + // remove all files above requested version + if let Some(version) = version { + commit_files.retain(|meta| { + if let Some(v) = meta.location.commit_version() { + v <= version + } else { + false + } + }); + } + + // get the effective version from chosen files + let version_eff = commit_files + .first() + .or(checkpoint_files.first()) + .and_then(|f| f.location.commit_version()) + .ok_or(Error::MissingVersion)?; // TODO: A more descriptive error + + if let Some(v) = version { + if version_eff != v { + // TODO more descriptive error + return Err(Error::MissingVersion); + } + } + + Ok(( + Self { + log_root: LogPath::ObjectStore(log_path), + commit_files, + checkpoint_files, + }, + version_eff, + )) + } + + pub(crate) fn commit_files(&self) -> impl Iterator { + self.commit_files.iter() + } + + pub async fn load( + &self, + object_store: Arc, + buffer_size: Option, + ) -> DeltaResult { + let buffer_size = buffer_size.unwrap_or_else(num_cpus::get); + let byte_stream = futures::stream::iter(self.commit_files()) + .map(|file| { + let store = object_store.clone(); + async move { + let path = match file.location { + LogPath::ObjectStore(ref path) => path.clone(), + LogPath::Url(_) => { + return Err(Error::Generic("Url handling not yet supported".into())) + } + }; + let data = store + .get(&path) + .await + .map_err(Error::from)? + .bytes() + .await + .map_err(Error::from)?; + + Ok(data) + } + }) + .buffered(buffer_size); + + let log_schema = Arc::new(get_log_schema()); + let decoder = ReaderBuilder::new(log_schema.clone()).build_decoder()?; + + let commit_batch = decode_commit_file_stream(decoder, byte_stream)? + .try_collect::>() + .await?; + let batch = concat_batches(&log_schema, &commit_batch)?; + + // let checkpoint_stream = ParquetRecordBatchStreamBuilder::new(input); + + Ok(batch) + } + + // Read a stream of log data from this log segment. + // + // The log files will be read from most recent to oldest. + // + // `read_schema` is the schema to read the log files with. This can be used + // to project the log files to a subset of the columns. + // + // `predicate` is an optional expression to filter the log files with. + // pub fn replay( + // &self, + // table_client: &dyn TableClient, + // read_schema: Arc, + // predicate: Option, + // ) -> DeltaResult>> { + // let mut commit_files: Vec<_> = self.commit_files().cloned().collect(); + // + // // NOTE this will already sort in reverse order + // commit_files.sort_unstable_by(|a, b| b.location.cmp(&a.location)); + // let json_client = table_client.get_json_handler(); + // let read_contexts = + // json_client.contextualize_file_reads(commit_files, predicate.clone())?; + // let commit_stream = json_client + // .read_json_files(read_contexts, Arc::new(read_schema.as_ref().try_into()?))?; + // + // let parquet_client = table_client.get_parquet_handler(); + // let read_contexts = + // parquet_client.contextualize_file_reads(self.checkpoint_files.clone(), predicate)?; + // let checkpoint_stream = parquet_client + // .read_parquet_files(read_contexts, Arc::new(read_schema.as_ref().try_into()?))?; + // + // let batches = commit_stream.chain(checkpoint_stream); + // + // Ok(batches) + // } +} + +fn decode_commit_file_stream> + Unpin>( + mut decoder: Decoder, + mut input: S, +) -> DeltaResult>> { + let mut buffered = Bytes::new(); + Ok(futures::stream::poll_fn(move |cx| { + loop { + if buffered.is_empty() { + buffered = match ready!(input.poll_next_unpin(cx)) { + Some(Ok(b)) => b, + Some(Err(e)) => { + return Poll::Ready(Some(Err(ArrowError::ExternalError(Box::new(e))))) + } + None => break, + }; + } + let decoded = match decoder.decode(buffered.as_ref()) { + Ok(decoded) => decoded, + Err(e) => return Poll::Ready(Some(Err(e))), + }; + let read = buffered.len(); + buffered.advance(decoded); + if decoded != read { + break; + } + } + Poll::Ready(decoder.flush().transpose()) + })) +} + +/// The last checkpoint file. +#[derive(Debug, Deserialize, Serialize)] +#[serde(rename_all = "camelCase")] +pub struct LastCheckpoint { + /// The version of the table when the last checkpoint was made. + pub version: i64, + /// The number of actions that are stored in the checkpoint. + pub size: i32, + /// The number of fragments if the last checkpoint was written in multiple parts. + pub parts: Option, + /// The number of bytes of the checkpoint. + pub size_in_bytes: Option, + /// The number of AddFile actions in the checkpoint. + pub num_of_add_files: Option, + /// The schema of the checkpoint file. + pub checkpoint_schema: Option, + /// The checksum of the last checkpoint JSON. + pub checksum: Option, +} + +/// Try reading the `_last_checkpoint` file. +/// +/// In case the file is not found, `None` is returned. +async fn read_last_checkpoint( + object_store: &dyn ObjectStore, + log_root: &Path, +) -> DeltaResult> { + let file_path = log_root.child(LAST_CHECKPOINT_FILE_NAME); + match object_store.get(&file_path).await { + Ok(data) => Ok(Some(serde_json::from_slice(&data.bytes().await?)?)), + Err(ObjectStoreError::NotFound { .. }) => Ok(None), + Err(err) => Err(err.into()), + } +} + +/// List all log files after a given checkpoint. +async fn list_log_files_with_checkpoint( + cp: &LastCheckpoint, + fs_client: &dyn ObjectStore, + log_root: &Path, +) -> DeltaResult<(Vec, Vec)> { + let version_prefix = format!("{:020}", cp.version); + let start_from = log_root.child(version_prefix); + + let files = fs_client + .list_with_offset(Some(log_root), &start_from) + .await? + .try_collect::>() + .await? + .into_iter() + .filter_map(|m| commit_version(m.location.filename().unwrap_or_default()).map(|_| m)) + .collect::>(); + + let mut commit_files = files + .iter() + .filter_map(|f| { + if is_commit_file(f.location.filename().unwrap_or_default()) { + Some(f.clone()) + } else { + None + } + }) + .collect::>(); + + // NOTE this will sort in reverse order + commit_files.sort_unstable_by(|a, b| b.location.cmp(&a.location)); + let commit_files = commit_files + .into_iter() + .map(|f| FileMeta { + location: LogPath::ObjectStore(f.location), + size: f.size, + last_modified: f.last_modified.timestamp(), + }) + .collect::>(); + + let checkpoint_files = files + .iter() + .filter_map(|f| { + if is_checkpoint_file(f.location.filename().unwrap_or_default()) { + Some(f.clone()) + } else { + None + } + }) + .collect::>(); + + // TODO raise a proper error + assert_eq!(checkpoint_files.len(), cp.parts.unwrap_or(1) as usize); + let checkpoint_files = checkpoint_files + .into_iter() + .map(|f| FileMeta { + location: LogPath::ObjectStore(f.location), + size: f.size, + last_modified: f.last_modified.timestamp(), + }) + .collect::>(); + + Ok((commit_files, checkpoint_files)) +} + +/// List relevant log files. +/// +/// Relevant files are the max checkpoint found and all subsequent commits. +async fn list_log_files( + fs_client: &dyn ObjectStore, + log_root: &Path, + max_version: Option, +) -> DeltaResult<(Vec, Vec)> { + let start_from = Path::from(format!("{:020}", 0)); + + let max_version = max_version.unwrap_or(i64::MAX); + let mut max_checkpoint_version = -1_i64; + let mut commit_files = Vec::new(); + let mut checkpoint_files = Vec::with_capacity(10); + let mut files = fs_client + .list_with_offset(Some(log_root), &start_from) + .await?; + + while let Some(maybe_meta) = files.next().await { + let meta = maybe_meta?; + let filename = meta.location.filename().unwrap_or_default(); + let version = commit_version(filename).unwrap_or(0) as i64; + if version <= max_version { + if is_commit_file(filename) { + commit_files.push(meta); + } else if is_checkpoint_file(filename) { + match version.cmp(&max_checkpoint_version) { + Ordering::Greater => { + max_checkpoint_version = version; + checkpoint_files.clear(); + checkpoint_files.push(meta); + } + Ordering::Equal => { + checkpoint_files.push(meta); + } + _ => {} + } + } + } + } + + // NOTE this will sort in reverse order + commit_files.sort_unstable_by(|a, b| b.location.cmp(&a.location)); + let commit_files = commit_files + .into_iter() + .map(|f| FileMeta { + location: LogPath::ObjectStore(f.location), + size: f.size, + last_modified: f.last_modified.timestamp(), + }) + .collect::>(); + + let commit_files = commit_files + .into_iter() + .filter(|f| f.location.commit_version().unwrap_or(0) > max_checkpoint_version) + .collect::>(); + + let checkpoint_files = checkpoint_files + .into_iter() + .map(|f| FileMeta { + location: LogPath::ObjectStore(f.location), + size: f.size, + last_modified: f.last_modified.timestamp(), + }) + .collect::>(); + + Ok((commit_files, checkpoint_files)) +} + +#[cfg(test)] +mod tests { + use std::path::PathBuf; + use std::sync::Arc; + + use object_store::local::LocalFileSystem; + use object_store::path::Path; + + use super::*; + use crate::kernel::schema::StructType; + + #[tokio::test] + async fn test_snapshot_read_metadata() { + let path = + std::fs::canonicalize(PathBuf::from("./tests/data/table-with-dv-small/")).unwrap(); + let url = Path::from_filesystem_path(path).unwrap(); + let store = Arc::new(LocalFileSystem::new()); + + let snapshot = TableStateArrow::load(LogPath::ObjectStore(url), store.clone(), Some(1)) + .await + .unwrap(); + + let protocol = snapshot.protocol().unwrap(); + let expected = Protocol { + min_reader_version: 3, + min_writer_version: 7, + reader_features: Some(vec!["deletionVectors".into()].into_iter().collect()), + writer_features: Some(vec!["deletionVectors".into()].into_iter().collect()), + }; + assert_eq!(protocol, expected); + + let schema_string = r#"{"type":"struct","fields":[{"name":"value","type":"integer","nullable":true,"metadata":{}}]}"#; + let expected: StructType = serde_json::from_str(schema_string).unwrap(); + let schema = snapshot.schema().unwrap(); + assert_eq!(schema, &expected); + } + + #[tokio::test] + async fn test_read_table_with_last_checkpoint() { + let path = std::fs::canonicalize(PathBuf::from( + "./tests/data/table-with-dv-small/_delta_log/", + )) + .unwrap(); + let path = Path::from_filesystem_path(path).unwrap(); + + let store = Arc::new(LocalFileSystem::new()); + let cp = read_last_checkpoint(store.as_ref(), &path).await.unwrap(); + assert!(cp.is_none()) + } + + #[tokio::test] + async fn test_read_table_with_checkpoint() { + let path = std::fs::canonicalize(PathBuf::from( + "./tests/data/with_checkpoint_no_last_checkpoint/", + )) + .unwrap(); + let location = Path::from_filesystem_path(path).unwrap(); + let store = Arc::new(LocalFileSystem::new()); + + let (log_segment, version) = LogSegment::create( + &LogPath::ObjectStore(location.clone()), + store.as_ref(), + Some(3), + ) + .await + .unwrap(); + + assert_eq!(version, 3); + assert_eq!(log_segment.checkpoint_files.len(), 1); + assert_eq!( + log_segment.checkpoint_files[0].location.commit_version(), + Some(2) + ); + assert_eq!(log_segment.commit_files.len(), 1); + assert_eq!( + log_segment.commit_files[0].location.commit_version(), + Some(3) + ); + + let (log_segment, version) = + LogSegment::create(&LogPath::ObjectStore(location), store.as_ref(), Some(1)) + .await + .unwrap(); + + assert_eq!(version, 1); + assert_eq!(log_segment.checkpoint_files.len(), 0); + assert_eq!(log_segment.commit_files.len(), 2); + assert_eq!( + log_segment.commit_files[0].location.commit_version(), + Some(1) + ); + } +} diff --git a/crates/deltalake-core/src/kernel/client/expressions.rs b/crates/deltalake-core/src/kernel/client/expressions.rs new file mode 100644 index 0000000000..c18fb5e8de --- /dev/null +++ b/crates/deltalake-core/src/kernel/client/expressions.rs @@ -0,0 +1,320 @@ +//! Default Expression handler. +//! +//! Expression handling based on arrow-rs compute kernels. + +use std::sync::Arc; + +use arrow_arith::boolean::{and, is_null, not, or}; +use arrow_arith::numeric::{add, div, mul, sub}; +use arrow_array::{ + Array, ArrayRef, BinaryArray, BooleanArray, Date32Array, Decimal128Array, Float32Array, + Int32Array, RecordBatch, StringArray, TimestampMicrosecondArray, +}; +use arrow_ord::cmp::{eq, gt, gt_eq, lt, lt_eq, neq}; + +use crate::kernel::error::{DeltaResult, Error}; +use crate::kernel::expressions::{scalars::Scalar, Expression}; +use crate::kernel::expressions::{BinaryOperator, UnaryOperator}; + +// TODO leverage scalars / Datum + +impl Scalar { + /// Convert scalar to arrow array. + pub fn to_array(&self, num_rows: usize) -> ArrayRef { + use Scalar::*; + match self { + Integer(val) => Arc::new(Int32Array::from(vec![*val; num_rows])), + Float(val) => Arc::new(Float32Array::from(vec![*val; num_rows])), + String(val) => Arc::new(StringArray::from(vec![val.clone(); num_rows])), + Boolean(val) => Arc::new(BooleanArray::from(vec![*val; num_rows])), + Timestamp(val) => Arc::new(TimestampMicrosecondArray::from(vec![*val; num_rows])), + Date(val) => Arc::new(Date32Array::from(vec![*val; num_rows])), + Binary(val) => Arc::new(BinaryArray::from(vec![val.as_slice(); num_rows])), + Decimal(val, precision, scale) => Arc::new( + Decimal128Array::from(vec![*val; num_rows]) + .with_precision_and_scale(*precision, *scale) + .unwrap(), + ), + Null(_) => todo!(), + } + } +} + +pub(crate) fn evaluate_expression( + expression: &Expression, + batch: &RecordBatch, +) -> DeltaResult { + match expression { + Expression::Literal(scalar) => Ok(scalar.to_array(batch.num_rows())), + Expression::Column(name) => batch + .column_by_name(name) + .ok_or(Error::MissingColumn(name.clone())) + .cloned(), + Expression::UnaryOperation { op, expr } => { + let arr = evaluate_expression(expr.as_ref(), batch)?; + match op { + UnaryOperator::Not => { + let arr = arr + .as_any() + .downcast_ref::() + .ok_or(Error::Generic("expected boolean array".to_string()))?; + let result = not(arr)?; + Ok(Arc::new(result)) + } + UnaryOperator::IsNull => { + let result = is_null(&arr)?; + Ok(Arc::new(result)) + } + } + } + Expression::BinaryOperation { op, left, right } => { + let left_arr = evaluate_expression(left.as_ref(), batch)?; + let right_arr = evaluate_expression(right.as_ref(), batch)?; + match op { + BinaryOperator::Plus => { + add(&left_arr, &right_arr).map_err(|err| Error::GenericError { + source: Box::new(err), + }) + } + BinaryOperator::Minus => { + sub(&left_arr, &right_arr).map_err(|err| Error::GenericError { + source: Box::new(err), + }) + } + BinaryOperator::Multiply => { + mul(&left_arr, &right_arr).map_err(|err| Error::GenericError { + source: Box::new(err), + }) + } + BinaryOperator::Divide => { + div(&left_arr, &right_arr).map_err(|err| Error::GenericError { + source: Box::new(err), + }) + } + BinaryOperator::LessThan => { + let result = lt(&left_arr, &right_arr).map_err(|err| Error::GenericError { + source: Box::new(err), + })?; + Ok(Arc::new(result)) + } + BinaryOperator::LessThanOrEqual => { + let result = + lt_eq(&left_arr, &right_arr).map_err(|err| Error::GenericError { + source: Box::new(err), + })?; + Ok(Arc::new(result)) + } + BinaryOperator::GreaterThan => { + let result = gt(&left_arr, &right_arr).map_err(|err| Error::GenericError { + source: Box::new(err), + })?; + Ok(Arc::new(result)) + } + BinaryOperator::GreaterThanOrEqual => { + let result = + gt_eq(&left_arr, &right_arr).map_err(|err| Error::GenericError { + source: Box::new(err), + })?; + Ok(Arc::new(result)) + } + BinaryOperator::Equal => { + let result = eq(&left_arr, &right_arr).map_err(|err| Error::GenericError { + source: Box::new(err), + })?; + Ok(Arc::new(result)) + } + BinaryOperator::NotEqual => { + let result = neq(&left_arr, &right_arr).map_err(|err| Error::GenericError { + source: Box::new(err), + })?; + Ok(Arc::new(result)) + } + BinaryOperator::And => { + let left_arr = evaluate_expression(left.as_ref(), batch)?; + let left_arr = left_arr + .as_any() + .downcast_ref::() + .ok_or(Error::Generic("expected boolean array".to_string()))?; + let right_arr = evaluate_expression(right.as_ref(), batch)?; + let right_arr = right_arr + .as_any() + .downcast_ref::() + .ok_or(Error::Generic("expected boolean array".to_string()))?; + let result = and(left_arr, right_arr).map_err(|err| Error::GenericError { + source: Box::new(err), + })?; + Ok(Arc::new(result)) + } + BinaryOperator::Or => { + let left_arr = evaluate_expression(left.as_ref(), batch)?; + let left_arr = left_arr + .as_any() + .downcast_ref::() + .ok_or(Error::Generic("expected boolean array".to_string()))?; + let right_arr = evaluate_expression(right.as_ref(), batch)?; + let right_arr = right_arr + .as_any() + .downcast_ref::() + .ok_or(Error::Generic("expected boolean array".to_string()))?; + let result = or(left_arr, right_arr).map_err(|err| Error::GenericError { + source: Box::new(err), + })?; + Ok(Arc::new(result)) + } + } + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow_array::Int32Array; + use arrow_schema::{DataType, Field, Schema}; + use std::ops::{Add, Div, Mul, Sub}; + + #[test] + fn test_binary_op_scalar() { + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let values = Int32Array::from(vec![1, 2, 3]); + let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(values)]).unwrap(); + let column = Expression::Column("a".to_string()); + + let expression = Box::new(column.clone().add(Expression::Literal(Scalar::Integer(1)))); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(Int32Array::from(vec![2, 3, 4])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().sub(Expression::Literal(Scalar::Integer(1)))); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(Int32Array::from(vec![0, 1, 2])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().mul(Expression::Literal(Scalar::Integer(2)))); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(Int32Array::from(vec![2, 4, 6])); + assert_eq!(results.as_ref(), expected.as_ref()); + + // TODO handle type casting + let expression = Box::new(column.div(Expression::Literal(Scalar::Integer(1)))); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(Int32Array::from(vec![1, 2, 3])); + assert_eq!(results.as_ref(), expected.as_ref()) + } + + #[test] + fn test_binary_op() { + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + ]); + let values = Int32Array::from(vec![1, 2, 3]); + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![Arc::new(values.clone()), Arc::new(values)], + ) + .unwrap(); + let column_a = Expression::Column("a".to_string()); + let column_b = Expression::Column("b".to_string()); + + let expression = Box::new(column_a.clone().add(column_b.clone())); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(Int32Array::from(vec![2, 4, 6])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column_a.clone().sub(column_b.clone())); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(Int32Array::from(vec![0, 0, 0])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column_a.clone().mul(column_b)); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(Int32Array::from(vec![1, 4, 9])); + assert_eq!(results.as_ref(), expected.as_ref()); + } + + #[test] + fn test_binary_cmp() { + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let values = Int32Array::from(vec![1, 2, 3]); + let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(values)]).unwrap(); + let column = Expression::Column("a".to_string()); + let lit = Expression::Literal(Scalar::Integer(2)); + + let expression = Box::new(column.clone().lt(lit.clone())); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![true, false, false])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().lt_eq(lit.clone())); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![true, true, false])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().gt(lit.clone())); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![false, false, true])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().gt_eq(lit.clone())); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![false, true, true])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().eq(lit.clone())); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![false, true, false])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column.clone().ne(lit.clone())); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![true, false, true])); + assert_eq!(results.as_ref(), expected.as_ref()); + } + + #[test] + fn test_logical() { + let schema = Schema::new(vec![ + Field::new("a", DataType::Boolean, false), + Field::new("b", DataType::Boolean, false), + ]); + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(BooleanArray::from(vec![true, false])), + Arc::new(BooleanArray::from(vec![false, true])), + ], + ) + .unwrap(); + let column_a = Expression::Column("a".to_string()); + let column_b = Expression::Column("b".to_string()); + + let expression = Box::new(column_a.clone().and(column_b.clone())); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![false, false])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new( + column_a + .clone() + .and(Expression::literal(Scalar::Boolean(true))), + ); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![true, false])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new(column_a.clone().or(column_b)); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![true, true])); + assert_eq!(results.as_ref(), expected.as_ref()); + + let expression = Box::new( + column_a + .clone() + .or(Expression::literal(Scalar::Boolean(false))), + ); + let results = evaluate_expression(&expression, &batch).unwrap(); + let expected = Arc::new(BooleanArray::from(vec![true, false])); + assert_eq!(results.as_ref(), expected.as_ref()); + } +} diff --git a/crates/deltalake-core/src/kernel/client/mod.rs b/crates/deltalake-core/src/kernel/client/mod.rs new file mode 100644 index 0000000000..038a51d794 --- /dev/null +++ b/crates/deltalake-core/src/kernel/client/mod.rs @@ -0,0 +1,40 @@ +//! Delta kernel client implementation. +use std::sync::Arc; + +use arrow_array::RecordBatch; + +use self::expressions::evaluate_expression; +use crate::kernel::error::DeltaResult; +use crate::kernel::expressions::Expression; +use crate::kernel::schema::SchemaRef; + +pub mod expressions; + +/// Interface for implementing an Expression evaluator. +/// +/// It contains one Expression which can be evaluated on multiple ColumnarBatches. +/// Connectors can implement this interface to optimize the evaluation using the +/// connector specific capabilities. +pub trait ExpressionEvaluator { + /// Evaluate the expression on given ColumnarBatch data. + /// + /// Contains one value for each row of the input. + /// The data type of the output is same as the type output of the expression this evaluator is using. + fn evaluate(&self, batch: &RecordBatch, output_schema: SchemaRef) -> DeltaResult; +} + +#[derive(Debug)] +/// Expression evaluator based on arrow compute kernels. +pub struct ArrowExpressionEvaluator { + _input_schema: SchemaRef, + expression: Box, +} + +impl ExpressionEvaluator for ArrowExpressionEvaluator { + fn evaluate(&self, batch: &RecordBatch, output_schema: SchemaRef) -> DeltaResult { + Ok(RecordBatch::try_new( + Arc::new(output_schema.as_ref().try_into()?), + vec![evaluate_expression(&self.expression, batch)?], + )?) + } +} diff --git a/crates/deltalake-core/src/kernel/error.rs b/crates/deltalake-core/src/kernel/error.rs index 8ec799ca96..a37dbdae67 100644 --- a/crates/deltalake-core/src/kernel/error.rs +++ b/crates/deltalake-core/src/kernel/error.rs @@ -23,9 +23,8 @@ pub enum Error { #[error("Arrow error: {0}")] Parquet(#[from] parquet::errors::ParquetError), - #[cfg(feature = "object_store")] #[error("Error interacting with object store: {0}")] - ObjectStore(object_store::Error), + ObjectStore(#[from] object_store::Error), #[error("File not found: {0}")] FileNotFound(String), diff --git a/crates/deltalake-core/src/kernel/expressions/mod.rs b/crates/deltalake-core/src/kernel/expressions/mod.rs new file mode 100644 index 0000000000..80c0a72cf3 --- /dev/null +++ b/crates/deltalake-core/src/kernel/expressions/mod.rs @@ -0,0 +1,283 @@ +//! expressions. + +use std::{ + collections::HashSet, + fmt::{Display, Formatter}, +}; + +use self::scalars::Scalar; + +pub mod scalars; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +/// A binary operator. +pub enum BinaryOperator { + /// Logical And + And, + /// Logical Or + Or, + /// Arithmetic Plus + Plus, + /// Arithmetic Minus + Minus, + /// Arithmetic Multiply + Multiply, + /// Arithmetic Divide + Divide, + /// Comparison Less Than + LessThan, + /// Comparison Less Than Or Equal + LessThanOrEqual, + /// Comparison Greater Than + GreaterThan, + /// Comparison Greater Than Or Equal + GreaterThanOrEqual, + /// Comparison Equal + Equal, + /// Comparison Not Equal + NotEqual, +} + +impl Display for BinaryOperator { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + Self::And => write!(f, "AND"), + Self::Or => write!(f, "OR"), + Self::Plus => write!(f, "+"), + Self::Minus => write!(f, "-"), + Self::Multiply => write!(f, "*"), + Self::Divide => write!(f, "/"), + Self::LessThan => write!(f, "<"), + Self::LessThanOrEqual => write!(f, "<="), + Self::GreaterThan => write!(f, ">"), + Self::GreaterThanOrEqual => write!(f, ">="), + Self::Equal => write!(f, "="), + Self::NotEqual => write!(f, "!="), + } + } +} + +#[derive(Debug, Clone, PartialEq)] +/// A unary operator. +pub enum UnaryOperator { + /// Unary Not + Not, + /// Unary Is Null + IsNull, +} + +/// A SQL expression. +/// +/// These expressions do not track or validate data types, other than the type +/// of literals. It is up to the expression evaluator to validate the +/// expression against a schema and add appropriate casts as required. +#[derive(Debug, Clone, PartialEq)] +pub enum Expression { + /// A literal value. + Literal(Scalar), + /// A column reference by name. + Column(String), + /// A binary operation. + BinaryOperation { + /// The operator. + op: BinaryOperator, + /// The left-hand side of the operation. + left: Box, + /// The right-hand side of the operation. + right: Box, + }, + /// A unary operation. + UnaryOperation { + /// The operator. + op: UnaryOperator, + /// The expression. + expr: Box, + }, + // TODO: support more expressions, such as IS IN, LIKE, etc. +} + +impl Display for Expression { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + Self::Literal(l) => write!(f, "{}", l), + Self::Column(name) => write!(f, "Column({})", name), + Self::BinaryOperation { op, left, right } => { + match op { + // OR requires parentheses + BinaryOperator::Or => write!(f, "({} OR {})", left, right), + _ => write!(f, "{} {} {}", left, op, right), + } + } + Self::UnaryOperation { op, expr } => match op { + UnaryOperator::Not => write!(f, "NOT {}", expr), + UnaryOperator::IsNull => write!(f, "{} IS NULL", expr), + }, + } + } +} + +impl Expression { + /// Returns a set of columns referenced by this expression. + pub fn references(&self) -> HashSet<&str> { + let mut set = HashSet::new(); + + for expr in self.walk() { + if let Self::Column(name) = expr { + set.insert(name.as_str()); + } + } + + set + } + + /// Create an new expression for a column reference + pub fn column(name: impl Into) -> Self { + Self::Column(name.into()) + } + + /// Create a new expression for a literal value + pub fn literal(value: impl Into) -> Self { + Self::Literal(value.into()) + } + + fn binary_op_impl(self, other: Self, op: BinaryOperator) -> Self { + Self::BinaryOperation { + op, + left: Box::new(self), + right: Box::new(other), + } + } + + /// Create a new expression `self == other` + pub fn eq(self, other: Self) -> Self { + self.binary_op_impl(other, BinaryOperator::Equal) + } + + /// Create a new expression `self != other` + pub fn ne(self, other: Self) -> Self { + self.binary_op_impl(other, BinaryOperator::NotEqual) + } + + /// Create a new expression `self < other` + pub fn lt(self, other: Self) -> Self { + self.binary_op_impl(other, BinaryOperator::LessThan) + } + + /// Create a new expression `self > other` + pub fn gt(self, other: Self) -> Self { + self.binary_op_impl(other, BinaryOperator::GreaterThan) + } + + /// Create a new expression `self >= other` + pub fn gt_eq(self, other: Self) -> Self { + self.binary_op_impl(other, BinaryOperator::GreaterThanOrEqual) + } + + /// Create a new expression `self <= other` + pub fn lt_eq(self, other: Self) -> Self { + self.binary_op_impl(other, BinaryOperator::LessThanOrEqual) + } + + /// Create a new expression `self AND other` + pub fn and(self, other: Self) -> Self { + self.binary_op_impl(other, BinaryOperator::And) + } + + /// Create a new expression `self OR other` + pub fn or(self, other: Self) -> Self { + self.binary_op_impl(other, BinaryOperator::Or) + } + + fn walk(&self) -> impl Iterator + '_ { + let mut stack = vec![self]; + std::iter::from_fn(move || { + let expr = stack.pop()?; + match expr { + Self::Literal(_) => {} + Self::Column { .. } => {} + Self::BinaryOperation { left, right, .. } => { + stack.push(left); + stack.push(right); + } + Self::UnaryOperation { expr, .. } => { + stack.push(expr); + } + } + Some(expr) + }) + } +} + +impl std::ops::Add for Expression { + type Output = Self; + + fn add(self, rhs: Expression) -> Self::Output { + self.binary_op_impl(rhs, BinaryOperator::Plus) + } +} + +impl std::ops::Sub for Expression { + type Output = Self; + + fn sub(self, rhs: Expression) -> Self::Output { + self.binary_op_impl(rhs, BinaryOperator::Minus) + } +} + +impl std::ops::Mul for Expression { + type Output = Self; + + fn mul(self, rhs: Expression) -> Self::Output { + self.binary_op_impl(rhs, BinaryOperator::Multiply) + } +} + +impl std::ops::Div for Expression { + type Output = Self; + + fn div(self, rhs: Expression) -> Self::Output { + self.binary_op_impl(rhs, BinaryOperator::Divide) + } +} + +#[cfg(test)] +mod tests { + use super::Expression as Expr; + + #[test] + fn test_expression_format() { + let col_ref = Expr::column("x"); + let cases = [ + (col_ref.clone(), "Column(x)"), + (col_ref.clone().eq(Expr::literal(2)), "Column(x) = 2"), + ( + col_ref + .clone() + .gt_eq(Expr::literal(2)) + .and(col_ref.clone().lt_eq(Expr::literal(10))), + "Column(x) >= 2 AND Column(x) <= 10", + ), + ( + col_ref + .clone() + .gt(Expr::literal(2)) + .or(col_ref.clone().lt(Expr::literal(10))), + "(Column(x) > 2 OR Column(x) < 10)", + ), + ( + (col_ref.clone() - Expr::literal(4)).lt(Expr::literal(10)), + "Column(x) - 4 < 10", + ), + ( + (col_ref.clone() + Expr::literal(4)) / Expr::literal(10) * Expr::literal(42), + "Column(x) + 4 / 10 * 42", + ), + (col_ref.eq(Expr::literal("foo")), "Column(x) = 'foo'"), + ]; + + for (expr, expected) in cases { + let result = format!("{}", expr); + assert_eq!(result, expected); + } + } +} diff --git a/crates/deltalake-core/src/kernel/expressions/scalars.rs b/crates/deltalake-core/src/kernel/expressions/scalars.rs new file mode 100644 index 0000000000..175470e19e --- /dev/null +++ b/crates/deltalake-core/src/kernel/expressions/scalars.rs @@ -0,0 +1,135 @@ +//! Scalar values for use in expressions. + +use std::{ + cmp::Ordering, + fmt::{Display, Formatter}, +}; + +use crate::kernel::schema::{DataType, PrimitiveType}; + +/// A single value, which can be null. Used for representing literal values +/// in [Expressions][crate::kernel::expressions::Expression]. +#[derive(Debug, Clone, PartialEq)] +pub enum Scalar { + /// A 32-bit integer. + Integer(i32), + /// A 64-bit floating point number. + Float(f32), + /// A string. + String(String), + /// A boolean. + Boolean(bool), + /// A timestamp. + Timestamp(i64), + /// A date. + Date(i32), + /// A binary value. + Binary(Vec), + /// A decimal value. + Decimal(i128, u8, i8), + /// A null value. + Null(DataType), +} + +impl Scalar { + /// Returns the [DataType] of the scalar. + pub fn data_type(&self) -> DataType { + match self { + Self::Integer(_) => DataType::Primitive(PrimitiveType::Integer), + Self::Float(_) => DataType::Primitive(PrimitiveType::Float), + Self::String(_) => DataType::Primitive(PrimitiveType::String), + Self::Boolean(_) => DataType::Primitive(PrimitiveType::Boolean), + Self::Timestamp(_) => DataType::Primitive(PrimitiveType::Timestamp), + Self::Date(_) => DataType::Primitive(PrimitiveType::Date), + Self::Binary(_) => DataType::Primitive(PrimitiveType::Binary), + Self::Decimal(_, precision, scale) => { + DataType::decimal(*precision as usize, *scale as usize) + } + Self::Null(data_type) => data_type.clone(), + } + } +} + +impl Display for Scalar { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + Self::Integer(i) => write!(f, "{}", i), + Self::Float(fl) => write!(f, "{}", fl), + Self::String(s) => write!(f, "'{}'", s), + Self::Boolean(b) => write!(f, "{}", b), + Self::Timestamp(ts) => write!(f, "{}", ts), + Self::Date(d) => write!(f, "{}", d), + Self::Binary(b) => write!(f, "{:?}", b), + Self::Decimal(value, _, scale) => match scale.cmp(&0) { + Ordering::Equal => { + write!(f, "{}", value) + } + Ordering::Greater => { + let scalar_multiple = 10_i128.pow(*scale as u32); + write!(f, "{}", value / scalar_multiple)?; + write!(f, ".")?; + write!( + f, + "{:0>scale$}", + value % scalar_multiple, + scale = *scale as usize + ) + } + Ordering::Less => { + write!(f, "{}", value)?; + for _ in 0..(scale.abs()) { + write!(f, "0")?; + } + Ok(()) + } + }, + Self::Null(_) => write!(f, "null"), + } + } +} + +impl From for Scalar { + fn from(i: i32) -> Self { + Self::Integer(i) + } +} + +impl From for Scalar { + fn from(b: bool) -> Self { + Self::Boolean(b) + } +} + +impl From<&str> for Scalar { + fn from(s: &str) -> Self { + Self::String(s.into()) + } +} + +impl From for Scalar { + fn from(value: String) -> Self { + Self::String(value) + } +} + +// TODO: add more From impls + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_decimal_display() { + let s = Scalar::Decimal(123456789, 9, 2); + assert_eq!(s.to_string(), "1234567.89"); + + let s = Scalar::Decimal(123456789, 9, 0); + assert_eq!(s.to_string(), "123456789"); + + let s = Scalar::Decimal(123456789, 9, 9); + assert_eq!(s.to_string(), "0.123456789"); + + let s = Scalar::Decimal(123, 9, -3); + assert_eq!(s.to_string(), "123000"); + } +} diff --git a/crates/deltalake-core/src/kernel/mod.rs b/crates/deltalake-core/src/kernel/mod.rs index 7785c273f9..a550f76731 100644 --- a/crates/deltalake-core/src/kernel/mod.rs +++ b/crates/deltalake-core/src/kernel/mod.rs @@ -1,9 +1,16 @@ //! Kernel module pub mod actions; +#[cfg(all(feature = "arrow", feature = "parquet"))] +pub mod arrow; +#[cfg(feature = "arrow")] +pub mod client; pub mod error; +pub mod expressions; pub mod schema; +pub mod snapshot; pub use actions::*; pub use error::*; +pub use expressions::*; pub use schema::*; diff --git a/crates/deltalake-core/src/kernel/snapshot.rs b/crates/deltalake-core/src/kernel/snapshot.rs new file mode 100644 index 0000000000..677a7e6ac5 --- /dev/null +++ b/crates/deltalake-core/src/kernel/snapshot.rs @@ -0,0 +1,26 @@ +//! Snapshot of a Delta table. + +use crate::kernel::error::DeltaResult; +use crate::kernel::{Add, Metadata, Protocol, StructType}; +use crate::table::config::TableConfig; + +/// A snapshot of a Delta table at a given version. +pub trait Snapshot: std::fmt::Display + Send + Sync + std::fmt::Debug + 'static { + /// The version of the table at this [`Snapshot`]. + fn version(&self) -> i64; + + /// Table [`Schema`](crate::kernel::schema::StructType) at this [`Snapshot`]'s version. + fn schema(&self) -> Option<&StructType>; + + /// Table [`Metadata`] at this [`Snapshot`]'s version. + fn metadata(&self) -> DeltaResult; + + /// Table [`Protocol`] at this [`Snapshot`]'s version. + fn protocol(&self) -> DeltaResult; + + /// Iterator over the [`Add`] actions at this [`Snapshot`]'s version. + fn files(&self) -> DeltaResult + Send + '_>>; + + /// Well known table [configuration](crate::table::config::TableConfig). + fn table_config(&self) -> TableConfig<'_>; +} diff --git a/crates/deltalake-core/src/logstore/mod.rs b/crates/deltalake-core/src/logstore/mod.rs index 7f1009b1de..fe6d001004 100644 --- a/crates/deltalake-core/src/logstore/mod.rs +++ b/crates/deltalake-core/src/logstore/mod.rs @@ -265,7 +265,7 @@ async fn get_latest_version(log_store: &dyn LogStore, current_version: i64) -> D return Err(DeltaTableError::not_a_table(log_store.root_uri())); } - Ok::(max_version) + Ok(max_version) } .await?; Ok(version) diff --git a/crates/deltalake-core/src/operations/create.rs b/crates/deltalake-core/src/operations/create.rs index 71398faf97..84c2e03627 100644 --- a/crates/deltalake-core/src/operations/create.rs +++ b/crates/deltalake-core/src/operations/create.rs @@ -7,8 +7,7 @@ use std::sync::Arc; use futures::future::BoxFuture; use serde_json::{Map, Value}; -use super::transaction::commit; -use super::{MAX_SUPPORTED_READER_VERSION, MAX_SUPPORTED_WRITER_VERSION}; +use super::transaction::{commit, PROTOCOL}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::kernel::{Action, DataType, Metadata, Protocol, StructField, StructType}; use crate::logstore::{LogStore, LogStoreRef}; @@ -245,8 +244,8 @@ impl CreateBuilder { _ => unreachable!(), }) .unwrap_or_else(|| Protocol { - min_reader_version: MAX_SUPPORTED_READER_VERSION, - min_writer_version: MAX_SUPPORTED_WRITER_VERSION, + min_reader_version: PROTOCOL.default_reader_version(), + min_writer_version: PROTOCOL.default_writer_version(), writer_features: None, reader_features: None, }); @@ -391,8 +390,14 @@ mod tests { .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.get_min_reader_version(), + PROTOCOL.default_reader_version() + ); + assert_eq!( + table.get_min_writer_version(), + PROTOCOL.default_writer_version() + ); assert_eq!(table.schema().unwrap(), &schema); // check we can overwrite default settings via adding actions diff --git a/crates/deltalake-core/src/operations/delete.rs b/crates/deltalake-core/src/operations/delete.rs index bd361c9707..f6c26575e3 100644 --- a/crates/deltalake-core/src/operations/delete.rs +++ b/crates/deltalake-core/src/operations/delete.rs @@ -34,6 +34,8 @@ use parquet::file::properties::WriterProperties; use serde::Serialize; use serde_json::Value; +use super::datafusion_utils::Expression; +use super::transaction::PROTOCOL; use crate::delta_datafusion::expr::fmt_expr_to_sql; use crate::delta_datafusion::{find_files, register_store, DeltaScanBuilder}; use crate::errors::{DeltaResult, DeltaTableError}; @@ -44,8 +46,6 @@ use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; use crate::DeltaTable; -use super::datafusion_utils::Expression; - /// Delete Records from the Delta Table. /// See this module's documentation for more information pub struct DeleteBuilder { @@ -135,7 +135,7 @@ async fn excute_non_empty_expr( // For each identified file perform a parquet scan + filter + limit (1) + count. // If returned count is not zero then append the file to be rewritten and removed from the log. Otherwise do nothing to the file. - let input_schema = snapshot.input_schema()?; + let input_schema = snapshot.arrow_schema(false)?; let input_dfschema: DFSchema = input_schema.clone().as_ref().clone().try_into()?; let table_partition_cols = snapshot @@ -274,6 +274,8 @@ impl std::future::IntoFuture for DeleteBuilder { let mut this = self; Box::pin(async move { + PROTOCOL.can_write_to(&this.snapshot)?; + let state = this.state.unwrap_or_else(|| { let session = SessionContext::new(); diff --git a/crates/deltalake-core/src/operations/load.rs b/crates/deltalake-core/src/operations/load.rs index 1a4c5c4cc6..c495d63a1b 100644 --- a/crates/deltalake-core/src/operations/load.rs +++ b/crates/deltalake-core/src/operations/load.rs @@ -6,6 +6,7 @@ use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; use futures::future::BoxFuture; +use super::transaction::PROTOCOL; use crate::errors::{DeltaResult, DeltaTableError}; use crate::logstore::LogStoreRef; use crate::table::state::DeltaTableState; @@ -46,8 +47,10 @@ impl std::future::IntoFuture for LoadBuilder { let this = self; Box::pin(async move { + PROTOCOL.can_read_from(&this.snapshot)?; + let table = DeltaTable::new_with_state(this.log_store, this.snapshot); - let schema = table.state.arrow_schema()?; + let schema = table.state.arrow_schema(true)?; let projection = this .columns .map(|cols| { diff --git a/crates/deltalake-core/src/operations/merge.rs b/crates/deltalake-core/src/operations/merge.rs index d38ddf0efb..239018e819 100644 --- a/crates/deltalake-core/src/operations/merge.rs +++ b/crates/deltalake-core/src/operations/merge.rs @@ -64,7 +64,7 @@ use serde::Serialize; use serde_json::Value; use super::datafusion_utils::{into_expr, maybe_into_expr, Expression}; -use super::transaction::commit; +use super::transaction::{commit, PROTOCOL}; use crate::delta_datafusion::expr::{fmt_expr_to_sql, parse_predicate_expression}; use crate::delta_datafusion::{register_store, DeltaScanBuilder}; use crate::kernel::{Action, Remove}; @@ -591,7 +591,7 @@ async fn execute( let target = Arc::new( DeltaScanBuilder::new(snapshot, log_store.clone(), &state) - .with_schema(snapshot.input_schema()?) + .with_schema(snapshot.arrow_schema(false)?) .build() .await?, ); @@ -1208,6 +1208,8 @@ impl std::future::IntoFuture for MergeBuilder { let mut this = self; Box::pin(async move { + PROTOCOL.can_write_to(&this.snapshot)?; + let state = this.state.unwrap_or_else(|| { let session = SessionContext::new(); diff --git a/crates/deltalake-core/src/operations/mod.rs b/crates/deltalake-core/src/operations/mod.rs index abf9753648..88f46506aa 100644 --- a/crates/deltalake-core/src/operations/mod.rs +++ b/crates/deltalake-core/src/operations/mod.rs @@ -50,11 +50,6 @@ pub mod write; #[cfg(all(feature = "arrow", feature = "parquet"))] pub mod writer; -/// 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 DeltaOps(pub DeltaTable); diff --git a/crates/deltalake-core/src/operations/optimize.rs b/crates/deltalake-core/src/operations/optimize.rs index d86ef6f3e5..ca4fe7f386 100644 --- a/crates/deltalake-core/src/operations/optimize.rs +++ b/crates/deltalake-core/src/operations/optimize.rs @@ -38,7 +38,7 @@ use parquet::errors::ParquetError; use parquet::file::properties::WriterProperties; use serde::{Deserialize, Serialize}; -use super::transaction::commit; +use super::transaction::{commit, PROTOCOL}; use super::writer::{PartitionWriter, PartitionWriterConfig}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::kernel::{Action, Remove}; @@ -260,6 +260,8 @@ impl<'a> std::future::IntoFuture for OptimizeBuilder<'a> { let this = self; Box::pin(async move { + PROTOCOL.can_write_to(&this.snapshot)?; + let writer_properties = this.writer_properties.unwrap_or_else(|| { WriterProperties::builder() .set_compression(Compression::ZSTD(ZstdLevel::try_new(4).unwrap())) diff --git a/crates/deltalake-core/src/operations/restore.rs b/crates/deltalake-core/src/operations/restore.rs index c391de6f04..be43bacf5f 100644 --- a/crates/deltalake-core/src/operations/restore.rs +++ b/crates/deltalake-core/src/operations/restore.rs @@ -245,7 +245,6 @@ async fn execute( datetime: datetime_to_restore.map(|time| -> i64 { time.timestamp_millis() }), }, &actions, - &snapshot, None, ) .await?; diff --git a/crates/deltalake-core/src/operations/transaction/conflict_checker.rs b/crates/deltalake-core/src/operations/transaction/conflict_checker.rs index 6cefe848b8..d635760a18 100644 --- a/crates/deltalake-core/src/operations/transaction/conflict_checker.rs +++ b/crates/deltalake-core/src/operations/transaction/conflict_checker.rs @@ -13,7 +13,7 @@ use crate::table::config::IsolationLevel; use crate::table::state::DeltaTableState; #[cfg(feature = "datafusion")] -use super::state::AddContainer; +use crate::delta_datafusion::pruning::AddContainer; #[cfg(feature = "datafusion")] use datafusion_expr::Expr; #[cfg(feature = "datafusion")] @@ -467,7 +467,7 @@ impl<'a> ConflictChecker<'a> { &self.txn_info.read_predicates, self.txn_info.read_whole_table(), ) { - let arrow_schema = self.txn_info.read_snapshot.arrow_schema().map_err(|err| { + let arrow_schema = self.txn_info.read_snapshot.arrow_schema(true).map_err(|err| { CommitConflictError::CorruptedState { source: Box::new(err), } diff --git a/crates/deltalake-core/src/operations/transaction/mod.rs b/crates/deltalake-core/src/operations/transaction/mod.rs index e5e808d2d5..c92372df54 100644 --- a/crates/deltalake-core/src/operations/transaction/mod.rs +++ b/crates/deltalake-core/src/operations/transaction/mod.rs @@ -7,21 +7,23 @@ use object_store::path::Path; use object_store::{Error as ObjectStoreError, ObjectStore}; use serde_json::Value; +use self::conflict_checker::{CommitConflictError, TransactionInfo, WinningCommitSummary}; use crate::crate_version; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::{Action, CommitInfo}; +use crate::kernel::{Action, CommitInfo, ReaderFeatures, WriterFeatures}; use crate::logstore::LogStore; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; +pub use self::protocol::INSTANCE as PROTOCOL; + mod conflict_checker; +mod protocol; #[cfg(feature = "datafusion")] mod state; #[cfg(test)] pub(crate) mod test_utils; -use self::conflict_checker::{CommitConflictError, TransactionInfo, WinningCommitSummary}; - const DELTA_LOG_FOLDER: &str = "_delta_log"; /// Error raised while commititng transaction @@ -45,17 +47,40 @@ pub enum TransactionError { #[from] source: ObjectStoreError, }, + /// Error returned when a commit conflict ocurred #[error("Failed to commit transaction: {0}")] CommitConflict(#[from] CommitConflictError), + /// Error returned when maximum number of commit trioals is exceeded #[error("Failed to commit transaction: {0}")] MaxCommitAttempts(i32), + /// The transaction includes Remove action with data change but Delta table is append-only #[error( "The transaction includes Remove action with data change but Delta table is append-only" )] DeltaTableAppendOnly, + + /// Error returned when unsupported reader features are required + #[error("Unsupported reader features required: {0:?}")] + UnsupportedReaderFeatures(Vec), + + /// Error returned when unsupported writer features are required + #[error("Unsupported writer features required: {0:?}")] + UnsupportedWriterFeatures(Vec), + + /// Error returned when writer features are required but not specified + #[error("Writer features must be specified for writerversion >= 7")] + WriterFeaturesRequired, + + /// Error returned when reader features are required but not specified + #[error("Reader features must be specified for reader version >= 3")] + ReaderFeaturesRequired, + + /// Error returned when the transaction is not valid + #[error("Kernel Error: {0}")] + Kernel(#[from] crate::kernel::error::Error), } impl From for DeltaTableError { @@ -76,18 +101,9 @@ impl From for DeltaTableError { // Convert actions to their json representation fn log_entry_from_actions<'a>( actions: impl IntoIterator, - read_snapshot: &DeltaTableState, ) -> Result { - let append_only = read_snapshot.table_config().append_only(); let mut jsons = Vec::::new(); for action in actions { - if append_only { - if let Action::Remove(remove) = action { - if remove.data_change { - return Err(TransactionError::DeltaTableAppendOnly); - } - } - } let json = serde_json::to_string(action) .map_err(|e| TransactionError::SerializeLogJson { json_err: e })?; jsons.push(json); @@ -98,7 +114,6 @@ fn log_entry_from_actions<'a>( pub(crate) fn get_commit_bytes( operation: &DeltaOperation, actions: &Vec, - read_snapshot: &DeltaTableState, app_metadata: Option>, ) -> Result { if !actions.iter().any(|a| matches!(a, Action::CommitInfo(..))) { @@ -117,13 +132,9 @@ pub(crate) fn get_commit_bytes( actions .iter() .chain(std::iter::once(&Action::CommitInfo(commit_info))), - read_snapshot, )?)) } else { - Ok(bytes::Bytes::from(log_entry_from_actions( - actions, - read_snapshot, - )?)) + Ok(bytes::Bytes::from(log_entry_from_actions(actions)?)) } } @@ -135,11 +146,10 @@ pub(crate) async fn prepare_commit<'a>( storage: &dyn ObjectStore, operation: &DeltaOperation, actions: &Vec, - read_snapshot: &DeltaTableState, app_metadata: Option>, ) -> Result { // Serialize all actions that are part of this log entry. - let log_entry = get_commit_bytes(operation, actions, read_snapshot, app_metadata)?; + let log_entry = get_commit_bytes(operation, actions, app_metadata)?; // 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. @@ -185,11 +195,11 @@ pub async fn commit_with_retries( app_metadata: Option>, max_retries: usize, ) -> DeltaResult { + PROTOCOL.can_commit(read_snapshot, actions)?; let tmp_commit = prepare_commit( log_store.object_store().as_ref(), &operation, actions, - read_snapshot, app_metadata, ) .await?; @@ -240,12 +250,9 @@ pub async fn commit_with_retries( mod tests { use std::{collections::HashMap, sync::Arc}; - use self::test_utils::{create_remove_action, init_table_actions}; + use self::test_utils::init_table_actions; use super::*; - use crate::{ - logstore::default_logstore::DefaultLogStore, storage::commit_uri_from_version, - DeltaConfigKey, - }; + use crate::{logstore::default_logstore::DefaultLogStore, storage::commit_uri_from_version}; use object_store::memory::InMemory; use url::Url; @@ -260,35 +267,12 @@ mod tests { #[test] fn test_log_entry_from_actions() { let actions = init_table_actions(None); - let state = DeltaTableState::from_actions(actions.clone(), 0).unwrap(); - let entry = log_entry_from_actions(&actions, &state).unwrap(); + let entry = log_entry_from_actions(&actions).unwrap(); let lines: Vec<_> = entry.lines().collect(); // writes every action to a line assert_eq!(actions.len(), lines.len()) } - fn remove_action_exists_when_delta_table_is_append_only( - data_change: bool, - ) -> Result { - let remove = create_remove_action("test_append_only", data_change); - let mut actions = init_table_actions(Some(HashMap::from([( - DeltaConfigKey::AppendOnly.as_ref().to_string(), - Some("true".to_string()), - )]))); - actions.push(remove); - let state = - DeltaTableState::from_actions(actions.clone(), 0).expect("Failed to get table state"); - log_entry_from_actions(&actions, &state) - } - - #[test] - fn test_remove_action_exists_when_delta_table_is_append_only() { - let _err = remove_action_exists_when_delta_table_is_append_only(true) - .expect_err("Remove action is included when Delta table is append-only. Should error"); - let _actions = remove_action_exists_when_delta_table_is_append_only(false) - .expect("Data is not changed by the Remove action. Should succeed"); - } - #[tokio::test] async fn test_try_commit_transaction() { let store = Arc::new(InMemory::new()); diff --git a/crates/deltalake-core/src/operations/transaction/protocol.rs b/crates/deltalake-core/src/operations/transaction/protocol.rs new file mode 100644 index 0000000000..e3f09a488d --- /dev/null +++ b/crates/deltalake-core/src/operations/transaction/protocol.rs @@ -0,0 +1,405 @@ +use std::collections::HashSet; + +use lazy_static::lazy_static; +use once_cell::sync::Lazy; + +use super::TransactionError; +use crate::kernel::snapshot::Snapshot; +use crate::kernel::{Action, ReaderFeatures, WriterFeatures}; +use crate::table::state::DeltaTableState; + +lazy_static! { + static ref READER_V2: HashSet = + HashSet::from_iter([ReaderFeatures::ColumnMapping]); + static ref WRITER_V2: HashSet = + HashSet::from_iter([WriterFeatures::AppendOnly, WriterFeatures::Invariants]); + static ref WRITER_V3: HashSet = HashSet::from_iter([ + WriterFeatures::AppendOnly, + WriterFeatures::Invariants, + WriterFeatures::CheckConstraints + ]); + static ref WRITER_V4: HashSet = HashSet::from_iter([ + WriterFeatures::AppendOnly, + WriterFeatures::Invariants, + WriterFeatures::CheckConstraints, + WriterFeatures::ChangeDataFeed, + WriterFeatures::GeneratedColumns + ]); + static ref WRITER_V5: HashSet = HashSet::from_iter([ + WriterFeatures::AppendOnly, + WriterFeatures::Invariants, + WriterFeatures::CheckConstraints, + WriterFeatures::ChangeDataFeed, + WriterFeatures::GeneratedColumns, + WriterFeatures::ColumnMapping, + ]); + static ref WRITER_V6: HashSet = HashSet::from_iter([ + WriterFeatures::AppendOnly, + WriterFeatures::Invariants, + WriterFeatures::CheckConstraints, + WriterFeatures::ChangeDataFeed, + WriterFeatures::GeneratedColumns, + WriterFeatures::ColumnMapping, + WriterFeatures::IdentityColumns, + ]); +} + +pub struct ProtocolChecker { + reader_features: HashSet, + writer_features: HashSet, +} + +impl ProtocolChecker { + /// Create a new protocol checker. + pub fn new( + reader_features: HashSet, + writer_features: HashSet, + ) -> Self { + Self { + reader_features, + writer_features, + } + } + + pub fn default_reader_version(&self) -> i32 { + 1 + } + + pub fn default_writer_version(&self) -> i32 { + 2 + } + + /// Check if delta-rs can read form the given delta table. + pub fn can_read_from(&self, snapshot: &dyn Snapshot) -> Result<(), TransactionError> { + let required_features: Option> = + match snapshot.protocol()?.min_reader_version { + 0 | 1 => None, + 2 => Some(READER_V2.clone()), + _ => snapshot.protocol()?.reader_features, + }; + if let Some(features) = required_features { + let mut diff = features.difference(&self.reader_features).peekable(); + if diff.peek().is_some() { + return Err(TransactionError::UnsupportedReaderFeatures( + diff.cloned().collect(), + )); + } + }; + Ok(()) + } + + /// Check if delta-rs can write to the given delta table. + pub fn can_write_to(&self, snapshot: &DeltaTableState) -> Result<(), TransactionError> { + // NOTE: writers must always support all required reader features + self.can_read_from(snapshot)?; + + let required_features: Option<&HashSet> = + match snapshot.min_writer_version() { + 0 | 1 => None, + 2 => Some(&WRITER_V2), + 3 => Some(&WRITER_V3), + 4 => Some(&WRITER_V4), + 5 => Some(&WRITER_V5), + 6 => Some(&WRITER_V6), + _ => snapshot.writer_features(), + }; + + if let Some(features) = required_features { + let mut diff = features.difference(&self.writer_features).peekable(); + if diff.peek().is_some() { + return Err(TransactionError::UnsupportedWriterFeatures( + diff.cloned().collect(), + )); + } + }; + Ok(()) + } + + pub fn can_commit( + &self, + snapshot: &DeltaTableState, + actions: &[Action], + ) -> Result<(), TransactionError> { + self.can_write_to(snapshot)?; + + // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#append-only-tables + let append_only_enabled = if snapshot.min_writer_version() < 2 { + false + } else if snapshot.min_writer_version() < 7 { + snapshot.table_config().append_only() + } else { + snapshot + .writer_features() + .ok_or(TransactionError::WriterFeaturesRequired)? + .contains(&WriterFeatures::AppendOnly) + && snapshot.table_config().append_only() + }; + if append_only_enabled { + actions.iter().try_for_each(|action| match action { + Action::Remove(remove) if remove.data_change => { + Err(TransactionError::DeltaTableAppendOnly) + } + _ => Ok(()), + })?; + } + + Ok(()) + } +} + +/// The global protocol checker instance to validate table versions and features. +/// +/// This instance is used by default in all transaction operations, since feature +/// support is not configurable but rather decided at compile time. +/// +/// As we implement new features, we need to update this instance accordingly. +/// resulting version support is determined by the supported table feature set. +pub static INSTANCE: Lazy = Lazy::new(|| { + let reader_features = HashSet::new(); + // reader_features.insert(ReaderFeatures::ColumnMapping); + + let mut writer_features = HashSet::new(); + writer_features.insert(WriterFeatures::AppendOnly); + writer_features.insert(WriterFeatures::Invariants); + // writer_features.insert(WriterFeatures::CheckConstraints); + // writer_features.insert(WriterFeatures::ChangeDataFeed); + // writer_features.insert(WriterFeatures::GeneratedColumns); + // writer_features.insert(WriterFeatures::ColumnMapping); + // writer_features.insert(WriterFeatures::IdentityColumns); + + ProtocolChecker::new(reader_features, writer_features) +}); + +#[cfg(test)] +mod tests { + use super::super::test_utils::create_metadata_action; + use super::*; + use crate::kernel::{Action, Add, Protocol, Remove}; + use crate::DeltaConfigKey; + use std::collections::HashMap; + + #[test] + fn test_can_commit_append_only() { + let append_actions = vec![Action::Add(Add { + path: "test".to_string(), + data_change: true, + ..Default::default() + })]; + let change_actions = vec![ + Action::Add(Add { + path: "test".to_string(), + data_change: true, + ..Default::default() + }), + Action::Remove(Remove { + path: "test".to_string(), + data_change: true, + ..Default::default() + }), + ]; + let neutral_actions = vec![ + Action::Add(Add { + path: "test".to_string(), + data_change: false, + ..Default::default() + }), + Action::Remove(Remove { + path: "test".to_string(), + data_change: false, + ..Default::default() + }), + ]; + + let create_actions = |writer: i32, append: &str, feat: Vec| { + vec![ + Action::Protocol(Protocol { + min_reader_version: 1, + min_writer_version: writer, + writer_features: Some(feat.into_iter().collect()), + ..Default::default() + }), + create_metadata_action( + None, + Some(HashMap::from([( + DeltaConfigKey::AppendOnly.as_ref().to_string(), + Some(append.to_string()), + )])), + ), + ] + }; + + let checker = ProtocolChecker::new(HashSet::new(), WRITER_V2.clone()); + + let actions = create_actions(1, "true", vec![]); + let snapshot = DeltaTableState::from_actions(actions, 1).unwrap(); + assert!(checker.can_commit(&snapshot, &append_actions).is_ok()); + assert!(checker.can_commit(&snapshot, &change_actions).is_ok()); + assert!(checker.can_commit(&snapshot, &neutral_actions).is_ok()); + + let actions = create_actions(2, "true", vec![]); + let snapshot = DeltaTableState::from_actions(actions, 1).unwrap(); + assert!(checker.can_commit(&snapshot, &append_actions).is_ok()); + assert!(checker.can_commit(&snapshot, &change_actions).is_err()); + assert!(checker.can_commit(&snapshot, &neutral_actions).is_ok()); + + let actions = create_actions(2, "false", vec![]); + let snapshot = DeltaTableState::from_actions(actions, 1).unwrap(); + assert!(checker.can_commit(&snapshot, &append_actions).is_ok()); + assert!(checker.can_commit(&snapshot, &change_actions).is_ok()); + assert!(checker.can_commit(&snapshot, &neutral_actions).is_ok()); + + let actions = create_actions(7, "true", vec![WriterFeatures::AppendOnly]); + let snapshot = DeltaTableState::from_actions(actions, 1).unwrap(); + assert!(checker.can_commit(&snapshot, &append_actions).is_ok()); + assert!(checker.can_commit(&snapshot, &change_actions).is_err()); + assert!(checker.can_commit(&snapshot, &neutral_actions).is_ok()); + + let actions = create_actions(7, "false", vec![WriterFeatures::AppendOnly]); + let snapshot = DeltaTableState::from_actions(actions, 1).unwrap(); + assert!(checker.can_commit(&snapshot, &append_actions).is_ok()); + assert!(checker.can_commit(&snapshot, &change_actions).is_ok()); + assert!(checker.can_commit(&snapshot, &neutral_actions).is_ok()); + + let actions = create_actions(7, "true", vec![]); + let snapshot = DeltaTableState::from_actions(actions, 1).unwrap(); + assert!(checker.can_commit(&snapshot, &append_actions).is_ok()); + assert!(checker.can_commit(&snapshot, &change_actions).is_ok()); + assert!(checker.can_commit(&snapshot, &neutral_actions).is_ok()); + } + + #[test] + fn test_versions() { + let checker_1 = ProtocolChecker::new(HashSet::new(), HashSet::new()); + let actions = vec![Action::Protocol(Protocol { + min_reader_version: 1, + min_writer_version: 1, + ..Default::default() + })]; + let snapshot_1 = DeltaTableState::from_actions(actions, 1).unwrap(); + assert!(checker_1.can_read_from(&snapshot_1).is_ok()); + assert!(checker_1.can_write_to(&snapshot_1).is_ok()); + + let checker_2 = ProtocolChecker::new(READER_V2.clone(), HashSet::new()); + let actions = vec![Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 1, + ..Default::default() + })]; + let snapshot_2 = DeltaTableState::from_actions(actions, 1).unwrap(); + assert!(checker_1.can_read_from(&snapshot_2).is_err()); + assert!(checker_1.can_write_to(&snapshot_2).is_err()); + assert!(checker_2.can_read_from(&snapshot_1).is_ok()); + assert!(checker_2.can_read_from(&snapshot_2).is_ok()); + assert!(checker_2.can_write_to(&snapshot_2).is_ok()); + + let checker_3 = ProtocolChecker::new(READER_V2.clone(), WRITER_V2.clone()); + let actions = vec![Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 2, + ..Default::default() + })]; + let snapshot_3 = DeltaTableState::from_actions(actions, 1).unwrap(); + assert!(checker_1.can_read_from(&snapshot_3).is_err()); + assert!(checker_1.can_write_to(&snapshot_3).is_err()); + assert!(checker_2.can_read_from(&snapshot_3).is_ok()); + assert!(checker_2.can_write_to(&snapshot_3).is_err()); + assert!(checker_3.can_read_from(&snapshot_1).is_ok()); + assert!(checker_3.can_read_from(&snapshot_2).is_ok()); + assert!(checker_3.can_read_from(&snapshot_3).is_ok()); + assert!(checker_3.can_write_to(&snapshot_3).is_ok()); + + let checker_4 = ProtocolChecker::new(READER_V2.clone(), WRITER_V3.clone()); + let actions = vec![Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 3, + ..Default::default() + })]; + let snapshot_4 = DeltaTableState::from_actions(actions, 1).unwrap(); + assert!(checker_1.can_read_from(&snapshot_4).is_err()); + assert!(checker_1.can_write_to(&snapshot_4).is_err()); + assert!(checker_2.can_read_from(&snapshot_4).is_ok()); + assert!(checker_2.can_write_to(&snapshot_4).is_err()); + assert!(checker_3.can_read_from(&snapshot_4).is_ok()); + assert!(checker_3.can_write_to(&snapshot_4).is_err()); + assert!(checker_4.can_read_from(&snapshot_1).is_ok()); + assert!(checker_4.can_read_from(&snapshot_2).is_ok()); + assert!(checker_4.can_read_from(&snapshot_3).is_ok()); + assert!(checker_4.can_read_from(&snapshot_4).is_ok()); + assert!(checker_4.can_write_to(&snapshot_4).is_ok()); + + let checker_5 = ProtocolChecker::new(READER_V2.clone(), WRITER_V4.clone()); + let actions = vec![Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 4, + ..Default::default() + })]; + let snapshot_5 = DeltaTableState::from_actions(actions, 1).unwrap(); + assert!(checker_1.can_read_from(&snapshot_5).is_err()); + assert!(checker_1.can_write_to(&snapshot_5).is_err()); + assert!(checker_2.can_read_from(&snapshot_5).is_ok()); + assert!(checker_2.can_write_to(&snapshot_5).is_err()); + assert!(checker_3.can_read_from(&snapshot_5).is_ok()); + assert!(checker_3.can_write_to(&snapshot_5).is_err()); + assert!(checker_4.can_read_from(&snapshot_5).is_ok()); + assert!(checker_4.can_write_to(&snapshot_5).is_err()); + assert!(checker_5.can_read_from(&snapshot_1).is_ok()); + assert!(checker_5.can_read_from(&snapshot_2).is_ok()); + assert!(checker_5.can_read_from(&snapshot_3).is_ok()); + assert!(checker_5.can_read_from(&snapshot_4).is_ok()); + assert!(checker_5.can_read_from(&snapshot_5).is_ok()); + assert!(checker_5.can_write_to(&snapshot_5).is_ok()); + + let checker_6 = ProtocolChecker::new(READER_V2.clone(), WRITER_V5.clone()); + let actions = vec![Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 5, + ..Default::default() + })]; + let snapshot_6 = DeltaTableState::from_actions(actions, 1).unwrap(); + assert!(checker_1.can_read_from(&snapshot_6).is_err()); + assert!(checker_1.can_write_to(&snapshot_6).is_err()); + assert!(checker_2.can_read_from(&snapshot_6).is_ok()); + assert!(checker_2.can_write_to(&snapshot_6).is_err()); + assert!(checker_3.can_read_from(&snapshot_6).is_ok()); + assert!(checker_3.can_write_to(&snapshot_6).is_err()); + assert!(checker_4.can_read_from(&snapshot_6).is_ok()); + assert!(checker_4.can_write_to(&snapshot_6).is_err()); + assert!(checker_5.can_read_from(&snapshot_6).is_ok()); + assert!(checker_5.can_write_to(&snapshot_6).is_err()); + assert!(checker_6.can_read_from(&snapshot_1).is_ok()); + assert!(checker_6.can_read_from(&snapshot_2).is_ok()); + assert!(checker_6.can_read_from(&snapshot_3).is_ok()); + assert!(checker_6.can_read_from(&snapshot_4).is_ok()); + assert!(checker_6.can_read_from(&snapshot_5).is_ok()); + assert!(checker_6.can_read_from(&snapshot_6).is_ok()); + assert!(checker_6.can_write_to(&snapshot_6).is_ok()); + + let checker_7 = ProtocolChecker::new(READER_V2.clone(), WRITER_V6.clone()); + let actions = vec![Action::Protocol(Protocol { + min_reader_version: 2, + min_writer_version: 6, + ..Default::default() + })]; + let snapshot_7 = DeltaTableState::from_actions(actions, 1).unwrap(); + assert!(checker_1.can_read_from(&snapshot_7).is_err()); + assert!(checker_1.can_write_to(&snapshot_7).is_err()); + assert!(checker_2.can_read_from(&snapshot_7).is_ok()); + assert!(checker_2.can_write_to(&snapshot_7).is_err()); + assert!(checker_3.can_read_from(&snapshot_7).is_ok()); + assert!(checker_3.can_write_to(&snapshot_7).is_err()); + assert!(checker_4.can_read_from(&snapshot_7).is_ok()); + assert!(checker_4.can_write_to(&snapshot_7).is_err()); + assert!(checker_5.can_read_from(&snapshot_7).is_ok()); + assert!(checker_5.can_write_to(&snapshot_7).is_err()); + assert!(checker_6.can_read_from(&snapshot_7).is_ok()); + assert!(checker_6.can_write_to(&snapshot_7).is_err()); + assert!(checker_7.can_read_from(&snapshot_1).is_ok()); + assert!(checker_7.can_read_from(&snapshot_2).is_ok()); + assert!(checker_7.can_read_from(&snapshot_3).is_ok()); + assert!(checker_7.can_read_from(&snapshot_4).is_ok()); + assert!(checker_7.can_read_from(&snapshot_5).is_ok()); + assert!(checker_7.can_read_from(&snapshot_6).is_ok()); + assert!(checker_7.can_read_from(&snapshot_7).is_ok()); + assert!(checker_7.can_write_to(&snapshot_7).is_ok()); + } +} diff --git a/crates/deltalake-core/src/operations/transaction/state.rs b/crates/deltalake-core/src/operations/transaction/state.rs index a209b7369d..3e56c4a9b0 100644 --- a/crates/deltalake-core/src/operations/transaction/state.rs +++ b/crates/deltalake-core/src/operations/transaction/state.rs @@ -1,35 +1,23 @@ use std::sync::Arc; -use arrow::array::ArrayRef; -use arrow::datatypes::{ - DataType, Field as ArrowField, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, -}; +use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion::datasource::physical_plan::wrap_partition_type_in_dict; use datafusion::execution::context::SessionState; use datafusion::optimizer::utils::conjunction; -use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; -use datafusion_common::scalar::ScalarValue; -use datafusion_common::{Column, DFSchema}; +use datafusion::physical_optimizer::pruning::PruningPredicate; +use datafusion_common::DFSchema; use datafusion_expr::Expr; use itertools::Either; -use object_store::ObjectStore; -use parquet::arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStreamBuilder}; use crate::delta_datafusion::expr::parse_predicate_expression; -use crate::delta_datafusion::{ - get_null_of_arrow_type, logical_expr_to_physical_expr, to_correct_scalar_value, -}; +use crate::delta_datafusion::logical_expr_to_physical_expr; use crate::errors::{DeltaResult, DeltaTableError}; use crate::kernel::Add; use crate::table::state::DeltaTableState; impl DeltaTableState { - /// Get the table schema as an [`ArrowSchemaRef`] - pub fn arrow_schema(&self) -> DeltaResult { - self._arrow_schema(true) - } - - fn _arrow_schema(&self, wrap_partitions: bool) -> DeltaResult { + /// Get the table schema as an [`SchemaRef`] + pub fn arrow_schema(&self, wrap_partitions: bool) -> DeltaResult { let meta = self.current_metadata().ok_or(DeltaTableError::NoMetadata)?; let fields = meta .schema @@ -43,7 +31,7 @@ impl DeltaTableState { .iter() .filter(|f| meta.partition_columns.contains(&f.name().to_string())) .map(|f| { - let field = ArrowField::try_from(f)?; + let field = Field::try_from(f)?; let corrected = if wrap_partitions { match field.data_type() { // Only dictionary-encode types that may be large @@ -62,13 +50,9 @@ impl DeltaTableState { Ok(field.with_data_type(corrected)) }), ) - .collect::, _>>()?; - - Ok(Arc::new(ArrowSchema::new(fields))) - } + .collect::, _>>()?; - pub(crate) fn input_schema(&self) -> DeltaResult { - self._arrow_schema(false) + Ok(Arc::new(Schema::new(fields))) } /// Iterate over all files in the log matching a predicate @@ -79,8 +63,8 @@ impl DeltaTableState { if let Some(Some(predicate)) = (!filters.is_empty()).then_some(conjunction(filters.iter().cloned())) { - let expr = logical_expr_to_physical_expr(&predicate, self.arrow_schema()?.as_ref()); - let pruning_predicate = PruningPredicate::try_new(expr, self.arrow_schema()?)?; + let expr = logical_expr_to_physical_expr(&predicate, self.arrow_schema(true)?.as_ref()); + let pruning_predicate = PruningPredicate::try_new(expr, self.arrow_schema(true)?)?; Ok(Either::Left( self.files() .iter() @@ -106,223 +90,9 @@ impl DeltaTableState { expr: impl AsRef, df_state: &SessionState, ) -> DeltaResult { - let schema = DFSchema::try_from(self.arrow_schema()?.as_ref().to_owned())?; + let schema = DFSchema::try_from(self.arrow_schema(true)?.as_ref().to_owned())?; parse_predicate_expression(&schema, expr, df_state) } - - /// Get the physical table schema. - /// - /// This will construct a schema derived from the parquet schema of the latest data file, - /// and fields for partition columns from the schema defined in table meta data. - pub async fn physical_arrow_schema( - &self, - object_store: Arc, - ) -> DeltaResult { - if let Some(add) = self.files().iter().max_by_key(|obj| obj.modification_time) { - let file_meta = add.try_into()?; - let file_reader = ParquetObjectReader::new(object_store, file_meta); - let file_schema = ParquetRecordBatchStreamBuilder::new(file_reader) - .await? - .build()? - .schema() - .clone(); - - let table_schema = Arc::new(ArrowSchema::new( - self.arrow_schema()? - .fields - .clone() - .into_iter() - .map(|field| { - // field is an &Arc - let owned_field: ArrowField = field.as_ref().clone(); - file_schema - .field_with_name(field.name()) - // yielded with &Field - .cloned() - .unwrap_or(owned_field) - }) - .collect::>(), - )); - - Ok(table_schema) - } else { - self.arrow_schema() - } - } -} - -pub struct AddContainer<'a> { - inner: &'a Vec, - partition_columns: &'a Vec, - schema: ArrowSchemaRef, -} - -impl<'a> AddContainer<'a> { - /// Create a new instance of [`AddContainer`] - pub fn new( - adds: &'a Vec, - partition_columns: &'a Vec, - schema: ArrowSchemaRef, - ) -> Self { - Self { - inner: adds, - partition_columns, - schema, - } - } - - pub fn get_prune_stats(&self, column: &Column, get_max: bool) -> Option { - let (_, field) = self.schema.column_with_name(&column.name)?; - - // See issue 1214. Binary type does not support natural order which is required for Datafusion to prune - if field.data_type() == &DataType::Binary { - return None; - } - - let data_type = field.data_type(); - - let values = self.inner.iter().map(|add| { - if self.partition_columns.contains(&column.name) { - let value = add.partition_values.get(&column.name).unwrap(); - let value = match value { - Some(v) => serde_json::Value::String(v.to_string()), - None => serde_json::Value::Null, - }; - to_correct_scalar_value(&value, data_type).unwrap_or( - get_null_of_arrow_type(data_type).expect("Could not determine null type"), - ) - } else if let Ok(Some(statistics)) = add.get_stats() { - let values = if get_max { - statistics.max_values - } else { - statistics.min_values - }; - - values - .get(&column.name) - .and_then(|f| to_correct_scalar_value(f.as_value()?, data_type)) - .unwrap_or( - get_null_of_arrow_type(data_type).expect("Could not determine null type"), - ) - } else { - get_null_of_arrow_type(data_type).expect("Could not determine null type") - } - }); - ScalarValue::iter_to_array(values).ok() - } - - /// Get an iterator of add actions / files, that MAY contain data matching the predicate. - /// - /// Expressions are evaluated for file statistics, essentially column-wise min max bounds, - /// so evaluating expressions is inexact. However, excluded files are guaranteed (for a correct log) - /// to not contain matches by the predicate expression. - pub fn predicate_matches(&self, predicate: Expr) -> DeltaResult> { - let expr = logical_expr_to_physical_expr(&predicate, &self.schema); - let pruning_predicate = PruningPredicate::try_new(expr, self.schema.clone())?; - Ok(self - .inner - .iter() - .zip(pruning_predicate.prune(self)?) - .filter_map( - |(action, keep_file)| { - if keep_file { - Some(action) - } else { - None - } - }, - )) - } -} - -impl<'a> PruningStatistics for AddContainer<'a> { - /// return the minimum values for the named column, if known. - /// Note: the returned array must contain `num_containers()` rows - fn min_values(&self, column: &Column) -> Option { - self.get_prune_stats(column, false) - } - - /// return the maximum values for the named column, if known. - /// Note: the returned array must contain `num_containers()` rows. - fn max_values(&self, column: &Column) -> Option { - self.get_prune_stats(column, true) - } - - /// return the number of containers (e.g. row groups) being - /// pruned with these statistics - fn num_containers(&self) -> usize { - self.inner.len() - } - - /// return the number of null values for the named column as an - /// `Option`. - /// - /// Note: the returned array must contain `num_containers()` rows. - fn null_counts(&self, column: &Column) -> Option { - let values = self.inner.iter().map(|add| { - if let Ok(Some(statistics)) = add.get_stats() { - if self.partition_columns.contains(&column.name) { - let value = add.partition_values.get(&column.name).unwrap(); - match value { - Some(_) => ScalarValue::UInt64(Some(0)), - None => ScalarValue::UInt64(Some(statistics.num_records as u64)), - } - } else { - statistics - .null_count - .get(&column.name) - .map(|f| ScalarValue::UInt64(f.as_value().map(|val| val as u64))) - .unwrap_or(ScalarValue::UInt64(None)) - } - } else if self.partition_columns.contains(&column.name) { - let value = add.partition_values.get(&column.name).unwrap(); - match value { - Some(_) => ScalarValue::UInt64(Some(0)), - None => ScalarValue::UInt64(None), - } - } else { - ScalarValue::UInt64(None) - } - }); - ScalarValue::iter_to_array(values).ok() - } -} - -impl PruningStatistics for DeltaTableState { - /// return the minimum values for the named column, if known. - /// Note: the returned array must contain `num_containers()` rows - fn min_values(&self, column: &Column) -> Option { - let partition_columns = &self.current_metadata()?.partition_columns; - let container = - AddContainer::new(self.files(), partition_columns, self.arrow_schema().ok()?); - container.min_values(column) - } - - /// return the maximum values for the named column, if known. - /// Note: the returned array must contain `num_containers()` rows. - fn max_values(&self, column: &Column) -> Option { - let partition_columns = &self.current_metadata()?.partition_columns; - let container = - AddContainer::new(self.files(), partition_columns, self.arrow_schema().ok()?); - container.max_values(column) - } - - /// return the number of containers (e.g. row groups) being - /// pruned with these statistics - fn num_containers(&self) -> usize { - self.files().len() - } - - /// return the number of null values for the named column as an - /// `Option`. - /// - /// Note: the returned array must contain `num_containers()` rows. - fn null_counts(&self, column: &Column) -> Option { - let partition_columns = &self.current_metadata()?.partition_columns; - let container = - AddContainer::new(self.files(), partition_columns, self.arrow_schema().ok()?); - container.null_counts(column) - } } #[cfg(test)] diff --git a/crates/deltalake-core/src/operations/transaction/test_utils.rs b/crates/deltalake-core/src/operations/transaction/test_utils.rs index 56b0894019..2efdcde2ea 100644 --- a/crates/deltalake-core/src/operations/transaction/test_utils.rs +++ b/crates/deltalake-core/src/operations/transaction/test_utils.rs @@ -6,6 +6,7 @@ use crate::kernel::{ Action, Add, CommitInfo, DataType, Metadata, PrimitiveType, Protocol, Remove, StructField, StructType, }; +use crate::operations::transaction::PROTOCOL; use crate::protocol::{DeltaOperation, SaveMode}; use crate::table::state::DeltaTableState; use crate::table::DeltaTableMetaData; @@ -49,8 +50,8 @@ pub fn create_remove_action(path: impl Into, data_change: bool) -> Actio pub fn create_protocol_action(max_reader: Option, max_writer: Option) -> Action { let protocol = Protocol { - min_reader_version: max_reader.unwrap_or(crate::operations::MAX_SUPPORTED_READER_VERSION), - min_writer_version: max_writer.unwrap_or(crate::operations::MAX_SUPPORTED_WRITER_VERSION), + min_reader_version: max_reader.unwrap_or(PROTOCOL.default_reader_version()), + min_writer_version: max_writer.unwrap_or(PROTOCOL.default_writer_version()), writer_features: None, reader_features: None, }; @@ -165,7 +166,6 @@ pub async fn create_initialized_table( log_store.object_store().as_ref(), &operation, &actions, - &state, None, ) .await diff --git a/crates/deltalake-core/src/operations/update.rs b/crates/deltalake-core/src/operations/update.rs index 9f51912579..8f609144d6 100644 --- a/crates/deltalake-core/src/operations/update.rs +++ b/crates/deltalake-core/src/operations/update.rs @@ -44,7 +44,7 @@ use serde::Serialize; use serde_json::Value; use super::datafusion_utils::{Expression, MetricObserverExec}; -use super::transaction::commit; +use super::transaction::{commit, PROTOCOL}; use super::write::write_execution_plan; use crate::delta_datafusion::expr::fmt_expr_to_sql; use crate::delta_datafusion::{find_files, register_store, DeltaScanBuilder}; @@ -232,7 +232,7 @@ async fn execute( let scan = Arc::new(scan); // Create a projection for a new column with the predicate evaluated - let input_schema = snapshot.input_schema()?; + let input_schema = snapshot.arrow_schema(false)?; let mut fields = Vec::new(); for field in input_schema.fields.iter() { @@ -426,6 +426,8 @@ impl std::future::IntoFuture for UpdateBuilder { let mut this = self; Box::pin(async move { + PROTOCOL.can_write_to(&this.snapshot)?; + let state = this.state.unwrap_or_else(|| { let session = SessionContext::new(); diff --git a/crates/deltalake-core/src/operations/vacuum.rs b/crates/deltalake-core/src/operations/vacuum.rs index efdde55347..6b756af1ec 100644 --- a/crates/deltalake-core/src/operations/vacuum.rs +++ b/crates/deltalake-core/src/operations/vacuum.rs @@ -163,7 +163,12 @@ impl VacuumBuilder { /// Determine which files can be deleted. Does not actually peform the deletion async fn create_vacuum_plan(&self) -> Result { - let min_retention = Duration::milliseconds(self.snapshot.tombstone_retention_millis()); + let min_retention = Duration::milliseconds( + self.snapshot + .table_config() + .deleted_file_retention_duration() + .as_millis() as i64, + ); let retention_period = self.retention_period.unwrap_or(min_retention); let enforce_retention_duration = self.enforce_retention_duration; diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index dec4b7ced7..2cb5b8864a 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -38,10 +38,10 @@ use futures::future::BoxFuture; use futures::StreamExt; use parquet::file::properties::WriterProperties; +use super::transaction::PROTOCOL; use super::writer::{DeltaWriter, WriterConfig}; -use super::MAX_SUPPORTED_WRITER_VERSION; use super::{transaction::commit, CreateBuilder}; -use crate::delta_datafusion::DeltaDataChecker; +use crate::delta_datafusion::{pruning::physical_arrow_schema, DeltaDataChecker}; use crate::errors::{DeltaResult, DeltaTableError}; use crate::kernel::{Action, Add, Remove, StructType}; use crate::logstore::LogStoreRef; @@ -60,16 +60,11 @@ enum WriteError { #[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), #[error( - "Specified table partitioning does not match table partitioning: expected: {:?}, got: {:?}", - expected, - got + "Specified table partitioning does not match table partitioning: expected: {expected:?}, got: {got:?}", )] PartitionColumnMismatch { expected: Vec, @@ -213,16 +208,12 @@ impl WriteBuilder { async fn check_preconditions(&self) -> DeltaResult> { match self.log_store.is_delta_table_location().await? { true => { - let min_writer = self.snapshot.min_writer_version(); - if min_writer > MAX_SUPPORTED_WRITER_VERSION { - Err(WriteError::UnsupportedWriterVersion(min_writer).into()) - } else { - match self.mode { - SaveMode::ErrorIfExists => { - Err(WriteError::AlreadyExists(self.log_store.root_uri()).into()) - } - _ => Ok(vec![]), + PROTOCOL.can_write_to(&self.snapshot)?; + match self.mode { + SaveMode::ErrorIfExists => { + Err(WriteError::AlreadyExists(self.log_store.root_uri()).into()) } + _ => Ok(vec![]), } } false => { @@ -267,7 +258,7 @@ pub(crate) async fn write_execution_plan( .unwrap_or_default(); // Use input schema to prevent wrapping partitions columns into a dictionary. - let schema = snapshot.input_schema().unwrap_or(plan.schema()); + let schema = snapshot.arrow_schema(false).unwrap_or(plan.schema()); let checker = DeltaDataChecker::new(invariants); @@ -355,12 +346,13 @@ impl std::future::IntoFuture for WriteBuilder { Err(WriteError::MissingData) } else { let schema = batches[0].schema(); - let table_schema = this - .snapshot - .physical_arrow_schema(this.log_store.object_store().clone()) - .await - .or_else(|_| this.snapshot.arrow_schema()) - .unwrap_or(schema.clone()); + let table_schema = physical_arrow_schema( + &this.snapshot, + this.log_store.object_store().clone(), + ) + .await + .or_else(|_| this.snapshot.arrow_schema(true)) + .unwrap_or(schema.clone()); if !can_cast_batch(schema.fields(), table_schema.fields()) { return Err(DeltaTableError::Generic( diff --git a/crates/deltalake-core/src/protocol/checkpoints.rs b/crates/deltalake-core/src/protocol/checkpoints.rs index a4cc1b66c7..9df2a80a40 100644 --- a/crates/deltalake-core/src/protocol/checkpoints.rs +++ b/crates/deltalake-core/src/protocol/checkpoints.rs @@ -18,7 +18,7 @@ use regex::Regex; use serde_json::Value; use super::{time_utils, ProtocolError}; -use crate::kernel::actions::arrow::delta_log_schema_for_table; +use crate::kernel::arrow::delta_log_schema_for_table; use crate::kernel::{ Action, Add as AddAction, DataType, Metadata, PrimitiveType, Protocol, StructField, StructType, Txn, @@ -82,8 +82,12 @@ pub async fn create_checkpoint(table: &DeltaTable) -> Result<(), ProtocolError> /// Delete expires log files before given version from table. The table log retention is based on /// the `logRetentionDuration` property of the Delta Table, 30 days by default. pub async fn cleanup_metadata(table: &DeltaTable) -> Result { - let log_retention_timestamp = - Utc::now().timestamp_millis() - table.get_state().log_retention_millis(); + let log_retention_timestamp = Utc::now().timestamp_millis() + - table + .get_state() + .table_config() + .log_retention_duration() + .as_millis() as i64; cleanup_expired_logs_for( table.version(), table.log_store.as_ref(), @@ -105,8 +109,12 @@ pub async fn create_checkpoint_from_table_uri_and_cleanup( .map_err(|err| ProtocolError::Generic(err.to_string()))?; create_checkpoint_for(version, table.get_state(), table.log_store.as_ref()).await?; - let enable_expired_log_cleanup = - cleanup.unwrap_or_else(|| table.get_state().enable_expired_log_cleanup()); + let enable_expired_log_cleanup = cleanup.unwrap_or_else(|| { + table + .get_state() + .table_config() + .enable_expired_log_cleanup() + }); if table.version() >= 0 && enable_expired_log_cleanup { let deleted_log_num = cleanup_metadata(&table).await?; diff --git a/crates/deltalake-core/src/schema/arrow_convert.rs b/crates/deltalake-core/src/schema/arrow_convert.rs deleted file mode 100644 index d292362604..0000000000 --- a/crates/deltalake-core/src/schema/arrow_convert.rs +++ /dev/null @@ -1,1049 +0,0 @@ -use std::sync::Arc; - -use arrow_schema::{ - ArrowError, DataType as ArrowDataType, Field as ArrowField, FieldRef as ArrowFieldRef, - Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, TimeUnit, -}; -use lazy_static::lazy_static; - -use super::super::schema::{ArrayType, DataType, MapType, PrimitiveType, StructField, StructType}; - -impl TryFrom<&StructType> for ArrowSchema { - type Error = ArrowError; - - fn try_from(s: &StructType) -> Result { - let fields = s - .fields() - .iter() - .map(>::try_from) - .collect::, ArrowError>>()?; - - Ok(ArrowSchema::new(fields)) - } -} - -impl TryFrom<&StructField> for ArrowField { - type Error = ArrowError; - - fn try_from(f: &StructField) -> Result { - let metadata = f - .metadata() - .iter() - .map(|(key, val)| Ok((key.clone(), serde_json::to_string(val)?))) - .collect::>() - .map_err(|err| ArrowError::JsonError(err.to_string()))?; - - let field = ArrowField::new( - f.name(), - ArrowDataType::try_from(f.data_type())?, - f.is_nullable(), - ) - .with_metadata(metadata); - - Ok(field) - } -} - -impl TryFrom<&ArrayType> for ArrowField { - type Error = ArrowError; - - fn try_from(a: &ArrayType) -> Result { - Ok(ArrowField::new( - "item", - ArrowDataType::try_from(a.element_type())?, - a.contains_null(), - )) - } -} - -impl TryFrom<&MapType> for ArrowField { - type Error = ArrowError; - - fn try_from(a: &MapType) -> Result { - Ok(ArrowField::new( - "entries", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::try_from(a.key_type())?, false), - ArrowField::new( - "value", - ArrowDataType::try_from(a.value_type())?, - a.value_contains_null(), - ), - ] - .into(), - ), - false, // always non-null - )) - } -} - -impl TryFrom<&DataType> for ArrowDataType { - type Error = ArrowError; - - fn try_from(t: &DataType) -> Result { - match t { - DataType::Primitive(p) => { - match p { - PrimitiveType::String => Ok(ArrowDataType::Utf8), - PrimitiveType::Long => Ok(ArrowDataType::Int64), // undocumented type - PrimitiveType::Integer => Ok(ArrowDataType::Int32), - PrimitiveType::Short => Ok(ArrowDataType::Int16), - PrimitiveType::Byte => Ok(ArrowDataType::Int8), - PrimitiveType::Float => Ok(ArrowDataType::Float32), - PrimitiveType::Double => Ok(ArrowDataType::Float64), - PrimitiveType::Boolean => Ok(ArrowDataType::Boolean), - PrimitiveType::Binary => Ok(ArrowDataType::Binary), - PrimitiveType::Decimal(precision, scale) => { - let precision = u8::try_from(*precision).map_err(|_| { - ArrowError::SchemaError(format!( - "Invalid precision for decimal: {}", - precision - )) - })?; - let scale = i8::try_from(*scale).map_err(|_| { - ArrowError::SchemaError(format!("Invalid scale for decimal: {}", scale)) - })?; - - if precision <= 38 { - Ok(ArrowDataType::Decimal128(precision, scale)) - } else if precision <= 76 { - Ok(ArrowDataType::Decimal256(precision, scale)) - } else { - Err(ArrowError::SchemaError(format!( - "Precision too large to be represented in Arrow: {}", - precision - ))) - } - } - PrimitiveType::Date => { - // A calendar date, represented as a year-month-day triple without a - // timezone. Stored as 4 bytes integer representing days since 1970-01-01 - Ok(ArrowDataType::Date32) - } - PrimitiveType::Timestamp => { - // Issue: https://github.com/delta-io/delta/issues/643 - Ok(ArrowDataType::Timestamp(TimeUnit::Microsecond, None)) - } - } - } - DataType::Struct(s) => Ok(ArrowDataType::Struct( - s.fields() - .iter() - .map(>::try_from) - .collect::, ArrowError>>()? - .into(), - )), - DataType::Array(a) => Ok(ArrowDataType::List(Arc::new(>::try_from(a)?))), - DataType::Map(m) => Ok(ArrowDataType::Map( - Arc::new(ArrowField::new( - "entries", - ArrowDataType::Struct( - vec![ - ArrowField::new( - "keys", - >::try_from(m.key_type())?, - false, - ), - ArrowField::new( - "values", - >::try_from(m.value_type())?, - m.value_contains_null(), - ), - ] - .into(), - ), - false, - )), - false, - )), - } - } -} - -impl TryFrom<&ArrowSchema> for StructType { - type Error = ArrowError; - - fn try_from(arrow_schema: &ArrowSchema) -> Result { - let new_fields: Result, _> = arrow_schema - .fields() - .iter() - .map(|field| field.as_ref().try_into()) - .collect(); - Ok(StructType::new(new_fields?)) - } -} - -impl TryFrom for StructType { - type Error = ArrowError; - - fn try_from(arrow_schema: ArrowSchemaRef) -> Result { - arrow_schema.as_ref().try_into() - } -} - -impl TryFrom<&ArrowField> for StructField { - type Error = ArrowError; - - fn try_from(arrow_field: &ArrowField) -> Result { - Ok(StructField::new( - arrow_field.name().clone(), - arrow_field.data_type().try_into()?, - arrow_field.is_nullable(), - ) - .with_metadata(arrow_field.metadata().iter().map(|(k, v)| (k.clone(), v)))) - } -} - -impl TryFrom<&ArrowDataType> for DataType { - type Error = ArrowError; - - fn try_from(arrow_datatype: &ArrowDataType) -> Result { - match arrow_datatype { - ArrowDataType::Utf8 => Ok(DataType::Primitive(PrimitiveType::String)), - ArrowDataType::LargeUtf8 => Ok(DataType::Primitive(PrimitiveType::String)), - ArrowDataType::Int64 => Ok(DataType::Primitive(PrimitiveType::Long)), // undocumented type - ArrowDataType::Int32 => Ok(DataType::Primitive(PrimitiveType::Integer)), - ArrowDataType::Int16 => Ok(DataType::Primitive(PrimitiveType::Short)), - ArrowDataType::Int8 => Ok(DataType::Primitive(PrimitiveType::Byte)), - ArrowDataType::UInt64 => Ok(DataType::Primitive(PrimitiveType::Long)), // undocumented type - ArrowDataType::UInt32 => Ok(DataType::Primitive(PrimitiveType::Integer)), - ArrowDataType::UInt16 => Ok(DataType::Primitive(PrimitiveType::Short)), - ArrowDataType::UInt8 => Ok(DataType::Primitive(PrimitiveType::Boolean)), - ArrowDataType::Float32 => Ok(DataType::Primitive(PrimitiveType::Float)), - ArrowDataType::Float64 => Ok(DataType::Primitive(PrimitiveType::Double)), - ArrowDataType::Boolean => Ok(DataType::Primitive(PrimitiveType::Boolean)), - ArrowDataType::Binary => Ok(DataType::Primitive(PrimitiveType::Binary)), - ArrowDataType::FixedSizeBinary(_) => Ok(DataType::Primitive(PrimitiveType::Binary)), - ArrowDataType::LargeBinary => Ok(DataType::Primitive(PrimitiveType::Binary)), - ArrowDataType::Decimal128(p, s) => Ok(DataType::Primitive(PrimitiveType::Decimal( - *p as i32, *s as i32, - ))), - ArrowDataType::Decimal256(p, s) => Ok(DataType::Primitive(PrimitiveType::Decimal( - *p as i32, *s as i32, - ))), - ArrowDataType::Date32 => Ok(DataType::Primitive(PrimitiveType::Date)), - ArrowDataType::Date64 => Ok(DataType::Primitive(PrimitiveType::Date)), - ArrowDataType::Timestamp(TimeUnit::Microsecond, None) => { - Ok(DataType::Primitive(PrimitiveType::Timestamp)) - } - ArrowDataType::Timestamp(TimeUnit::Microsecond, Some(tz)) - if tz.eq_ignore_ascii_case("utc") => - { - Ok(DataType::Primitive(PrimitiveType::Timestamp)) - } - ArrowDataType::Struct(fields) => { - let converted_fields: Result, _> = fields - .iter() - .map(|field| field.as_ref().try_into()) - .collect(); - Ok(DataType::Struct(Box::new(StructType::new( - converted_fields?, - )))) - } - ArrowDataType::List(field) => Ok(DataType::Array(Box::new(ArrayType::new( - (*field).data_type().try_into()?, - (*field).is_nullable(), - )))), - ArrowDataType::LargeList(field) => Ok(DataType::Array(Box::new(ArrayType::new( - (*field).data_type().try_into()?, - (*field).is_nullable(), - )))), - ArrowDataType::FixedSizeList(field, _) => Ok(DataType::Array(Box::new( - ArrayType::new((*field).data_type().try_into()?, (*field).is_nullable()), - ))), - ArrowDataType::Map(field, _) => { - if let ArrowDataType::Struct(struct_fields) = field.data_type() { - let key_type = struct_fields[0].data_type().try_into()?; - let value_type = struct_fields[1].data_type().try_into()?; - let value_type_nullable = struct_fields[1].is_nullable(); - Ok(DataType::Map(Box::new(MapType::new( - key_type, - value_type, - value_type_nullable, - )))) - } else { - panic!("DataType::Map should contain a struct field child"); - } - } - s => Err(ArrowError::SchemaError(format!( - "Invalid data type for Delta Lake: {s}" - ))), - } - } -} - -macro_rules! arrow_map { - ($fieldname: ident, null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::Map( - Arc::new(ArrowField::new( - "entries", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Utf8, false), - ArrowField::new("value", ArrowDataType::Utf8, true), - ] - .into(), - ), - false, - )), - false, - ), - true, - ) - }; - ($fieldname: ident, not_null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::Map( - Arc::new(ArrowField::new( - "entries", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Utf8, false), - ArrowField::new("value", ArrowDataType::Utf8, false), - ] - .into(), - ), - false, - )), - false, - ), - false, - ) - }; -} - -macro_rules! arrow_field { - ($fieldname:ident, $type_qual:ident, null) => { - ArrowField::new(stringify!($fieldname), ArrowDataType::$type_qual, true) - }; - ($fieldname:ident, $type_qual:ident, not_null) => { - ArrowField::new(stringify!($fieldname), ArrowDataType::$type_qual, false) - }; -} - -macro_rules! arrow_list { - ($fieldname:ident, $element_name:ident, $type_qual:ident, null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::List(Arc::new(ArrowField::new( - stringify!($element_name), - ArrowDataType::$type_qual, - true, - ))), - true, - ) - }; - ($fieldname:ident, $element_name:ident, $type_qual:ident, not_null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::List(Arc::new(ArrowField::new( - stringify!($element_name), - ArrowDataType::$type_qual, - true, - ))), - false, - ) - }; -} - -macro_rules! arrow_struct { - ($fieldname:ident, [$($inner:tt)+], null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::Struct( - arrow_defs! [$($inner)+].into() - ), - true - ) - }; - ($fieldname:ident, [$($inner:tt)+], not_null) => { - ArrowField::new( - stringify!($fieldname), - ArrowDataType::Struct( - arrow_defs! [$($inner)+].into() - ), - false - ) - } -} - -macro_rules! arrow_def { - ($fieldname:ident $(null)?) => { - arrow_map!($fieldname, null) - }; - ($fieldname:ident not_null) => { - arrow_map!($fieldname, not_null) - }; - ($fieldname:ident[$inner_name:ident]{$type_qual:ident} $(null)?) => { - arrow_list!($fieldname, $inner_name, $type_qual, null) - }; - ($fieldname:ident[$inner_name:ident]{$type_qual:ident} not_null) => { - arrow_list!($fieldname, $inner_name, $type_qual, not_null) - }; - ($fieldname:ident:$type_qual:ident $(null)?) => { - arrow_field!($fieldname, $type_qual, null) - }; - ($fieldname:ident:$type_qual:ident not_null) => { - arrow_field!($fieldname, $type_qual, not_null) - }; - ($fieldname:ident[$($inner:tt)+] $(null)?) => { - arrow_struct!($fieldname, [$($inner)+], null) - }; - ($fieldname:ident[$($inner:tt)+] not_null) => { - arrow_struct!($fieldname, [$($inner)+], not_null) - } -} - -/// A helper macro to create more readable Arrow field definitions, delimited by commas -/// -/// The argument patterns are as follows: -/// -/// fieldname (null|not_null)? -- An arrow field of type map with name "fieldname" consisting of Utf8 key-value pairs, and an -/// optional nullability qualifier (null if not specified). -/// -/// fieldname:type (null|not_null)? -- An Arrow field consisting of an atomic type. For example, -/// id:Utf8 gets mapped to ArrowField::new("id", ArrowDataType::Utf8, true). -/// where customerCount:Int64 not_null gets mapped to gets mapped to -/// ArrowField::new("customerCount", ArrowDataType::Utf8, true) -/// -/// fieldname[list_element]{list_element_type} (null|not_null)? -- An Arrow list, with the name of the elements wrapped in square brackets -/// and the type of the list elements wrapped in curly brackets. For example, -/// customers[name]{Utf8} is an nullable arrow field of type arrow list consisting -/// of elements called "name" with type Utf8. -/// -/// fieldname[element1, element2, element3, ....] (null|not_null)? -- An arrow struct with name "fieldname" consisting of elements adhering to any of the patterns -/// documented, including additional structs arbitrarily nested up to the recursion -/// limit for Rust macros. -macro_rules! arrow_defs { - () => { - vec![] as Vec - }; - ($($fieldname:ident$(:$type_qual:ident)?$([$($inner:tt)+])?$({$list_type_qual:ident})? $($nullable:ident)?),+) => { - vec![ - $(arrow_def!($fieldname$(:$type_qual)?$([$($inner)+])?$({$list_type_qual})? $($nullable)?)),+ - ] - } -} - -/// Returns an arrow schema representing the delta log for use in checkpoints -/// -/// # Arguments -/// -/// * `table_schema` - The arrow schema representing the table backed by the delta log -/// * `partition_columns` - The list of partition columns of the table. -/// * `use_extended_remove_schema` - Whether to include extended file metadata in remove action schema. -/// Required for compatibility with different versions of Databricks runtime. -pub(crate) fn delta_log_schema_for_table( - table_schema: ArrowSchema, - partition_columns: &[String], - use_extended_remove_schema: bool, -) -> ArrowSchemaRef { - lazy_static! { - static ref SCHEMA_FIELDS: Vec = arrow_defs![ - metaData[ - id:Utf8, - name:Utf8, - description:Utf8, - schemaString:Utf8, - createdTime:Int64, - partitionColumns[element]{Utf8}, - configuration, - format[provider:Utf8, options] - ], - protocol[ - minReaderVersion:Int32, - minWriterVersion:Int32 - ], - txn[ - appId:Utf8, - version:Int64 - ] - ]; - static ref ADD_FIELDS: Vec = arrow_defs![ - path:Utf8, - size:Int64, - modificationTime:Int64, - dataChange:Boolean, - stats:Utf8, - partitionValues, - tags, - deletionVector[ - storageType:Utf8 not_null, - pathOrInlineDv:Utf8 not_null, - offset:Int32 null, - sizeInBytes:Int32 not_null, - cardinality:Int64 not_null - ] - ]; - static ref REMOVE_FIELDS: Vec = arrow_defs![ - path: Utf8, - deletionTimestamp: Int64, - dataChange: Boolean, - extendedFileMetadata: Boolean - ]; - static ref REMOVE_EXTENDED_FILE_METADATA_FIELDS: Vec = - arrow_defs![size: Int64, partitionValues, tags]; - }; - - // create add fields according to the specific data table schema - let (partition_fields, non_partition_fields): (Vec, Vec) = - table_schema - .fields() - .iter() - .map(|field| field.to_owned()) - .partition(|field| partition_columns.contains(field.name())); - - let mut stats_parsed_fields: Vec = - vec![ArrowField::new("numRecords", ArrowDataType::Int64, true)]; - if !non_partition_fields.is_empty() { - let mut max_min_vec = Vec::new(); - non_partition_fields - .iter() - .for_each(|f| max_min_schema_for_fields(&mut max_min_vec, f)); - - stats_parsed_fields.extend(["minValues", "maxValues"].into_iter().map(|name| { - ArrowField::new( - name, - ArrowDataType::Struct(max_min_vec.clone().into()), - true, - ) - })); - - let mut null_count_vec = Vec::new(); - non_partition_fields - .iter() - .for_each(|f| null_count_schema_for_fields(&mut null_count_vec, f)); - let null_count_struct = ArrowField::new( - "nullCount", - ArrowDataType::Struct(null_count_vec.into()), - true, - ); - - stats_parsed_fields.push(null_count_struct); - } - let mut add_fields = ADD_FIELDS.clone(); - add_fields.push(ArrowField::new( - "stats_parsed", - ArrowDataType::Struct(stats_parsed_fields.into()), - true, - )); - if !partition_fields.is_empty() { - add_fields.push(ArrowField::new( - "partitionValues_parsed", - ArrowDataType::Struct(partition_fields.into()), - true, - )); - } - - // create remove fields with or without extendedFileMetadata - let mut remove_fields = REMOVE_FIELDS.clone(); - if use_extended_remove_schema { - remove_fields.extend(REMOVE_EXTENDED_FILE_METADATA_FIELDS.clone()); - } - - // include add and remove fields in checkpoint schema - let mut schema_fields = SCHEMA_FIELDS.clone(); - schema_fields.push(ArrowField::new( - "add", - ArrowDataType::Struct(add_fields.into()), - true, - )); - schema_fields.push(ArrowField::new( - "remove", - ArrowDataType::Struct(remove_fields.into()), - true, - )); - - let arrow_schema = ArrowSchema::new(schema_fields); - - std::sync::Arc::new(arrow_schema) -} - -fn max_min_schema_for_fields(dest: &mut Vec, f: &ArrowField) { - match f.data_type() { - ArrowDataType::Struct(struct_fields) => { - let mut child_dest = Vec::new(); - - for f in struct_fields { - max_min_schema_for_fields(&mut child_dest, f); - } - - dest.push(ArrowField::new( - f.name(), - ArrowDataType::Struct(child_dest.into()), - true, - )); - } - // don't compute min or max for list, map or binary types - ArrowDataType::List(_) | ArrowDataType::Map(_, _) | ArrowDataType::Binary => { /* noop */ } - _ => { - let f = f.clone(); - dest.push(f); - } - } -} - -fn null_count_schema_for_fields(dest: &mut Vec, f: &ArrowField) { - match f.data_type() { - ArrowDataType::Struct(struct_fields) => { - let mut child_dest = Vec::new(); - - for f in struct_fields { - null_count_schema_for_fields(&mut child_dest, f); - } - - dest.push(ArrowField::new( - f.name(), - ArrowDataType::Struct(child_dest.into()), - true, - )); - } - _ => { - let f = ArrowField::new(f.name(), ArrowDataType::Int64, true); - dest.push(f); - } - } -} - -#[cfg(test)] -mod tests { - use arrow::array::ArrayData; - use arrow_array::Array; - use arrow_array::{make_array, ArrayRef, MapArray, StringArray, StructArray}; - use arrow_buffer::{Buffer, ToByteSlice}; - use arrow_schema::Field; - - use super::*; - use std::collections::HashMap; - use std::sync::Arc; - - #[test] - fn delta_log_schema_for_table_test() { - // NOTE: We should future proof the checkpoint schema in case action schema changes. - // See https://github.com/delta-io/delta-rs/issues/287 - - let table_schema = ArrowSchema::new(vec![ - ArrowField::new("pcol", ArrowDataType::Int32, true), - ArrowField::new("col1", ArrowDataType::Int32, true), - ]); - let partition_columns = vec!["pcol".to_string()]; - let log_schema = - delta_log_schema_for_table(table_schema.clone(), partition_columns.as_slice(), false); - - // verify top-level schema contains all expected fields and they are named correctly. - let expected_fields = ["metaData", "protocol", "txn", "remove", "add"]; - for f in log_schema.fields().iter() { - assert!(expected_fields.contains(&f.name().as_str())); - } - assert_eq!(5, log_schema.fields().len()); - - // verify add fields match as expected. a lot of transformation goes into these. - let add_fields: Vec<_> = log_schema - .fields() - .iter() - .filter(|f| f.name() == "add") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .collect(); - let field_names: Vec<&String> = add_fields.iter().map(|v| v.name()).collect(); - assert_eq!( - vec![ - "path", - "size", - "modificationTime", - "dataChange", - "stats", - "partitionValues", - "tags", - "deletionVector", - "stats_parsed", - "partitionValues_parsed" - ], - field_names - ); - let add_field_map: HashMap<_, _> = add_fields - .iter() - .map(|f| (f.name().to_owned(), f.clone())) - .collect(); - let partition_values_parsed = add_field_map.get("partitionValues_parsed").unwrap(); - if let ArrowDataType::Struct(fields) = partition_values_parsed.data_type() { - assert_eq!(1, fields.len()); - let field = fields.get(0).unwrap().to_owned(); - assert_eq!( - Arc::new(ArrowField::new("pcol", ArrowDataType::Int32, true)), - field - ); - } else { - unreachable!(); - } - let stats_parsed = add_field_map.get("stats_parsed").unwrap(); - if let ArrowDataType::Struct(fields) = stats_parsed.data_type() { - assert_eq!(4, fields.len()); - - let field_map: HashMap<_, _> = fields - .iter() - .map(|f| (f.name().to_owned(), f.clone())) - .collect(); - - for (k, v) in field_map.iter() { - match k.as_ref() { - "minValues" | "maxValues" | "nullCount" => match v.data_type() { - ArrowDataType::Struct(fields) => { - assert_eq!(1, fields.len()); - let field = fields.get(0).unwrap().to_owned(); - let data_type = if k == "nullCount" { - ArrowDataType::Int64 - } else { - ArrowDataType::Int32 - }; - assert_eq!(Arc::new(ArrowField::new("col1", data_type, true)), field); - } - _ => unreachable!(), - }, - "numRecords" => {} - _ => panic!(), - } - } - } else { - unreachable!(); - } - - // verify extended remove schema fields **ARE NOT** included when `use_extended_remove_schema` is false. - let num_remove_fields = log_schema - .fields() - .iter() - .filter(|f| f.name() == "remove") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .count(); - assert_eq!(4, num_remove_fields); - - // verify extended remove schema fields **ARE** included when `use_extended_remove_schema` is true. - let log_schema = - delta_log_schema_for_table(table_schema, partition_columns.as_slice(), true); - let remove_fields: Vec<_> = log_schema - .fields() - .iter() - .filter(|f| f.name() == "remove") - .flat_map(|f| { - if let ArrowDataType::Struct(fields) = f.data_type() { - fields.iter().cloned() - } else { - unreachable!(); - } - }) - .collect(); - assert_eq!(7, remove_fields.len()); - let expected_fields = [ - "path", - "deletionTimestamp", - "dataChange", - "extendedFileMetadata", - "partitionValues", - "size", - "tags", - ]; - for f in remove_fields.iter() { - assert!(expected_fields.contains(&f.name().as_str())); - } - } - - #[test] - fn test_arrow_from_delta_decimal_type() { - let precision = 20; - let scale = 2; - let decimal_field = DataType::Primitive(PrimitiveType::Decimal(precision, scale)); - assert_eq!( - >::try_from(&decimal_field).unwrap(), - ArrowDataType::Decimal128(precision as u8, scale as i8) - ); - } - - #[test] - fn test_arrow_from_delta_timestamp_type() { - let timestamp_field = DataType::Primitive(PrimitiveType::Timestamp); - assert_eq!( - >::try_from(×tamp_field).unwrap(), - ArrowDataType::Timestamp(TimeUnit::Microsecond, None) - ); - } - - #[test] - fn test_delta_from_arrow_timestamp_type() { - let timestamp_field = ArrowDataType::Timestamp(TimeUnit::Microsecond, None); - assert_eq!( - >::try_from(×tamp_field).unwrap(), - DataType::Primitive(PrimitiveType::Timestamp) - ); - } - - #[test] - fn test_delta_from_arrow_timestamp_type_with_tz() { - let timestamp_field = - ArrowDataType::Timestamp(TimeUnit::Microsecond, Some("UTC".to_string().into())); - assert_eq!( - >::try_from(×tamp_field).unwrap(), - DataType::Primitive(PrimitiveType::Timestamp) - ); - } - - #[test] - fn test_delta_from_arrow_map_type() { - let arrow_map = ArrowDataType::Map( - Arc::new(ArrowField::new( - "entries", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Int8, false), - ArrowField::new("value", ArrowDataType::Binary, true), - ] - .into(), - ), - false, - )), - false, - ); - let converted_map: DataType = (&arrow_map).try_into().unwrap(); - - assert_eq!( - converted_map, - DataType::Map(Box::new(MapType::new( - DataType::Primitive(PrimitiveType::Byte), - DataType::Primitive(PrimitiveType::Binary), - true, - ))) - ); - } - - #[test] - fn test_record_batch_from_map_type() { - let keys = vec!["0", "1", "5", "6", "7"]; - let values: Vec<&[u8]> = vec![ - b"test_val_1", - b"test_val_2", - b"long_test_val_3", - b"4", - b"test_val_5", - ]; - let entry_offsets = vec![0u32, 1, 1, 4, 5, 5]; - let num_rows = keys.len(); - - // Copied the function `new_from_string` with the patched code from https://github.com/apache/arrow-rs/pull/4808 - // This should be reverted back [`MapArray::new_from_strings`] once arrow is upgraded in this project. - fn new_from_strings<'a>( - keys: impl Iterator, - values: &dyn Array, - entry_offsets: &[u32], - ) -> Result { - let entry_offsets_buffer = Buffer::from(entry_offsets.to_byte_slice()); - let keys_data = StringArray::from_iter_values(keys); - - let keys_field = Arc::new(Field::new("keys", ArrowDataType::Utf8, false)); - let values_field = Arc::new(Field::new( - "values", - values.data_type().clone(), - values.null_count() > 0, - )); - - let entry_struct = StructArray::from(vec![ - (keys_field, Arc::new(keys_data) as ArrayRef), - (values_field, make_array(values.to_data())), - ]); - - let map_data_type = ArrowDataType::Map( - Arc::new(Field::new( - "entries", - entry_struct.data_type().clone(), - false, - )), - false, - ); - - let map_data = ArrayData::builder(map_data_type) - .len(entry_offsets.len() - 1) - .add_buffer(entry_offsets_buffer) - .add_child_data(entry_struct.into_data()) - .build()?; - - Ok(MapArray::from(map_data)) - } - - let map_array = new_from_strings( - keys.into_iter(), - &arrow::array::BinaryArray::from(values), - entry_offsets.as_slice(), - ) - .expect("Could not create a map array"); - - let schema = - >::try_from(&StructType::new(vec![ - StructField::new( - "example".to_string(), - DataType::Map(Box::new(MapType::new( - DataType::Primitive(PrimitiveType::String), - DataType::Primitive(PrimitiveType::Binary), - false, - ))), - false, - ), - ])) - .expect("Could not get schema"); - - let record_batch = - arrow::record_batch::RecordBatch::try_new(Arc::new(schema), vec![Arc::new(map_array)]) - .expect("Failed to create RecordBatch"); - - assert_eq!(record_batch.num_columns(), 1); - assert_eq!(record_batch.num_rows(), num_rows); - } - - #[test] - fn test_max_min_schema_for_fields() { - let mut max_min_vec: Vec = Vec::new(); - let fields = [ - ArrowField::new("simple", ArrowDataType::Int32, true), - ArrowField::new( - "struct", - ArrowDataType::Struct( - vec![ArrowField::new("simple", ArrowDataType::Int32, true)].into(), - ), - true, - ), - ArrowField::new( - "list", - ArrowDataType::List(Arc::new(ArrowField::new( - "simple", - ArrowDataType::Int32, - true, - ))), - true, - ), - ArrowField::new( - "map", - ArrowDataType::Map( - Arc::new(ArrowField::new( - "struct", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Int32, true), - ArrowField::new("value", ArrowDataType::Int32, true), - ] - .into(), - ), - true, - )), - true, - ), - true, - ), - ArrowField::new("binary", ArrowDataType::Binary, true), - ]; - - let expected = vec![fields[0].clone(), fields[1].clone()]; - - fields - .iter() - .for_each(|f| max_min_schema_for_fields(&mut max_min_vec, f)); - - assert_eq!(max_min_vec, expected); - } - - #[test] - fn test_null_count_schema_for_fields() { - let mut null_count_vec: Vec = Vec::new(); - let fields = [ - ArrowField::new("int32", ArrowDataType::Int32, true), - ArrowField::new("int64", ArrowDataType::Int64, true), - ArrowField::new("Utf8", ArrowDataType::Utf8, true), - ArrowField::new( - "list", - ArrowDataType::List(Arc::new(ArrowField::new( - "simple", - ArrowDataType::Int32, - true, - ))), - true, - ), - ArrowField::new( - "map", - ArrowDataType::Map( - Arc::new(ArrowField::new( - "struct", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Int32, true), - ArrowField::new("value", ArrowDataType::Int32, true), - ] - .into(), - ), - true, - )), - true, - ), - true, - ), - ArrowField::new( - "struct", - ArrowDataType::Struct( - vec![ArrowField::new("int32", ArrowDataType::Int32, true)].into(), - ), - true, - ), - ]; - let expected = vec![ - ArrowField::new(fields[0].name(), ArrowDataType::Int64, true), - ArrowField::new(fields[1].name(), ArrowDataType::Int64, true), - ArrowField::new(fields[2].name(), ArrowDataType::Int64, true), - ArrowField::new(fields[3].name(), ArrowDataType::Int64, true), - ArrowField::new(fields[4].name(), ArrowDataType::Int64, true), - ArrowField::new( - fields[5].name(), - ArrowDataType::Struct( - vec![ArrowField::new("int32", ArrowDataType::Int64, true)].into(), - ), - true, - ), - ]; - fields - .iter() - .for_each(|f| null_count_schema_for_fields(&mut null_count_vec, f)); - assert_eq!(null_count_vec, expected); - } - - /* - * This test validates the trait implementation of - * TryFrom<&Arc> for schema::SchemaField which is required with Arrow 37 since - * iterators on Fields will give an &Arc - */ - #[test] - fn tryfrom_arrowfieldref_with_structs() { - let field = Arc::new(ArrowField::new( - "test_struct", - ArrowDataType::Struct( - vec![ - ArrowField::new("key", ArrowDataType::Int32, true), - ArrowField::new("value", ArrowDataType::Int32, true), - ] - .into(), - ), - true, - )); - let _converted: StructField = field.as_ref().try_into().unwrap(); - } -} diff --git a/crates/deltalake-core/src/storage/mod.rs b/crates/deltalake-core/src/storage/mod.rs index b571905f8b..2fa755be1c 100644 --- a/crates/deltalake-core/src/storage/mod.rs +++ b/crates/deltalake-core/src/storage/mod.rs @@ -6,6 +6,7 @@ use lazy_static::lazy_static; pub mod config; pub mod file; +pub mod path; pub mod utils; #[cfg(any(feature = "s3", feature = "s3-native-tls"))] diff --git a/crates/deltalake-core/src/storage/path.rs b/crates/deltalake-core/src/storage/path.rs new file mode 100644 index 0000000000..52e79758de --- /dev/null +++ b/crates/deltalake-core/src/storage/path.rs @@ -0,0 +1,154 @@ +#![allow(dead_code)] +//! Utilities to make working with directory and file paths easier + +use lazy_static::lazy_static; +use regex::Regex; +use url::Url; + +use crate::{DeltaResult, DeltaTableError}; +use object_store::path::Path; + +/// The delimiter to separate object namespaces, creating a directory structure. +const DELIMITER: &str = "/"; + +lazy_static! { + static ref CHECKPOINT_FILE_PATTERN: Regex = + Regex::new(r"\d+\.checkpoint(\.\d+\.\d+)?\.parquet").unwrap(); + static ref DELTA_FILE_PATTERN: Regex = Regex::new(r"\d+\.json").unwrap(); +} + +/// The metadata that describes an object. +#[derive(Debug, Clone, PartialEq, Eq)] +pub(crate) struct FileMeta { + /// The fully qualified path to the object + pub location: LogPath, + /// The last modified time + pub last_modified: i64, + /// The size in bytes of the object + pub size: usize, +} + +/// A file paths defined in the delta log +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum LogPath { + /// Path relkative to the table root + ObjectStore(Path), + /// fully qualified url for file path + Url(Url), +} + +impl LogPath { + pub(crate) fn child(&self, path: impl AsRef) -> DeltaResult { + match self { + LogPath::ObjectStore(p) => Ok(Self::ObjectStore(p.child(path.as_ref()))), + LogPath::Url(url) => Ok(Self::Url(url.join(path.as_ref()).map_err(|err| { + DeltaTableError::GenericError { + source: Box::new(err), + } + })?)), + } + } + + /// Returns the last path segment containing the filename stored in this [`LogPath`] + pub(crate) fn filename(&self) -> Option<&str> { + match self { + LogPath::ObjectStore(p) => p.filename(), + LogPath::Url(url) => match url.path().is_empty() || url.path().ends_with('/') { + true => None, + false => url.path().split(DELIMITER).last(), + }, + } + } + + pub(crate) fn is_checkpoint_file(&self) -> bool { + self.filename().map(is_checkpoint_file).unwrap_or(false) + } + + pub(crate) fn is_commit_file(&self) -> bool { + self.filename().map(is_commit_file).unwrap_or(false) + } + + /// Parse the version number assuming a commit json or checkpoint parquet file + pub(crate) fn commit_version(&self) -> Option { + self.filename().and_then(commit_version) + } +} + +pub(crate) fn is_checkpoint_file(path: &str) -> bool { + CHECKPOINT_FILE_PATTERN.captures(path).is_some() +} + +pub(crate) fn is_commit_file(path: &str) -> bool { + DELTA_FILE_PATTERN.captures(path).is_some() +} + +pub(crate) fn commit_version(path: &str) -> Option { + path.split_once('.').and_then(|(name, _)| name.parse().ok()) +} + +// impl<'a> AsRef for LogPath<'a> { +// fn as_ref(&self) -> &Url { +// self.0 +// } +// } + +impl AsRef for LogPath { + fn as_ref(&self) -> &str { + match self { + LogPath::ObjectStore(p) => p.as_ref(), + LogPath::Url(url) => url.as_str(), + } + } +} + +#[cfg(test)] +mod tests { + use object_store::path::Path; + use std::path::PathBuf; + + use super::*; + + fn table_url() -> Url { + let path = + std::fs::canonicalize(PathBuf::from("./tests/data/table-with-dv-small/")).unwrap(); + url::Url::from_file_path(path).unwrap() + } + + fn test_path(log_path: LogPath) { + let path = log_path + .child("_delta_log") + .unwrap() + .child("00000000000000000000.json") + .unwrap(); + + assert_eq!("00000000000000000000.json", path.filename().unwrap()); + assert!(path.is_commit_file()); + assert!(!path.is_checkpoint_file()); + assert_eq!(path.commit_version(), Some(0)); + + let path = log_path.child("00000000000000000005.json").unwrap(); + assert_eq!(path.commit_version(), Some(5)); + + let path = log_path + .child("00000000000000000002.checkpoint.parquet") + .unwrap(); + assert_eq!( + "00000000000000000002.checkpoint.parquet", + path.filename().unwrap() + ); + assert!(!path.is_commit_file()); + assert!(path.is_checkpoint_file()); + assert_eq!(path.commit_version(), Some(2)); + } + + #[test] + fn test_file_patterns() { + let path = + std::fs::canonicalize(PathBuf::from("./tests/data/table-with-dv-small/")).unwrap(); + let log_path = LogPath::Url(url::Url::from_file_path(path.clone()).unwrap()); + test_path(log_path); + + let log_path = LogPath::ObjectStore(Path::from_filesystem_path(path).unwrap()); + test_path(log_path); + } +} diff --git a/crates/deltalake-core/src/table/config.rs b/crates/deltalake-core/src/table/config.rs index 3fa021ce6e..81011f3c9a 100644 --- a/crates/deltalake-core/src/table/config.rs +++ b/crates/deltalake-core/src/table/config.rs @@ -1,4 +1,5 @@ //! Delta Table configuration +use std::convert::Infallible; use std::time::Duration; use std::{collections::HashMap, str::FromStr}; @@ -11,6 +12,7 @@ use crate::errors::DeltaTableError; /// /// #[derive(PartialEq, Eq, Hash)] +#[non_exhaustive] pub enum DeltaConfigKey { /// true for this Delta table to be append-only. If append-only, /// existing records cannot be deleted, and existing values cannot be updated. @@ -100,6 +102,9 @@ pub enum DeltaConfigKey { /// The target file size in bytes or higher units for file tuning. For example, 104857600 (bytes) or 100mb. TuneFileSizesForRewrites, + + /// 'classic' for classic Delta Lake checkpoints. 'v2' for v2 checkpoints. + CheckpointPolicy, } impl AsRef for DeltaConfigKey { @@ -111,6 +116,7 @@ impl AsRef for DeltaConfigKey { Self::AutoOptimizeOptimizeWrite => "delta.autoOptimize.optimizeWrite", Self::CheckpointWriteStatsAsJson => "delta.checkpoint.writeStatsAsJson", Self::CheckpointWriteStatsAsStruct => "delta.checkpoint.writeStatsAsStruct", + Self::CheckpointPolicy => "delta.checkpointPolicy", Self::ColumnMappingMode => "delta.columnMapping.mode", Self::DataSkippingNumIndexedCols => "delta.dataSkippingNumIndexedCols", Self::DeletedFileRetentionDuration => "delta.deletedFileRetentionDuration", @@ -140,6 +146,7 @@ impl FromStr for DeltaConfigKey { "delta.autoOptimize.optimizeWrite" => Ok(Self::AutoOptimizeOptimizeWrite), "delta.checkpoint.writeStatsAsJson" => Ok(Self::CheckpointWriteStatsAsJson), "delta.checkpoint.writeStatsAsStruct" => Ok(Self::CheckpointWriteStatsAsStruct), + "delta.checkpointPolicy" => Ok(Self::CheckpointPolicy), "delta.columnMapping.mode" => Ok(Self::ColumnMappingMode), "delta.dataSkippingNumIndexedCols" => Ok(Self::DataSkippingNumIndexedCols), "delta.deletedFileRetentionDuration" | "deletedFileRetentionDuration" => { @@ -280,6 +287,14 @@ impl<'a> TableConfig<'a> { .and_then(|o| o.as_ref().and_then(|v| v.parse().ok())) .unwrap_or_default() } + + /// Policy applied during chepoint creation + pub fn checkpoint_policy(&self) -> CheckpointPolicy { + self.0 + .get(DeltaConfigKey::CheckpointPolicy.as_ref()) + .and_then(|o| o.as_ref().and_then(|v| v.parse().ok())) + .unwrap_or_default() + } } #[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] @@ -338,6 +353,46 @@ impl FromStr for IsolationLevel { } } +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] +/// The checkpoint policy applied when writing checkpoints +#[serde(rename_all = "camelCase")] +pub enum CheckpointPolicy { + /// classic Delta Lake checkpoints + Classic, + /// v2 checkpoints + V2, + /// unknown checkpoint policy + Other(String), +} + +impl Default for CheckpointPolicy { + fn default() -> Self { + Self::Classic + } +} + +impl AsRef for CheckpointPolicy { + fn as_ref(&self) -> &str { + match self { + Self::Classic => "classic", + Self::V2 => "v2", + Self::Other(s) => s, + } + } +} + +impl FromStr for CheckpointPolicy { + type Err = Infallible; + + fn from_str(s: &str) -> Result { + match s.to_ascii_lowercase().as_str() { + "classic" => Ok(Self::Classic), + "v2" => Ok(Self::V2), + other => Ok(Self::Other(other.to_string())), + } + } +} + const SECONDS_PER_MINUTE: u64 = 60; const SECONDS_PER_HOUR: u64 = 60 * SECONDS_PER_MINUTE; const SECONDS_PER_DAY: u64 = 24 * SECONDS_PER_HOUR; diff --git a/crates/deltalake-core/src/table/state.rs b/crates/deltalake-core/src/table/state.rs index 8fa51c55fd..8a296e6a7a 100644 --- a/crates/deltalake-core/src/table/state.rs +++ b/crates/deltalake-core/src/table/state.rs @@ -12,6 +12,8 @@ use serde::{Deserialize, Serialize}; use super::config::TableConfig; use crate::errors::DeltaTableError; +use crate::kernel::error::DeltaResult; +use crate::kernel::snapshot::Snapshot; use crate::kernel::{ Action, Add, CommitInfo, DataType, DomainMetadata, ReaderFeatures, Remove, StructType, WriterFeatures, @@ -40,18 +42,76 @@ pub struct DeltaTableState { commit_infos: Vec, // Domain metadatas provided by the system or user domain_metadatas: Vec, + // last txn version for each app id app_transaction_version: HashMap, - min_reader_version: i32, - min_writer_version: i32, - reader_features: Option>, - writer_features: Option>, // table metadata corresponding to current version current_metadata: Option, - // retention period for tombstones in milli-seconds - tombstone_retention_millis: i64, - // retention period for log entries in milli-seconds - log_retention_millis: i64, - enable_expired_log_cleanup: bool, + // protocol information corresponding to current version + current_protocol: Option, +} + +impl std::fmt::Display for DeltaTableState { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{:?}", self.schema()) + } +} + +impl Snapshot for DeltaTableState { + fn version(&self) -> i64 { + self.version + } + + fn schema(&self) -> Option<&StructType> { + self.schema() + } + + fn files(&self) -> DeltaResult + Send + '_>> { + Ok(Box::new(self.files().clone().into_iter())) + } + + fn table_config(&self) -> TableConfig<'_> { + self.table_config() + } + + fn protocol(&self) -> DeltaResult { + todo!() + } + + fn metadata(&self) -> DeltaResult { + Ok(crate::kernel::Metadata { + id: self + .current_metadata + .as_ref() + .map(|m| m.id.clone()) + .unwrap_or("00000000-0000-0000-0000-000000000000".into()), + name: self.current_metadata.as_ref().and_then(|m| m.name.clone()), + description: self + .current_metadata + .as_ref() + .and_then(|m| m.description.clone()), + format: self + .current_metadata + .as_ref() + .map(|m| m.format.clone()) + .unwrap_or_default(), + schema_string: self + .current_metadata + .as_ref() + .map(|m| serde_json::to_string(&m.schema).unwrap()) + .unwrap_or_default(), + partition_columns: self + .current_metadata + .as_ref() + .map(|m| m.partition_columns.clone()) + .unwrap_or_default(), + configuration: self + .current_metadata + .as_ref() + .map(|m| m.configuration.clone()) + .unwrap_or_default(), + created_time: self.current_metadata.as_ref().and_then(|m| m.created_time), + }) + } } impl DeltaTableState { @@ -173,21 +233,6 @@ impl DeltaTableState { &self.commit_infos } - /// Retention of tombstone in milliseconds. - pub fn tombstone_retention_millis(&self) -> i64 { - self.tombstone_retention_millis - } - - /// Retention of logs in milliseconds. - pub fn log_retention_millis(&self) -> i64 { - self.log_retention_millis - } - - /// Whether to clean up expired checkpoints and delta logs. - pub fn enable_expired_log_cleanup(&self) -> bool { - self.enable_expired_log_cleanup - } - /// Full list of tombstones (remove actions) representing files removed from table state). pub fn all_tombstones(&self) -> &HashSet { &self.tombstones @@ -196,7 +241,11 @@ impl DeltaTableState { /// List of unexpired tombstones (remove actions) representing files removed from table state. /// The retention period is set by `deletedFileRetentionDuration` with default value of 1 week. pub fn unexpired_tombstones(&self) -> impl Iterator { - let retention_timestamp = Utc::now().timestamp_millis() - self.tombstone_retention_millis; + let retention_timestamp = Utc::now().timestamp_millis() + - self + .table_config() + .deleted_file_retention_duration() + .as_millis() as i64; self.tombstones .iter() .filter(move |t| t.deletion_timestamp.unwrap_or(0) > retention_timestamp) @@ -225,22 +274,32 @@ impl DeltaTableState { /// The min reader version required by the protocol. pub fn min_reader_version(&self) -> i32 { - self.min_reader_version + self.current_protocol + .as_ref() + .map(|p| p.min_reader_version.clone()) + .unwrap_or(0) } /// The min writer version required by the protocol. pub fn min_writer_version(&self) -> i32 { - self.min_writer_version + self.current_protocol + .as_ref() + .map(|p| p.min_writer_version.clone()) + .unwrap_or(0) } /// Current supported reader features pub fn reader_features(&self) -> Option<&HashSet> { - self.reader_features.as_ref() + self.current_protocol + .as_ref() + .and_then(|p| p.reader_features.as_ref()) } /// Current supported writer features pub fn writer_features(&self) -> Option<&HashSet> { - self.writer_features.as_ref() + self.current_protocol + .as_ref() + .and_then(|p| p.writer_features.as_ref()) } /// The most recent metadata of the table. @@ -299,23 +358,11 @@ impl DeltaTableState { self.files.append(&mut new_state.files); } - if new_state.min_reader_version > 0 { - self.min_reader_version = new_state.min_reader_version; - self.min_writer_version = new_state.min_writer_version; - } - - if new_state.min_writer_version >= 5 { - self.writer_features = new_state.writer_features; - } - - if new_state.min_reader_version >= 3 { - self.reader_features = new_state.reader_features; + if new_state.current_protocol.is_some() { + self.current_protocol = new_state.current_protocol.take(); } if new_state.current_metadata.is_some() { - self.tombstone_retention_millis = new_state.tombstone_retention_millis; - self.log_retention_millis = new_state.log_retention_millis; - self.enable_expired_log_cleanup = new_state.enable_expired_log_cleanup; self.current_metadata = new_state.current_metadata.take(); } @@ -359,20 +406,10 @@ impl DeltaTableState { } } Action::Protocol(v) => { - self.min_reader_version = v.min_reader_version; - self.min_writer_version = v.min_writer_version; - self.reader_features = v.reader_features; - self.writer_features = v.writer_features; + self.current_protocol = Some(v); } Action::Metadata(v) => { - let md = DeltaTableMetaData::try_from(v)?; - let table_config = TableConfig(&md.configuration); - self.tombstone_retention_millis = - table_config.deleted_file_retention_duration().as_millis() as i64; - self.log_retention_millis = - table_config.log_retention_duration().as_millis() as i64; - self.enable_expired_log_cleanup = table_config.enable_expired_log_cleanup(); - self.current_metadata = Some(md); + self.current_metadata = Some(DeltaTableMetaData::try_from(v)?); } Action::Txn(v) => { *self @@ -444,14 +481,8 @@ mod tests { commit_infos: vec![], domain_metadatas: vec![], app_transaction_version: Default::default(), - min_reader_version: 0, - min_writer_version: 0, - reader_features: None, - writer_features: None, current_metadata: None, - tombstone_retention_millis: 0, - log_retention_millis: 0, - enable_expired_log_cleanup: false, + current_protocol: None, }; let bytes = serde_json::to_vec(&expected).unwrap(); let actual: DeltaTableState = serde_json::from_slice(&bytes).unwrap(); @@ -470,15 +501,9 @@ mod tests { commit_infos: vec![], domain_metadatas: vec![], tombstones: HashSet::new(), - current_metadata: None, - min_reader_version: 1, - min_writer_version: 1, - reader_features: None, - writer_features: None, app_transaction_version, - tombstone_retention_millis: 0, - log_retention_millis: 0, - enable_expired_log_cleanup: true, + current_metadata: None, + current_protocol: None, }; let txn_action = Action::Txn(Txn { diff --git a/crates/deltalake-core/src/table/state_arrow.rs b/crates/deltalake-core/src/table/state_arrow.rs index 9d82c87326..8b20fecd5d 100644 --- a/crates/deltalake-core/src/table/state_arrow.rs +++ b/crates/deltalake-core/src/table/state_arrow.rs @@ -708,6 +708,7 @@ fn json_value_to_array_general<'a>( .map(|value| value.and_then(|value| value.as_str().map(|value| value.as_bytes()))) .collect_vec(), ))), + // TODO should this actually be the type for timestamp without timezone? DataType::Timestamp(TimeUnit::Microsecond, None) => { Ok(Arc::new(TimestampMicrosecondArray::from( values @@ -719,6 +720,21 @@ fn json_value_to_array_general<'a>( .collect_vec(), ))) } + DataType::Timestamp(TimeUnit::Microsecond, Some(tz)) + if tz.eq_ignore_ascii_case("utc") + || tz.eq_ignore_ascii_case("+00:00") + || tz.eq_ignore_ascii_case("-00:00") => + { + Ok(Arc::new(TimestampMicrosecondArray::from( + values + .map(|value| { + value.and_then(|value| { + value.as_str().and_then(TimestampMicrosecondType::parse) + }) + }) + .collect_vec(), + ))) + } DataType::Date32 => Ok(Arc::new(Date32Array::from( values .map(|value| value.and_then(|value| value.as_str().and_then(Date32Type::parse))) diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-0 b/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-0 deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-1 b/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-1 deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-2 b/crates/deltalake-core/tests/data/table_with_liquid_clustering/_delta_log/.s3-optimization-2 deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/crates/deltalake-core/tests/integration_concurrent_writes.rs b/crates/deltalake-core/tests/integration_concurrent_writes.rs index 90dba7659a..79c16e85dc 100644 --- a/crates/deltalake-core/tests/integration_concurrent_writes.rs +++ b/crates/deltalake-core/tests/integration_concurrent_writes.rs @@ -69,7 +69,7 @@ async fn prepare_table( assert_eq!(0, table.version()); assert_eq!(1, table.get_min_reader_version()); - assert_eq!(1, table.get_min_writer_version()); + assert_eq!(2, table.get_min_writer_version()); assert_eq!(0, table.get_files().len()); Ok((table, table_uri)) diff --git a/crates/deltalake-core/tests/integration_datafusion.rs b/crates/deltalake-core/tests/integration_datafusion.rs index 7a9c38463f..b53b16a15f 100644 --- a/crates/deltalake-core/tests/integration_datafusion.rs +++ b/crates/deltalake-core/tests/integration_datafusion.rs @@ -30,7 +30,7 @@ use datafusion_proto::bytes::{ }; use url::Url; -use deltalake_core::delta_datafusion::{DeltaPhysicalCodec, DeltaScan}; +use deltalake_core::delta_datafusion::{scan::DeltaScan, DeltaPhysicalCodec}; use deltalake_core::kernel::{DataType, MapType, PrimitiveType, StructField, StructType}; use deltalake_core::operations::create::CreateBuilder; use deltalake_core::protocol::SaveMode; diff --git a/crates/deltalake-sql/src/parser.rs b/crates/deltalake-sql/src/parser.rs index c76cced9bd..3287c87215 100644 --- a/crates/deltalake-sql/src/parser.rs +++ b/crates/deltalake-sql/src/parser.rs @@ -63,10 +63,10 @@ impl fmt::Display for Statement { } } -/// Delta Lake SQL Parser based on [`sqlparser`] +/// Delta Lake SQL Parser based on [`sqlparser`](https://crates.io/crates/sqlparser) /// /// This parser handles Delta Lake specific statements, delegating to -/// [`DFParser`](datafusion_sql::parser::DFParser) for other SQL statements. +/// [`DFParser`]for other SQL statements. pub struct DeltaParser<'a> { sql: &'a str, parser: Parser<'a>,