From ba8309eee68da96147ee079de8f6c95bb03bdc6f Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 1 Nov 2024 12:01:44 -0700 Subject: [PATCH 01/20] checkpoint with serde but think i need to change that --- kernel/src/actions/mod.rs | 9 +- kernel/src/lib.rs | 3 +- kernel/src/scan/mod.rs | 6 +- kernel/src/scan/state.rs | 2 +- kernel/src/schema.rs | 2 +- kernel/src/snapshot.rs | 21 +- .../column_mapping.rs | 5 +- .../src/{features => table_features}/mod.rs | 1 - kernel/src/table_properties.rs | 736 ++++++++++++++++++ 9 files changed, 763 insertions(+), 22 deletions(-) rename kernel/src/{features => table_features}/column_mapping.rs (85%) rename kernel/src/{features => table_features}/mod.rs (98%) create mode 100644 kernel/src/table_properties.rs diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index b23dd6511..54a6d9531 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -9,7 +9,8 @@ use visitors::{AddVisitor, MetadataVisitor, ProtocolVisitor}; use self::deletion_vector::DeletionVectorDescriptor; use crate::actions::schemas::GetStructField; -use crate::features::{ReaderFeatures, WriterFeatures}; +use crate::table_properties::TableProperties; +use crate::table_features::{ReaderFeatures, WriterFeatures}; use crate::schema::{SchemaRef, StructType}; use crate::{DeltaResult, EngineData}; @@ -99,7 +100,7 @@ pub struct Metadata { pub partition_columns: Vec, /// The time when this metadata action is created, in milliseconds since the Unix epoch pub created_time: Option, - /// Configuration options for the metadata action + /// Configuration options for the metadata action. These are parsed into `TableProperties`. pub configuration: HashMap, } @@ -113,6 +114,10 @@ impl Metadata { pub fn schema(&self) -> DeltaResult { Ok(serde_json::from_str(&self.schema_string)?) } + + pub fn get_table_properties(&self) -> DeltaResult { + todo!() + } } #[derive(Default, Debug, Clone, PartialEq, Eq, Schema, Serialize, Deserialize)] diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index 2f686a3ad..593a110a2 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -62,7 +62,7 @@ pub mod actions; pub mod engine_data; pub mod error; pub mod expressions; -pub mod features; +pub mod table_features; #[cfg(feature = "developer-visibility")] pub mod path; @@ -74,6 +74,7 @@ pub mod schema; pub mod snapshot; pub mod table; pub mod transaction; +pub mod table_properties; pub(crate) mod utils; pub use delta_kernel_derive; diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index 78a3c74ae..8863aab39 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -10,7 +10,7 @@ use url::Url; use crate::actions::deletion_vector::{split_vector, treemap_to_bools, DeletionVectorDescriptor}; use crate::actions::{get_log_add_schema, get_log_schema, ADD_NAME, REMOVE_NAME}; use crate::expressions::{ColumnName, Expression, ExpressionRef, Scalar}; -use crate::features::ColumnMappingMode; +use crate::table_features::ColumnMappingMode; use crate::scan::state::{DvInfo, Stats}; use crate::schema::{DataType, Schema, SchemaRef, StructField, StructType}; use crate::snapshot::Snapshot; @@ -95,7 +95,7 @@ impl ScanBuilder { let (all_fields, read_fields, have_partition_cols) = get_state_info( logical_schema.as_ref(), &self.snapshot.metadata().partition_columns, - self.snapshot.column_mapping_mode, + self.snapshot.table_properties().column_mapping_mode, )?; let physical_schema = Arc::new(StructType::new(read_fields)); Ok(Scan { @@ -247,7 +247,7 @@ impl Scan { partition_columns: self.snapshot.metadata().partition_columns.clone(), logical_schema: self.logical_schema.clone(), read_schema: self.physical_schema.clone(), - column_mapping_mode: self.snapshot.column_mapping_mode, + column_mapping_mode: self.snapshot.table_properties().column_mapping_mode, } } diff --git a/kernel/src/scan/state.rs b/kernel/src/scan/state.rs index 2dc30f232..e8cf1db16 100644 --- a/kernel/src/scan/state.rs +++ b/kernel/src/scan/state.rs @@ -8,7 +8,7 @@ use crate::{ visitors::visit_deletion_vector_at, }, engine_data::{GetData, TypedGetData}, - features::ColumnMappingMode, + table_features::ColumnMappingMode, schema::SchemaRef, DataVisitor, DeltaResult, Engine, EngineData, Error, }; diff --git a/kernel/src/schema.rs b/kernel/src/schema.rs index 61dd6e05b..dd7a37fe5 100644 --- a/kernel/src/schema.rs +++ b/kernel/src/schema.rs @@ -9,7 +9,7 @@ use indexmap::IndexMap; use itertools::Itertools; use serde::{Deserialize, Serialize}; -use crate::features::ColumnMappingMode; +use crate::table_features::ColumnMappingMode; use crate::utils::require; use crate::{DeltaResult, Error}; diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 93d808585..b3a42c2c0 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -12,10 +12,11 @@ use url::Url; use crate::actions::{get_log_schema, Metadata, Protocol, METADATA_NAME, PROTOCOL_NAME}; use crate::expressions::column_expr; -use crate::features::{ColumnMappingMode, COLUMN_MAPPING_MODE_KEY}; use crate::path::ParsedLogPath; use crate::scan::ScanBuilder; use crate::schema::{Schema, SchemaRef}; +use crate::table_features::ColumnMappingMode; +use crate::table_properties::TableProperties; use crate::utils::require; use crate::{DeltaResult, Engine, Error, FileMeta, FileSystemClient, Version}; use crate::{EngineData, Expression, ExpressionRef}; @@ -134,7 +135,7 @@ pub struct Snapshot { metadata: Metadata, protocol: Protocol, schema: Schema, - pub(crate) column_mapping_mode: ColumnMappingMode, + table_properties: TableProperties, } impl Drop for Snapshot { @@ -230,10 +231,12 @@ impl Snapshot { .read_metadata(engine)? .ok_or(Error::MissingMetadata)?; let schema = metadata.schema()?; - let column_mapping_mode = match metadata.configuration.get(COLUMN_MAPPING_MODE_KEY) { - Some(mode) if protocol.min_reader_version >= 2 => mode.as_str().try_into(), - _ => Ok(ColumnMappingMode::None), - }?; + // FIXME + // let column_mapping_mode = match metadata.configuration.get(COLUMN_MAPPING_MODE_KEY) { + // Some(mode) if protocol.min_reader_version >= 2 => mode.as_str().try_into(), + // _ => Ok(ColumnMappingMode::None), + // }?; + let table_properties = metadata.get_table_properties()?; Ok(Self { table_root: location, log_segment, @@ -241,7 +244,7 @@ impl Snapshot { metadata, protocol, schema, - column_mapping_mode, + table_properties, }) } @@ -278,8 +281,8 @@ impl Snapshot { /// Get the [column mapping /// mode](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#column-mapping) at this /// `Snapshot`s version. - pub fn column_mapping_mode(&self) -> ColumnMappingMode { - self.column_mapping_mode + pub fn table_properties(&self) -> &TableProperties { + &self.table_properties } /// Create a [`ScanBuilder`] for an `Arc`. diff --git a/kernel/src/features/column_mapping.rs b/kernel/src/table_features/column_mapping.rs similarity index 85% rename from kernel/src/features/column_mapping.rs rename to kernel/src/table_features/column_mapping.rs index 91739aef2..9365ccd4b 100644 --- a/kernel/src/features/column_mapping.rs +++ b/kernel/src/table_features/column_mapping.rs @@ -6,7 +6,7 @@ use serde::{Deserialize, Serialize}; use crate::{DeltaResult, Error}; /// Modes of column mapping a table can be in -#[derive(Serialize, Deserialize, Debug, Copy, Clone, PartialEq)] +#[derive(Serialize, Deserialize, Debug, Copy, Clone, PartialEq, Eq)] #[serde(rename_all = "camelCase")] pub enum ColumnMappingMode { /// No column mapping is applied @@ -17,9 +17,6 @@ pub enum ColumnMappingMode { Name, } -// key to look in metadata.configuration for to get column mapping mode -pub(crate) const COLUMN_MAPPING_MODE_KEY: &str = "delta.columnMapping.mode"; - impl TryFrom<&str> for ColumnMappingMode { type Error = Error; diff --git a/kernel/src/features/mod.rs b/kernel/src/table_features/mod.rs similarity index 98% rename from kernel/src/features/mod.rs rename to kernel/src/table_features/mod.rs index cef49d906..ee28fbfb5 100644 --- a/kernel/src/features/mod.rs +++ b/kernel/src/table_features/mod.rs @@ -1,7 +1,6 @@ use serde::{Deserialize, Serialize}; pub use column_mapping::ColumnMappingMode; -pub(crate) use column_mapping::COLUMN_MAPPING_MODE_KEY; use strum::{AsRefStr, Display as StrumDisplay, EnumString, VariantNames}; mod column_mapping; diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs new file mode 100644 index 000000000..49daf3b1b --- /dev/null +++ b/kernel/src/table_properties.rs @@ -0,0 +1,736 @@ +//! Delta Table properties. Note this module implements per-table configuration which governs how +//! table-level capabilities/properties are configured (turned on/off etc.). This is orthogonal to +//! protocol-level 'table features' which enable or disable reader/writer features (which then +//! usually must be enabled/configured by table properties). +//! +//! For example (from delta's protocol.md): A feature being supported does not imply that it is +//! active. For example, a table may have the `appendOnly` feature listed in writerFeatures, but it +//! does not have a table property delta.appendOnly that is set to `true`. In such a case the table +//! is not append-only, and writers are allowed to change, remove, and rearrange data. However, +//! writers must know that the table property delta.appendOnly should be checked before writing the +//! table. +// +// Delete me? +// Delta table properties are use to configure the table. They are defined in the 'Metadata' action +// under 'configuration' map. Every table must have at exactly one Protocol and one Metadata action. +// Only the most recent is used in the case of multiple P/M actions present in a log segment. +// +// ex: "configuration":{"delta.enableDeletionVectors":"true"} +// parsed into a map of DeltaTableProperty +// = DeltaTableProperty::EnableDeletionVectors -> true + +use std::time::Duration; + +use serde::Deserialize; +use serde::de::{self, Deserializer}; + +use crate::error::Error; +use crate::table_features::ColumnMappingMode; + +/// Default num index cols +pub const DEFAULT_NUM_INDEX_COLS: i32 = 32; + +// #[derive(Debug)] +// struct BoundedInt(i32); +// +// impl<'de> Deserialize<'de> for BoundedInt { +// fn deserialize(deserializer: D) -> Result +// where +// D: Deserializer<'de>, +// { +// let val = i32::deserialize(deserializer)?; +// if val >= 1 && val <= 10 { +// Ok(BoundedInt(val)) +// } else { +// Err(de::Error::custom(format!( +// "Value {} is out of bounds (1-10)", +// val +// ))) +// } +// } +// } + +/// Delta table properties +#[derive(Deserialize, Debug, Clone, Eq, PartialEq)] +pub struct TableProperties { + /// true for this Delta table to be append-only. If append-only, + /// existing records cannot be deleted, and existing values cannot be updated. + #[serde(rename = "delta.appendOnly")] + pub append_only: Option, + + /// true for Delta Lake to automatically optimize the layout of the files for this Delta table. + #[serde(rename = "delta.autoOptimize.autoCompact")] + pub auto_compact: Option, + + /// true for Delta Lake to automatically optimize the layout of the files for this Delta table + /// during writes. + #[serde(rename = "delta.autoOptimize.optimizeWrite")] + pub optimize_write: Option, + + /// Interval (expressed as number of commits) after which a new checkpoint should be created. + /// E.g. if checkpoint interval = 10, then a checkpoint should be written every 10 commits. + #[serde(rename = "delta.checkpointInterval")] + pub checkpoint_interval: Option, + + /// true for Delta Lake to write file statistics in checkpoints in JSON format for the stats column. + #[serde(rename = "delta.checkpoint.writeStatsAsJson")] + pub checkpoint_write_stats_as_json: Option, + + /// true for Delta Lake to write file statistics to checkpoints in struct format for the + /// stats_parsed column and to write partition values as a struct for partitionValues_parsed. + #[serde(rename = "delta.checkpoint.writeStatsAsStruct")] + pub checkpoint_write_stats_as_struct: Option, + + /// Whether column mapping is enabled for Delta table columns and the corresponding + /// Parquet columns that use different names. + #[serde(rename = "delta.columnMapping.mode")] + pub column_mapping_mode: ColumnMappingMode, + + /// The number of columns for Delta Lake to collect statistics about for data skipping. + /// A value of -1 means to collect statistics for all columns. Updating this property does + /// not automatically collect statistics again; instead, it redefines the statistics schema + /// of the Delta table. Specifically, it changes the behavior of future statistics collection + /// (such as during appends and optimizations) as well as data skipping (such as ignoring column + /// statistics beyond this number, even when such statistics exist). + #[serde(rename = "delta.dataSkippingNumIndexedCols")] + pub data_skipping_num_indexed_cols: Option, + + /// A comma-separated list of column names on which Delta Lake collects statistics to enhance + /// data skipping functionality. This property takes precedence over + /// [DataSkippingNumIndexedCols](DeltaConfigKey::DataSkippingNumIndexedCols). + #[serde(rename = "delta.dataSkippingStatsColumns")] + pub data_skipping_stats_columns: Option, + + /// The shortest duration for Delta Lake to keep logically deleted data files before deleting + /// them physically. This is to prevent failures in stale readers after compactions or partition + /// overwrites. + /// + /// This value should be large enough to ensure that: + /// + /// * It is larger than the longest possible duration of a job if you run VACUUM when there are + /// concurrent readers or writers accessing the Delta table. + /// * If you run a streaming query that reads from the table, that query does not stop for + /// longer than this value. Otherwise, the query may not be able to restart, as it must still + /// read old files. + #[serde(rename = "delta.deletedFileRetentionDuration")] + #[serde(deserialize_with = "deserialize_interval")] + pub deleted_file_retention_duration: Option, + + /// true to enable change data feed. + #[serde(rename = "delta.enableChangeDataFeed")] + pub enable_change_data_feed: Option, + + /// true to enable deletion vectors and predictive I/O for updates. + #[serde(rename = "delta.enableDeletionVectors")] + pub enable_deletion_vectors: Option, + + /// The degree to which a transaction must be isolated from modifications made by concurrent transactions. + /// + /// Valid values are `Serializable` and `WriteSerializable`. + #[serde(rename = "delta.isolationLevel")] + pub isolation_level: Option, + + /// How long the history for a Delta table is kept. + /// + /// Each time a checkpoint is written, Delta Lake automatically cleans up log entries older + /// than the retention interval. If you set this property to a large enough value, many log + /// entries are retained. This should not impact performance as operations against the log are + /// constant time. Operations on history are parallel but will become more expensive as the log + /// size increases. + #[serde(rename = "delta.logRetentionDuration")] + #[serde(deserialize_with = "deserialize_interval")] + pub log_retention_duration: Option, + + /// TODO I could not find this property in the documentation, but was defined here and makes sense..? + #[serde(rename = "delta.enableExpiredLogCleanup")] + pub enable_expired_log_cleanup: Option, + + /// The minimum required protocol reader version for a reader that allows to read from this Delta table. + #[serde(rename = "delta.minReaderVersion")] + pub min_reader_version: Option, + + /// The minimum required protocol writer version for a writer that allows to write to this Delta table. + #[serde(rename = "delta.minWriterVersion")] + pub min_writer_version: Option, + + /// true for Delta to generate a random prefix for a file path instead of partition information. + /// + /// For example, this may improve Amazon S3 performance when Delta Lake needs to send very high + /// volumes of Amazon S3 calls to better partition across S3 servers. + #[serde(rename = "delta.randomizeFilePrefixes")] + pub randomize_file_prefixes: Option, + + /// When delta.randomizeFilePrefixes is set to true, the number of characters that Delta + /// generates for random prefixes. + #[serde(rename = "delta.randomPrefixLength")] + pub random_prefix_length: Option, + + /// The shortest duration within which new snapshots will retain transaction identifiers (for + /// example, SetTransactions). When a new snapshot sees a transaction identifier older than or + /// equal to the duration specified by this property, the snapshot considers it expired and + /// ignores it. The SetTransaction identifier is used when making the writes idempotent. + #[serde(rename = "delta.setTransactionRetentionDuration")] + #[serde(deserialize_with = "deserialize_interval")] + pub set_transaction_retention_duration: Option, + + /// The target file size in bytes or higher units for file tuning. For example, 104857600 + /// (bytes) or 100mb. + #[serde(rename = "delta.targetFileSize")] + pub target_file_size: Option, + + /// The target file size in bytes or higher units for file tuning. For example, 104857600 + /// (bytes) or 100mb. + #[serde(rename = "delta.tuneFileSizesForRewrites")] + pub tune_file_sizes_for_rewrites: Option, + + /// 'classic' for classic Delta Lake checkpoints. 'v2' for v2 checkpoints. + #[serde(rename = "delta.checkpointPolicy")] + pub checkpoint_policy: Option, +} + +// /// Delta configuration error +// #[derive(thiserror::Error, Debug, PartialEq, Eq)] +// pub enum DeltaConfigError { +// /// Error returned when configuration validation failed. +// #[error("Validation failed - {0}")] +// Validation(String), +// } +// +// impl From for Error { +// fn from(e: DeltaConfigError) -> Self { +// Error::InvalidConfiguration(e.to_string()) +// } +// } + +// /// The shortest duration for Delta Lake to keep logically deleted data files before deleting +// /// them physically. This is to prevent failures in stale readers after compactions or partition +// /// overwrites. +// /// +// /// This value should be large enough to ensure that: +// /// +// /// * It is larger than the longest possible duration of a job if you run VACUUM when there are +// /// concurrent readers or writers accessing the Delta table. +// /// * If you run a streaming query that reads from the table, that query does not stop for +// /// longer than this value. Otherwise, the query may not be able to restart, as it must still +// /// read old files. +// pub fn deleted_file_retention_duration(&self) -> Duration { +// static DEFAULT_FILE_RETENTION_DURATION: LazyLock = +// LazyLock::new(|| parse_interval("interval 1 weeks").unwrap()); +// self.0 +// .get(DeltaTableProperty::DeletedFileRetentionDuration.as_ref()) +// .and_then(|v| parse_interval(v).ok()) +// .unwrap_or_else(|| DEFAULT_FILE_RETENTION_DURATION.to_owned()) +// } +// +// /// How long the history for a Delta table is kept. +// /// +// /// Each time a checkpoint is written, Delta Lake automatically cleans up log entries older +// /// than the retention interval. If you set this property to a large enough value, many log +// /// entries are retained. This should not impact performance as operations against the log are +// /// constant time. Operations on history are parallel but will become more expensive as the log +// /// size increases. +// pub fn log_retention_duration(&self) -> Duration { +// static DEFAULT_LOG_RETENTION_DURATION: LazyLock = +// LazyLock::new(|| parse_interval("interval 30 days").unwrap()); +// self.0 +// .get(DeltaTableProperty::LogRetentionDuration.as_ref()) +// .and_then(|v| parse_interval(v).ok()) +// .unwrap_or_else(|| DEFAULT_LOG_RETENTION_DURATION.to_owned()) +// } +// +// /// The degree to which a transaction must be isolated from modifications made by concurrent +// /// transactions. +// /// +// /// Valid values are `Serializable` and `WriteSerializable`. +// pub fn isolation_level(&self) -> IsolationLevel { +// self.0 +// .get(DeltaTableProperty::IsolationLevel.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(DeltaTableProperty::CheckpointPolicy.as_ref()) +// .and_then(|v| v.parse().ok()) +// .unwrap_or_default() +// } +// +// /// Return the column mapping mode according to delta.columnMapping.mode +// pub fn column_mapping_mode(&self) -> ColumnMappingMode { +// self.0 +// .get(DeltaTableProperty::ColumnMappingMode.as_ref()) +// .and_then(|v| v.parse().ok()) +// .unwrap_or_default() +// } +// +// /// Return the check constraints on the current table +// pub fn get_constraints(&self) -> Vec { +// self.0 +// .iter() +// .filter_map(|(field, value)| { +// if field.starts_with("delta.constraints") { +// field +// .splitn(3, '.') +// .last() +// .map(|n| Constraint::new(n, value)) +// } else { +// None +// } +// }) +// .collect() +// } +// +// /// Column names on which Delta Lake collects statistics to enhance data skipping functionality. +// /// This property takes precedence over [num_indexed_cols](Self::num_indexed_cols). +// pub fn stats_columns(&self) -> Option> { +// self.0 +// .get(DeltaTableProperty::DataSkippingStatsColumns.as_ref()) +// .map(|v| v.split(',').collect()) +// } + +/// The isolation level applied during transaction +#[derive(Deserialize, Debug, Copy, Clone, PartialEq, Eq)] +#[serde(rename_all = "camelCase")] +pub enum IsolationLevel { + /// The strongest isolation level. It ensures that committed write operations + /// and all reads are Serializable. Operations are allowed as long as there + /// exists a serial sequence of executing them one-at-a-time that generates + /// the same outcome as that seen in the table. For the write operations, + /// the serial sequence is exactly the same as that seen in the table’s history. + Serializable, + + /// A weaker isolation level than Serializable. It ensures only that the write + /// operations (that is, not reads) are serializable. However, this is still stronger + /// than Snapshot isolation. WriteSerializable is the default isolation level because + /// it provides great balance of data consistency and availability for most common operations. + WriteSerializable, + + /// SnapshotIsolation is a guarantee that all reads made in a transaction will see a consistent + /// snapshot of the database (in practice it reads the last committed values that existed at the + /// time it started), and the transaction itself will successfully commit only if no updates + /// it has made conflict with any concurrent updates made since that snapshot. + SnapshotIsolation, +} + +// Spark assumes Serializable as default isolation level +// https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1023 +impl Default for IsolationLevel { + fn default() -> Self { + Self::Serializable + } +} + +#[derive(Deserialize, Debug, Clone, PartialEq, Eq)] +/// The checkpoint policy applied when writing checkpoints +#[serde(rename_all = "camelCase")] +pub enum CheckpointPolicy { + /// classic Delta Lake checkpoints + Classic, + /// v2 checkpoints + V2, +} + +impl Default for CheckpointPolicy { + fn default() -> Self { + Self::Classic + } +} + +const SECONDS_PER_MINUTE: u64 = 60; +const SECONDS_PER_HOUR: u64 = 60 * SECONDS_PER_MINUTE; +const SECONDS_PER_DAY: u64 = 24 * SECONDS_PER_HOUR; +const SECONDS_PER_WEEK: u64 = 7 * SECONDS_PER_DAY; + +fn deserialize_interval<'de, D>(deserializer: D) -> Result, D::Error> +where + D: Deserializer<'de>, +{ + let opt = Option::::deserialize(deserializer)?; + match opt { + Some(s) => parse_interval(&s).map(Some).map_err(de::Error::custom), + None => Ok(None), + } +} + +fn parse_interval(value: &str) -> Result { + let not_an_interval = || format!("'{value}' is not an interval"); + + if !value.starts_with("interval ") { + return Err(not_an_interval()); + } + let mut it = value.split_whitespace(); + let _ = it.next(); // skip "interval" + let number = parse_int(it.next().ok_or_else(not_an_interval)?)?; + if number < 0 { + return Err(format!("interval '{value}' cannot be negative")); + } + let number = number as u64; + + let duration = match it.next().ok_or_else(not_an_interval)? { + "nanosecond" | "nanoseconds" => Duration::from_nanos(number), + "microsecond" | "microseconds" => Duration::from_micros(number), + "millisecond" | "milliseconds" => Duration::from_millis(number), + "second" | "seconds" => Duration::from_secs(number), + "minute" | "minutes" => Duration::from_secs(number * SECONDS_PER_MINUTE), + "hour" | "hours" => Duration::from_secs(number * SECONDS_PER_HOUR), + "day" | "days" => Duration::from_secs(number * SECONDS_PER_DAY), + "week" | "weeks" => Duration::from_secs(number * SECONDS_PER_WEEK), + unit => { + return Err(format!("Unknown unit '{unit}'")); + } + }; + + Ok(duration) +} + +fn parse_int(value: &str) -> Result { + value + .parse() + .map_err(|e| format!("Cannot parse '{value}' as integer: {e}")) +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::actions::{Format, Metadata}; + use crate::schema::StructType; + use std::collections::HashMap; + + fn dummy_metadata() -> Metadata { + let schema = StructType::new(Vec::new()); + Metadata { + id: "id".into(), + name: None, + description: None, + format: Format::default(), + schema_string: serde_json::to_string(&schema).unwrap(), + partition_columns: Vec::new(), + configuration: HashMap::new(), + created_time: None, + } + } + + #[test] + fn zach() { + + } + + // #[test] + // fn get_interval_from_metadata_test() { + // let md = dummy_metadata(); + // let config = TableConfig(&md.configuration); + + // // default 1 week + // assert_eq!( + // config.deleted_file_retention_duration().as_secs(), + // SECONDS_PER_WEEK, + // ); + + // // change to 2 day + // let mut md = dummy_metadata(); + // md.configuration.insert( + // DeltaTableProperty::DeletedFileRetentionDuration + // .as_ref() + // .to_string(), + // "interval 2 day".to_string(), + // ); + // let config = TableConfig(&md.configuration); + + // assert_eq!( + // config.deleted_file_retention_duration().as_secs(), + // 2 * SECONDS_PER_DAY, + // ); + // } + + // #[test] + // fn get_long_from_metadata_test() { + // let md = dummy_metadata(); + // let config = TableConfig(&md.configuration); + // assert_eq!(config.checkpoint_interval(), 10,) + // } + + // #[test] + // fn get_boolean_from_metadata_test() { + // let md = dummy_metadata(); + // let config = TableConfig(&md.configuration); + + // // default value is true + // assert!(config.enable_expired_log_cleanup()); + + // // change to false + // let mut md = dummy_metadata(); + // md.configuration.insert( + // DeltaTableProperty::EnableExpiredLogCleanup.as_ref().into(), + // "false".to_string(), + // ); + // let config = TableConfig(&md.configuration); + + // assert!(!config.enable_expired_log_cleanup()); + // } + + // #[test] + // fn parse_interval_test() { + // assert_eq!( + // parse_interval("interval 123 nanosecond").unwrap(), + // Duration::from_nanos(123) + // ); + + // assert_eq!( + // parse_interval("interval 123 nanoseconds").unwrap(), + // Duration::from_nanos(123) + // ); + + // assert_eq!( + // parse_interval("interval 123 microsecond").unwrap(), + // Duration::from_micros(123) + // ); + + // assert_eq!( + // parse_interval("interval 123 microseconds").unwrap(), + // Duration::from_micros(123) + // ); + + // assert_eq!( + // parse_interval("interval 123 millisecond").unwrap(), + // Duration::from_millis(123) + // ); + + // assert_eq!( + // parse_interval("interval 123 milliseconds").unwrap(), + // Duration::from_millis(123) + // ); + + // assert_eq!( + // parse_interval("interval 123 second").unwrap(), + // Duration::from_secs(123) + // ); + + // assert_eq!( + // parse_interval("interval 123 seconds").unwrap(), + // Duration::from_secs(123) + // ); + + // assert_eq!( + // parse_interval("interval 123 minute").unwrap(), + // Duration::from_secs(123 * 60) + // ); + + // assert_eq!( + // parse_interval("interval 123 minutes").unwrap(), + // Duration::from_secs(123 * 60) + // ); + + // assert_eq!( + // parse_interval("interval 123 hour").unwrap(), + // Duration::from_secs(123 * 3600) + // ); + + // assert_eq!( + // parse_interval("interval 123 hours").unwrap(), + // Duration::from_secs(123 * 3600) + // ); + + // assert_eq!( + // parse_interval("interval 123 day").unwrap(), + // Duration::from_secs(123 * 86400) + // ); + + // assert_eq!( + // parse_interval("interval 123 days").unwrap(), + // Duration::from_secs(123 * 86400) + // ); + + // assert_eq!( + // parse_interval("interval 123 week").unwrap(), + // Duration::from_secs(123 * 604800) + // ); + + // assert_eq!( + // parse_interval("interval 123 week").unwrap(), + // Duration::from_secs(123 * 604800) + // ); + // } + + // #[test] + // fn parse_interval_invalid_test() { + // assert_eq!( + // parse_interval("whatever").err().unwrap(), + // DeltaConfigError::Validation("'whatever' is not an interval".to_string()) + // ); + + // assert_eq!( + // parse_interval("interval").err().unwrap(), + // DeltaConfigError::Validation("'interval' is not an interval".to_string()) + // ); + + // assert_eq!( + // parse_interval("interval 2").err().unwrap(), + // DeltaConfigError::Validation("'interval 2' is not an interval".to_string()) + // ); + + // assert_eq!( + // parse_interval("interval 2 years").err().unwrap(), + // DeltaConfigError::Validation("Unknown unit 'years'".to_string()) + // ); + + // assert_eq!( + // parse_interval("interval two years").err().unwrap(), + // DeltaConfigError::Validation( + // "Cannot parse 'two' as integer: invalid digit found in string".to_string() + // ) + // ); + + // assert_eq!( + // parse_interval("interval -25 hours").err().unwrap(), + // DeltaConfigError::Validation( + // "interval 'interval -25 hours' cannot be negative".to_string() + // ) + // ); + // } + + // #[test] + // fn test_constraint() { + // let md = dummy_metadata(); + // let config = TableConfig(&md.configuration); + + // assert_eq!(config.get_constraints().len(), 0); + + // let mut md = dummy_metadata(); + // md.configuration.insert( + // "delta.constraints.name".to_string(), + // "name = 'foo'".to_string(), + // ); + // md.configuration + // .insert("delta.constraints.age".to_string(), "age > 10".to_string()); + // let config = TableConfig(&md.configuration); + + // let constraints = config.get_constraints(); + // assert_eq!(constraints.len(), 2); + // assert!(constraints.contains(&Constraint::new("name", "name = 'foo'"))); + // assert!(constraints.contains(&Constraint::new("age", "age > 10"))); + // } + + // #[test] + // fn test_roundtrip_config_key() { + // let cases = [ + // (DeltaTableProperty::AppendOnly, "delta.appendOnly"), + // ( + // DeltaTableProperty::AutoOptimizeAutoCompact, + // "delta.autoOptimize.autoCompact", + // ), + // ( + // DeltaTableProperty::AutoOptimizeOptimizeWrite, + // "delta.autoOptimize.optimizeWrite", + // ), + // ( + // DeltaTableProperty::CheckpointInterval, + // "delta.checkpointInterval", + // ), + // ( + // DeltaTableProperty::CheckpointWriteStatsAsJson, + // "delta.checkpoint.writeStatsAsJson", + // ), + // ( + // DeltaTableProperty::CheckpointWriteStatsAsStruct, + // "delta.checkpoint.writeStatsAsStruct", + // ), + // ( + // DeltaTableProperty::ColumnMappingMode, + // "delta.columnMapping.mode", + // ), + // ( + // DeltaTableProperty::DataSkippingNumIndexedCols, + // "delta.dataSkippingNumIndexedCols", + // ), + // ( + // DeltaTableProperty::DataSkippingStatsColumns, + // "delta.dataSkippingStatsColumns", + // ), + // ( + // DeltaTableProperty::DeletedFileRetentionDuration, + // "delta.deletedFileRetentionDuration", + // ), + // ( + // DeltaTableProperty::EnableChangeDataFeed, + // "delta.enableChangeDataFeed", + // ), + // ( + // DeltaTableProperty::EnableDeletionVectors, + // "delta.enableDeletionVectors", + // ), + // (DeltaTableProperty::IsolationLevel, "delta.isolationLevel"), + // ( + // DeltaTableProperty::LogRetentionDuration, + // "delta.logRetentionDuration", + // ), + // ( + // DeltaTableProperty::MinReaderVersion, + // "delta.minReaderVersion", + // ), + // ( + // DeltaTableProperty::MinWriterVersion, + // "delta.minWriterVersion", + // ), + // ( + // DeltaTableProperty::RandomizeFilePrefixes, + // "delta.randomizeFilePrefixes", + // ), + // ( + // DeltaTableProperty::RandomPrefixLength, + // "delta.randomPrefixLength", + // ), + // ( + // DeltaTableProperty::SetTransactionRetentionDuration, + // "delta.setTransactionRetentionDuration", + // ), + // ( + // DeltaTableProperty::EnableExpiredLogCleanup, + // "delta.enableExpiredLogCleanup", + // ), + // (DeltaTableProperty::TargetFileSize, "delta.targetFileSize"), + // ( + // DeltaTableProperty::TuneFileSizesForRewrites, + // "delta.tuneFileSizesForRewrites", + // ), + // ( + // DeltaTableProperty::CheckpointPolicy, + // "delta.checkpointPolicy", + // ), + // ]; + + // assert_eq!(DeltaTableProperty::VARIANTS.len(), cases.len()); + + // for (key, expected) in cases { + // assert_eq!(key.as_ref(), expected); + + // let serialized = serde_json::to_string(&key).unwrap(); + // assert_eq!(serialized, format!("\"{}\"", expected)); + + // let deserialized: DeltaTableProperty = serde_json::from_str(&serialized).unwrap(); + // assert_eq!(deserialized, key); + + // let from_str: DeltaTableProperty = expected.parse().unwrap(); + // assert_eq!(from_str, key); + // } + // } + + // #[test] + // fn test_default_config() { + // let md = dummy_metadata(); + // let config = TableConfig(&md.configuration); + + // assert_eq!(config.append_only(), false); + // // assert_eq!(config.auto_optimize_auto_compact(), false); + // assert_eq!(config.auto_optimize_optimize_write(), false); + // assert_eq!(config.checkpoint_interval(), 10); + // assert_eq!(config.write_stats_as_json(), true); + // assert_eq!(config.write_stats_as_struct(), false); + // assert_eq!(config.target_file_size(), 104857600); + // assert_eq!(config.enable_change_data_feed(), false); + // assert_eq!(config.enable_deletion_vectors(), false); + // assert_eq!(config.num_indexed_cols(), 32); + // assert_eq!(config.enable_expired_log_cleanup(), true); + // } +} From 9d4b599f880a5635ee387b90d5b493f542d69bfd Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 4 Nov 2024 16:31:31 -0800 Subject: [PATCH 02/20] rough draft serde for table props --- kernel/src/actions/mod.rs | 7 +- kernel/src/expressions/column_names.rs | 5 +- kernel/src/lib.rs | 2 +- kernel/src/scan/mod.rs | 2 +- kernel/src/scan/state.rs | 2 +- kernel/src/snapshot.rs | 1 - kernel/src/table_properties.rs | 253 ++++++++++++--------- kernel/src/table_properties/deserialize.rs | 157 +++++++++++++ 8 files changed, 308 insertions(+), 121 deletions(-) create mode 100644 kernel/src/table_properties/deserialize.rs diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 54a6d9531..985bda04b 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -9,9 +9,9 @@ use visitors::{AddVisitor, MetadataVisitor, ProtocolVisitor}; use self::deletion_vector::DeletionVectorDescriptor; use crate::actions::schemas::GetStructField; -use crate::table_properties::TableProperties; -use crate::table_features::{ReaderFeatures, WriterFeatures}; use crate::schema::{SchemaRef, StructType}; +use crate::table_features::{ReaderFeatures, WriterFeatures}; +use crate::table_properties::TableProperties; use crate::{DeltaResult, EngineData}; pub mod deletion_vector; @@ -115,8 +115,9 @@ impl Metadata { Ok(serde_json::from_str(&self.schema_string)?) } + /// Parse the metadata configuration HashMap into a TableProperties struct. pub fn get_table_properties(&self) -> DeltaResult { - todo!() + TableProperties::new(&self.configuration) } } diff --git a/kernel/src/expressions/column_names.rs b/kernel/src/expressions/column_names.rs index ca0e2ca70..8e5e2101b 100644 --- a/kernel/src/expressions/column_names.rs +++ b/kernel/src/expressions/column_names.rs @@ -2,9 +2,12 @@ use std::borrow::Borrow; use std::fmt::{Display, Formatter}; use std::hash::{Hash, Hasher}; use std::ops::Deref; + +use serde::Deserialize; + /// A (possibly nested) column name. // TODO: Track name as a path rather than a single string -#[derive(Debug, Clone, PartialEq, PartialOrd, Eq, Ord)] +#[derive(Debug, Clone, PartialEq, PartialOrd, Eq, Ord, Deserialize)] pub struct ColumnName { path: String, } diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index 593a110a2..5feffcbd9 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -73,8 +73,8 @@ pub mod scan; pub mod schema; pub mod snapshot; pub mod table; -pub mod transaction; pub mod table_properties; +pub mod transaction; pub(crate) mod utils; pub use delta_kernel_derive; diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index 8863aab39..5a2353789 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -10,10 +10,10 @@ use url::Url; use crate::actions::deletion_vector::{split_vector, treemap_to_bools, DeletionVectorDescriptor}; use crate::actions::{get_log_add_schema, get_log_schema, ADD_NAME, REMOVE_NAME}; use crate::expressions::{ColumnName, Expression, ExpressionRef, Scalar}; -use crate::table_features::ColumnMappingMode; use crate::scan::state::{DvInfo, Stats}; use crate::schema::{DataType, Schema, SchemaRef, StructField, StructType}; use crate::snapshot::Snapshot; +use crate::table_features::ColumnMappingMode; use crate::{DeltaResult, Engine, EngineData, Error, FileMeta}; use self::log_replay::scan_action_iter; diff --git a/kernel/src/scan/state.rs b/kernel/src/scan/state.rs index e8cf1db16..3ac3ee0d2 100644 --- a/kernel/src/scan/state.rs +++ b/kernel/src/scan/state.rs @@ -8,8 +8,8 @@ use crate::{ visitors::visit_deletion_vector_at, }, engine_data::{GetData, TypedGetData}, - table_features::ColumnMappingMode, schema::SchemaRef, + table_features::ColumnMappingMode, DataVisitor, DeltaResult, Engine, EngineData, Error, }; use serde::{Deserialize, Serialize}; diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index b3a42c2c0..129a3ed93 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -15,7 +15,6 @@ use crate::expressions::column_expr; use crate::path::ParsedLogPath; use crate::scan::ScanBuilder; use crate::schema::{Schema, SchemaRef}; -use crate::table_features::ColumnMappingMode; use crate::table_properties::TableProperties; use crate::utils::require; use crate::{DeltaResult, Engine, Error, FileMeta, FileSystemClient, Version}; diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index 49daf3b1b..63b38c79b 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -9,81 +9,69 @@ //! is not append-only, and writers are allowed to change, remove, and rearrange data. However, //! writers must know that the table property delta.appendOnly should be checked before writing the //! table. -// -// Delete me? -// Delta table properties are use to configure the table. They are defined in the 'Metadata' action -// under 'configuration' map. Every table must have at exactly one Protocol and one Metadata action. -// Only the most recent is used in the case of multiple P/M actions present in a log segment. -// -// ex: "configuration":{"delta.enableDeletionVectors":"true"} -// parsed into a map of DeltaTableProperty -// = DeltaTableProperty::EnableDeletionVectors -> true +use std::collections::HashMap; use std::time::Duration; use serde::Deserialize; -use serde::de::{self, Deserializer}; -use crate::error::Error; +use crate::expressions::ColumnName; use crate::table_features::ColumnMappingMode; +use crate::DeltaResult; + +mod deserialize; +use deserialize::*; /// Default num index cols pub const DEFAULT_NUM_INDEX_COLS: i32 = 32; -// #[derive(Debug)] -// struct BoundedInt(i32); -// -// impl<'de> Deserialize<'de> for BoundedInt { -// fn deserialize(deserializer: D) -> Result -// where -// D: Deserializer<'de>, -// { -// let val = i32::deserialize(deserializer)?; -// if val >= 1 && val <= 10 { -// Ok(BoundedInt(val)) -// } else { -// Err(de::Error::custom(format!( -// "Value {} is out of bounds (1-10)", -// val -// ))) -// } -// } -// } - -/// Delta table properties +/// Delta table properties. These are parsed from the 'configuration' map in the most recent +/// 'Metadata' action of a table. #[derive(Deserialize, Debug, Clone, Eq, PartialEq)] pub struct TableProperties { /// true for this Delta table to be append-only. If append-only, /// existing records cannot be deleted, and existing values cannot be updated. #[serde(rename = "delta.appendOnly")] - pub append_only: Option, + #[serde(deserialize_with = "deserialize_bool")] + #[serde(default)] + pub append_only: bool, /// true for Delta Lake to automatically optimize the layout of the files for this Delta table. #[serde(rename = "delta.autoOptimize.autoCompact")] - pub auto_compact: Option, + #[serde(deserialize_with = "deserialize_bool")] + #[serde(default)] + pub auto_compact: bool, /// true for Delta Lake to automatically optimize the layout of the files for this Delta table /// during writes. #[serde(rename = "delta.autoOptimize.optimizeWrite")] - pub optimize_write: Option, + #[serde(deserialize_with = "deserialize_bool")] + #[serde(default)] + pub optimize_write: bool, /// Interval (expressed as number of commits) after which a new checkpoint should be created. /// E.g. if checkpoint interval = 10, then a checkpoint should be written every 10 commits. #[serde(rename = "delta.checkpointInterval")] - pub checkpoint_interval: Option, + #[serde(deserialize_with = "deserialize_pos_int")] + #[serde(default = "default_checkpoint_interval")] + pub checkpoint_interval: u64, /// true for Delta Lake to write file statistics in checkpoints in JSON format for the stats column. #[serde(rename = "delta.checkpoint.writeStatsAsJson")] - pub checkpoint_write_stats_as_json: Option, + #[serde(deserialize_with = "deserialize_bool")] + #[serde(default)] + pub checkpoint_write_stats_as_json: bool, /// true for Delta Lake to write file statistics to checkpoints in struct format for the /// stats_parsed column and to write partition values as a struct for partitionValues_parsed. #[serde(rename = "delta.checkpoint.writeStatsAsStruct")] - pub checkpoint_write_stats_as_struct: Option, + #[serde(deserialize_with = "deserialize_bool")] + #[serde(default)] + pub checkpoint_write_stats_as_struct: bool, /// Whether column mapping is enabled for Delta table columns and the corresponding /// Parquet columns that use different names. - #[serde(rename = "delta.columnMapping.mode")] + #[serde(rename = "delta.columnMapping.mode", default)] pub column_mapping_mode: ColumnMappingMode, /// The number of columns for Delta Lake to collect statistics about for data skipping. @@ -92,14 +80,16 @@ pub struct TableProperties { /// of the Delta table. Specifically, it changes the behavior of future statistics collection /// (such as during appends and optimizations) as well as data skipping (such as ignoring column /// statistics beyond this number, even when such statistics exist). - #[serde(rename = "delta.dataSkippingNumIndexedCols")] - pub data_skipping_num_indexed_cols: Option, + #[serde(rename = "delta.dataSkippingNumIndexedCols", default, flatten)] + pub data_skipping_num_indexed_cols: Option, /// A comma-separated list of column names on which Delta Lake collects statistics to enhance /// data skipping functionality. This property takes precedence over /// [DataSkippingNumIndexedCols](DeltaConfigKey::DataSkippingNumIndexedCols). #[serde(rename = "delta.dataSkippingStatsColumns")] - pub data_skipping_stats_columns: Option, + #[serde(deserialize_with = "deserialize_column_names")] + #[serde(default)] + pub data_skipping_stats_columns: Vec, /// The shortest duration for Delta Lake to keep logically deleted data files before deleting /// them physically. This is to prevent failures in stale readers after compactions or partition @@ -114,21 +104,28 @@ pub struct TableProperties { /// read old files. #[serde(rename = "delta.deletedFileRetentionDuration")] #[serde(deserialize_with = "deserialize_interval")] + #[serde(default)] pub deleted_file_retention_duration: Option, /// true to enable change data feed. #[serde(rename = "delta.enableChangeDataFeed")] - pub enable_change_data_feed: Option, + #[serde(deserialize_with = "deserialize_bool")] + #[serde(default)] + pub enable_change_data_feed: bool, /// true to enable deletion vectors and predictive I/O for updates. #[serde(rename = "delta.enableDeletionVectors")] - pub enable_deletion_vectors: Option, + #[serde(deserialize_with = "deserialize_bool")] + #[serde(default)] + pub enable_deletion_vectors: bool, - /// The degree to which a transaction must be isolated from modifications made by concurrent transactions. + /// The degree to which a transaction must be isolated from modifications made by concurrent + /// transactions. /// /// Valid values are `Serializable` and `WriteSerializable`. #[serde(rename = "delta.isolationLevel")] - pub isolation_level: Option, + #[serde(default)] + pub isolation_level: IsolationLevel, /// How long the history for a Delta table is kept. /// @@ -139,31 +136,36 @@ pub struct TableProperties { /// size increases. #[serde(rename = "delta.logRetentionDuration")] #[serde(deserialize_with = "deserialize_interval")] + #[serde(default)] pub log_retention_duration: Option, - /// TODO I could not find this property in the documentation, but was defined here and makes sense..? + /// TODO docs #[serde(rename = "delta.enableExpiredLogCleanup")] - pub enable_expired_log_cleanup: Option, - - /// The minimum required protocol reader version for a reader that allows to read from this Delta table. - #[serde(rename = "delta.minReaderVersion")] - pub min_reader_version: Option, + #[serde(default)] + pub enable_expired_log_cleanup: bool, - /// The minimum required protocol writer version for a writer that allows to write to this Delta table. - #[serde(rename = "delta.minWriterVersion")] - pub min_writer_version: Option, + // /// The minimum required protocol reader version for a reader that allows to read from this Delta table. + // #[serde(rename = "delta.minReaderVersion")] + // pub min_reader_version: Option, + // /// The minimum required protocol writer version for a writer that allows to write to this Delta table. + // #[serde(rename = "delta.minWriterVersion")] + // pub min_writer_version: Option, + // /// true for Delta to generate a random prefix for a file path instead of partition information. /// /// For example, this may improve Amazon S3 performance when Delta Lake needs to send very high /// volumes of Amazon S3 calls to better partition across S3 servers. #[serde(rename = "delta.randomizeFilePrefixes")] - pub randomize_file_prefixes: Option, + #[serde(deserialize_with = "deserialize_bool")] + #[serde(default)] + pub randomize_file_prefixes: bool, /// When delta.randomizeFilePrefixes is set to true, the number of characters that Delta /// generates for random prefixes. #[serde(rename = "delta.randomPrefixLength")] - pub random_prefix_length: Option, + #[serde(default)] + pub random_prefix_length: Option, /// The shortest duration within which new snapshots will retain transaction identifiers (for /// example, SetTransactions). When a new snapshot sees a transaction identifier older than or @@ -171,21 +173,66 @@ pub struct TableProperties { /// ignores it. The SetTransaction identifier is used when making the writes idempotent. #[serde(rename = "delta.setTransactionRetentionDuration")] #[serde(deserialize_with = "deserialize_interval")] + #[serde(default)] pub set_transaction_retention_duration: Option, /// The target file size in bytes or higher units for file tuning. For example, 104857600 /// (bytes) or 100mb. + /// + /// TODO: kernel doesn't govern file writes? should we pass this through just with a note that + /// we don't pay attention to it? Scenario: engine calls + /// snapshot.table_properties().target_file_size in order to know how big _it_ should write + /// parquet files. #[serde(rename = "delta.targetFileSize")] + #[serde(default)] pub target_file_size: Option, /// The target file size in bytes or higher units for file tuning. For example, 104857600 /// (bytes) or 100mb. + /// + /// See TODO above #[serde(rename = "delta.tuneFileSizesForRewrites")] + #[serde(default)] pub tune_file_sizes_for_rewrites: Option, /// 'classic' for classic Delta Lake checkpoints. 'v2' for v2 checkpoints. #[serde(rename = "delta.checkpointPolicy")] - pub checkpoint_policy: Option, + #[serde(default)] + pub checkpoint_policy: CheckpointPolicy, +} + +impl TableProperties { + pub(crate) fn new(config_map: &HashMap) -> DeltaResult { + let deserializer = StringMapDeserializer::new(config_map); + // FIXME error + TableProperties::deserialize(deserializer).map_err(|e| crate::Error::Generic(e.to_string())) + } +} + +fn default_checkpoint_interval() -> u64 { + 10 +} + +#[derive(Debug, PartialEq, Eq, Clone, Deserialize)] +#[serde(try_from = "String")] +pub enum DataSkippingNumIndexedCols { + AllColumns, + NumColumns(u64), +} + +impl TryFrom for DataSkippingNumIndexedCols { + type Error = String; + + fn try_from(value: String) -> Result { + let num: i64 = value + .parse() + .map_err(|_| format!("couldn't parse {value} as i64"))?; // FIXME + match num { + -1 => Ok(DataSkippingNumIndexedCols::AllColumns), + x if x > -1 => Ok(DataSkippingNumIndexedCols::NumColumns(x as u64)), + _ => Err(format!("Invalid value: {}", value)), + } + } } // /// Delta configuration error @@ -338,59 +385,6 @@ impl Default for CheckpointPolicy { } } -const SECONDS_PER_MINUTE: u64 = 60; -const SECONDS_PER_HOUR: u64 = 60 * SECONDS_PER_MINUTE; -const SECONDS_PER_DAY: u64 = 24 * SECONDS_PER_HOUR; -const SECONDS_PER_WEEK: u64 = 7 * SECONDS_PER_DAY; - -fn deserialize_interval<'de, D>(deserializer: D) -> Result, D::Error> -where - D: Deserializer<'de>, -{ - let opt = Option::::deserialize(deserializer)?; - match opt { - Some(s) => parse_interval(&s).map(Some).map_err(de::Error::custom), - None => Ok(None), - } -} - -fn parse_interval(value: &str) -> Result { - let not_an_interval = || format!("'{value}' is not an interval"); - - if !value.starts_with("interval ") { - return Err(not_an_interval()); - } - let mut it = value.split_whitespace(); - let _ = it.next(); // skip "interval" - let number = parse_int(it.next().ok_or_else(not_an_interval)?)?; - if number < 0 { - return Err(format!("interval '{value}' cannot be negative")); - } - let number = number as u64; - - let duration = match it.next().ok_or_else(not_an_interval)? { - "nanosecond" | "nanoseconds" => Duration::from_nanos(number), - "microsecond" | "microseconds" => Duration::from_micros(number), - "millisecond" | "milliseconds" => Duration::from_millis(number), - "second" | "seconds" => Duration::from_secs(number), - "minute" | "minutes" => Duration::from_secs(number * SECONDS_PER_MINUTE), - "hour" | "hours" => Duration::from_secs(number * SECONDS_PER_HOUR), - "day" | "days" => Duration::from_secs(number * SECONDS_PER_DAY), - "week" | "weeks" => Duration::from_secs(number * SECONDS_PER_WEEK), - unit => { - return Err(format!("Unknown unit '{unit}'")); - } - }; - - Ok(duration) -} - -fn parse_int(value: &str) -> Result { - value - .parse() - .map_err(|e| format!("Cannot parse '{value}' as integer: {e}")) -} - #[cfg(test)] mod tests { use super::*; @@ -413,8 +407,10 @@ mod tests { } #[test] - fn zach() { - + fn fail_known_keys() { + let properties = HashMap::from([("delta.appendOnly".to_string(), "wack".to_string())]); + let de = StringMapDeserializer::new(&properties); + assert!(TableProperties::deserialize(de).is_err()); } // #[test] @@ -733,4 +729,35 @@ mod tests { // assert_eq!(config.num_indexed_cols(), 32); // assert_eq!(config.enable_expired_log_cleanup(), true); // } + + #[test] + fn it_works() { + let properties = HashMap::from([("delta.appendOnly".to_string(), "true".to_string())]); + let de = StringMapDeserializer::new(&properties); + let actual = TableProperties::deserialize(de).unwrap(); + let expected = TableProperties { + append_only: true, + optimize_write: false, + auto_compact: false, + checkpoint_interval: 10, + checkpoint_write_stats_as_json: false, + checkpoint_write_stats_as_struct: false, + column_mapping_mode: ColumnMappingMode::None, + data_skipping_num_indexed_cols: None, + data_skipping_stats_columns: vec![], + deleted_file_retention_duration: None, + enable_change_data_feed: false, + enable_deletion_vectors: false, + isolation_level: IsolationLevel::Serializable, + log_retention_duration: None, + enable_expired_log_cleanup: false, + randomize_file_prefixes: false, + random_prefix_length: None, + set_transaction_retention_duration: None, + target_file_size: None, + tune_file_sizes_for_rewrites: None, + checkpoint_policy: CheckpointPolicy::Classic, + }; + assert_eq!(actual, expected); + } } diff --git a/kernel/src/table_properties/deserialize.rs b/kernel/src/table_properties/deserialize.rs new file mode 100644 index 000000000..247593e98 --- /dev/null +++ b/kernel/src/table_properties/deserialize.rs @@ -0,0 +1,157 @@ +use crate::expressions::ColumnName; + +use std::collections::HashMap; +use std::time::Duration; + +use serde::de::{self, DeserializeSeed, Deserializer, MapAccess, Visitor}; +use serde::Deserialize; + +const SECONDS_PER_MINUTE: u64 = 60; +const SECONDS_PER_HOUR: u64 = 60 * SECONDS_PER_MINUTE; +const SECONDS_PER_DAY: u64 = 24 * SECONDS_PER_HOUR; +const SECONDS_PER_WEEK: u64 = 7 * SECONDS_PER_DAY; + +pub(crate) struct StringMapDeserializer<'de> { + iter: std::collections::hash_map::Iter<'de, String, String>, +} + +impl<'de> StringMapDeserializer<'de> { + pub(crate) fn new(map: &'de HashMap) -> Self { + StringMapDeserializer { iter: map.iter() } + } +} + +impl<'de> Deserializer<'de> for StringMapDeserializer<'de> { + type Error = de::value::Error; + + fn deserialize_any(self, visitor: V) -> Result + where + V: Visitor<'de>, + { + visitor.visit_map(HashMapMapAccess { + iter: self.iter, + value: None, + }) + } + + serde::forward_to_deserialize_any! { + bool i8 i16 i32 i64 u8 u16 u32 u64 f32 f64 char str string bytes + byte_buf option unit unit_struct newtype_struct seq tuple tuple_struct + map struct enum identifier ignored_any + } +} + +struct HashMapMapAccess<'de> { + iter: std::collections::hash_map::Iter<'de, String, String>, + value: Option<&'de String>, +} + +impl<'de> MapAccess<'de> for HashMapMapAccess<'de> { + type Error = de::value::Error; + + fn next_key_seed(&mut self, seed: K) -> Result, Self::Error> + where + K: DeserializeSeed<'de>, + { + match self.iter.next() { + Some((key, value)) => { + self.value = Some(value); + let de = de::value::StrDeserializer::new(key); + seed.deserialize(de).map(Some) + } + None => Ok(None), + } + } + + fn next_value_seed(&mut self, seed: V) -> Result + where + V: DeserializeSeed<'de>, + { + let value = self.value.take().unwrap(); // FIXME + let de = de::value::StrDeserializer::new(value); + seed.deserialize(de) + } +} + +pub(crate) fn deserialize_pos_int<'de, D>(deserializer: D) -> Result +where + D: de::Deserializer<'de>, +{ + let s: String = de::Deserialize::deserialize(deserializer)?; + let n: u64 = s.parse().unwrap(); // FIXME + if n == 0 { + panic!("FIXME"); + // return Err("something"); + } + Ok(n) +} + +pub(crate) fn deserialize_bool<'de, D>(deserializer: D) -> Result +where + D: de::Deserializer<'de>, +{ + let s: String = de::Deserialize::deserialize(deserializer)?; + match s.as_str() { + "true" => Ok(true), + "false" => Ok(false), + _ => Err(de::Error::unknown_variant(&s, &["true", "false"])), + } +} + +pub(crate) fn deserialize_column_names<'de, D>(deserializer: D) -> Result, D::Error> +where + D: de::Deserializer<'de>, +{ + let s: String = de::Deserialize::deserialize(deserializer)?; + Ok(s.split(',') + .map(|name: &str| ColumnName::new([name])) + .collect()) +} + +pub(crate) fn deserialize_interval<'de, D>(deserializer: D) -> Result, D::Error> +where + D: Deserializer<'de>, +{ + let opt = Option::::deserialize(deserializer)?; + match opt { + Some(s) => parse_interval(&s).map(Some).map_err(de::Error::custom), + None => Ok(None), + } +} + +fn parse_interval(value: &str) -> Result { + let not_an_interval = || format!("'{value}' is not an interval"); + + if !value.starts_with("interval ") { + return Err(not_an_interval()); + } + let mut it = value.split_whitespace(); + let _ = it.next(); // skip "interval" + let number = parse_int(it.next().ok_or_else(not_an_interval)?)?; + if number < 0 { + return Err(format!("interval '{value}' cannot be negative")); + } + let number = number as u64; + + let duration = match it.next().ok_or_else(not_an_interval)? { + "nanosecond" | "nanoseconds" => Duration::from_nanos(number), + "microsecond" | "microseconds" => Duration::from_micros(number), + "millisecond" | "milliseconds" => Duration::from_millis(number), + "second" | "seconds" => Duration::from_secs(number), + "minute" | "minutes" => Duration::from_secs(number * SECONDS_PER_MINUTE), + "hour" | "hours" => Duration::from_secs(number * SECONDS_PER_HOUR), + "day" | "days" => Duration::from_secs(number * SECONDS_PER_DAY), + "week" | "weeks" => Duration::from_secs(number * SECONDS_PER_WEEK), + unit => { + return Err(format!("Unknown unit '{unit}'")); + } + }; + + Ok(duration) +} + +fn parse_int(value: &str) -> Result { + value + .parse() + .map_err(|e| format!("Cannot parse '{value}' as integer: {e}")) +} From 1b7b193b3abd205bbdbaabd85bf3f0161ea66b83 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Tue, 5 Nov 2024 14:58:38 -0800 Subject: [PATCH 03/20] make everything optional --- kernel/src/scan/mod.rs | 4 +- kernel/src/table_properties.rs | 610 ++++----------------- kernel/src/table_properties/deserialize.rs | 170 +++++- 3 files changed, 262 insertions(+), 522 deletions(-) diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index 5a2353789..4865be786 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -95,7 +95,7 @@ impl ScanBuilder { let (all_fields, read_fields, have_partition_cols) = get_state_info( logical_schema.as_ref(), &self.snapshot.metadata().partition_columns, - self.snapshot.table_properties().column_mapping_mode, + self.snapshot.table_properties().get_column_mapping_mode(), )?; let physical_schema = Arc::new(StructType::new(read_fields)); Ok(Scan { @@ -247,7 +247,7 @@ impl Scan { partition_columns: self.snapshot.metadata().partition_columns.clone(), logical_schema: self.logical_schema.clone(), read_schema: self.physical_schema.clone(), - column_mapping_mode: self.snapshot.table_properties().column_mapping_mode, + column_mapping_mode: self.snapshot.table_properties().get_column_mapping_mode(), } } diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index 63b38c79b..80aed9932 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -27,52 +27,48 @@ pub const DEFAULT_NUM_INDEX_COLS: i32 = 32; /// Delta table properties. These are parsed from the 'configuration' map in the most recent /// 'Metadata' action of a table. -#[derive(Deserialize, Debug, Clone, Eq, PartialEq)] +#[derive(Deserialize, Debug, Clone, Eq, PartialEq, Default)] +#[serde(default)] pub struct TableProperties { /// true for this Delta table to be append-only. If append-only, /// existing records cannot be deleted, and existing values cannot be updated. #[serde(rename = "delta.appendOnly")] #[serde(deserialize_with = "deserialize_bool")] - #[serde(default)] - pub append_only: bool, + pub append_only: Option, /// true for Delta Lake to automatically optimize the layout of the files for this Delta table. #[serde(rename = "delta.autoOptimize.autoCompact")] #[serde(deserialize_with = "deserialize_bool")] - #[serde(default)] - pub auto_compact: bool, + pub auto_compact: Option, /// true for Delta Lake to automatically optimize the layout of the files for this Delta table /// during writes. #[serde(rename = "delta.autoOptimize.optimizeWrite")] #[serde(deserialize_with = "deserialize_bool")] - #[serde(default)] - pub optimize_write: bool, + pub optimize_write: Option, /// Interval (expressed as number of commits) after which a new checkpoint should be created. /// E.g. if checkpoint interval = 10, then a checkpoint should be written every 10 commits. #[serde(rename = "delta.checkpointInterval")] #[serde(deserialize_with = "deserialize_pos_int")] - #[serde(default = "default_checkpoint_interval")] - pub checkpoint_interval: u64, + pub checkpoint_interval: Option, /// true for Delta Lake to write file statistics in checkpoints in JSON format for the stats column. #[serde(rename = "delta.checkpoint.writeStatsAsJson")] #[serde(deserialize_with = "deserialize_bool")] - #[serde(default)] - pub checkpoint_write_stats_as_json: bool, + pub checkpoint_write_stats_as_json: Option, /// true for Delta Lake to write file statistics to checkpoints in struct format for the /// stats_parsed column and to write partition values as a struct for partitionValues_parsed. #[serde(rename = "delta.checkpoint.writeStatsAsStruct")] #[serde(deserialize_with = "deserialize_bool")] - #[serde(default)] - pub checkpoint_write_stats_as_struct: bool, + pub checkpoint_write_stats_as_struct: Option, /// Whether column mapping is enabled for Delta table columns and the corresponding /// Parquet columns that use different names. - #[serde(rename = "delta.columnMapping.mode", default)] - pub column_mapping_mode: ColumnMappingMode, + #[serde(rename = "delta.columnMapping.mode")] + #[serde(deserialize_with = "deserialize_option")] + pub column_mapping_mode: Option, /// The number of columns for Delta Lake to collect statistics about for data skipping. /// A value of -1 means to collect statistics for all columns. Updating this property does @@ -80,7 +76,8 @@ pub struct TableProperties { /// of the Delta table. Specifically, it changes the behavior of future statistics collection /// (such as during appends and optimizations) as well as data skipping (such as ignoring column /// statistics beyond this number, even when such statistics exist). - #[serde(rename = "delta.dataSkippingNumIndexedCols", default, flatten)] + #[serde(rename = "delta.dataSkippingNumIndexedCols")] + #[serde(deserialize_with = "deserialize_option")] pub data_skipping_num_indexed_cols: Option, /// A comma-separated list of column names on which Delta Lake collects statistics to enhance @@ -88,8 +85,7 @@ pub struct TableProperties { /// [DataSkippingNumIndexedCols](DeltaConfigKey::DataSkippingNumIndexedCols). #[serde(rename = "delta.dataSkippingStatsColumns")] #[serde(deserialize_with = "deserialize_column_names")] - #[serde(default)] - pub data_skipping_stats_columns: Vec, + pub data_skipping_stats_columns: Option>, /// The shortest duration for Delta Lake to keep logically deleted data files before deleting /// them physically. This is to prevent failures in stale readers after compactions or partition @@ -104,28 +100,25 @@ pub struct TableProperties { /// read old files. #[serde(rename = "delta.deletedFileRetentionDuration")] #[serde(deserialize_with = "deserialize_interval")] - #[serde(default)] pub deleted_file_retention_duration: Option, /// true to enable change data feed. #[serde(rename = "delta.enableChangeDataFeed")] #[serde(deserialize_with = "deserialize_bool")] - #[serde(default)] - pub enable_change_data_feed: bool, + pub enable_change_data_feed: Option, /// true to enable deletion vectors and predictive I/O for updates. #[serde(rename = "delta.enableDeletionVectors")] #[serde(deserialize_with = "deserialize_bool")] - #[serde(default)] - pub enable_deletion_vectors: bool, + pub enable_deletion_vectors: Option, /// The degree to which a transaction must be isolated from modifications made by concurrent /// transactions. /// /// Valid values are `Serializable` and `WriteSerializable`. #[serde(rename = "delta.isolationLevel")] - #[serde(default)] - pub isolation_level: IsolationLevel, + #[serde(deserialize_with = "deserialize_option")] + pub isolation_level: Option, /// How long the history for a Delta table is kept. /// @@ -136,13 +129,12 @@ pub struct TableProperties { /// size increases. #[serde(rename = "delta.logRetentionDuration")] #[serde(deserialize_with = "deserialize_interval")] - #[serde(default)] pub log_retention_duration: Option, /// TODO docs #[serde(rename = "delta.enableExpiredLogCleanup")] - #[serde(default)] - pub enable_expired_log_cleanup: bool, + #[serde(deserialize_with = "deserialize_bool")] + pub enable_expired_log_cleanup: Option, // /// The minimum required protocol reader version for a reader that allows to read from this Delta table. // #[serde(rename = "delta.minReaderVersion")] @@ -158,14 +150,13 @@ pub struct TableProperties { /// volumes of Amazon S3 calls to better partition across S3 servers. #[serde(rename = "delta.randomizeFilePrefixes")] #[serde(deserialize_with = "deserialize_bool")] - #[serde(default)] - pub randomize_file_prefixes: bool, + pub randomize_file_prefixes: Option, /// When delta.randomizeFilePrefixes is set to true, the number of characters that Delta /// generates for random prefixes. #[serde(rename = "delta.randomPrefixLength")] - #[serde(default)] - pub random_prefix_length: Option, + #[serde(deserialize_with = "deserialize_pos_int")] + pub random_prefix_length: Option, /// The shortest duration within which new snapshots will retain transaction identifiers (for /// example, SetTransactions). When a new snapshot sees a transaction identifier older than or @@ -173,7 +164,6 @@ pub struct TableProperties { /// ignores it. The SetTransaction identifier is used when making the writes idempotent. #[serde(rename = "delta.setTransactionRetentionDuration")] #[serde(deserialize_with = "deserialize_interval")] - #[serde(default)] pub set_transaction_retention_duration: Option, /// The target file size in bytes or higher units for file tuning. For example, 104857600 @@ -184,7 +174,7 @@ pub struct TableProperties { /// snapshot.table_properties().target_file_size in order to know how big _it_ should write /// parquet files. #[serde(rename = "delta.targetFileSize")] - #[serde(default)] + #[serde(deserialize_with = "deserialize_pos_int")] pub target_file_size: Option, /// The target file size in bytes or higher units for file tuning. For example, 104857600 @@ -192,13 +182,18 @@ pub struct TableProperties { /// /// See TODO above #[serde(rename = "delta.tuneFileSizesForRewrites")] - #[serde(default)] + #[serde(deserialize_with = "deserialize_bool")] pub tune_file_sizes_for_rewrites: Option, /// 'classic' for classic Delta Lake checkpoints. 'v2' for v2 checkpoints. #[serde(rename = "delta.checkpointPolicy")] - #[serde(default)] - pub checkpoint_policy: CheckpointPolicy, + #[serde(deserialize_with = "deserialize_option")] + pub checkpoint_policy: Option, + + /// TODO + #[serde(rename = "delta.enableRowTracking")] + #[serde(deserialize_with = "deserialize_bool")] + pub enable_row_tracking: Option, } impl TableProperties { @@ -207,10 +202,10 @@ impl TableProperties { // FIXME error TableProperties::deserialize(deserializer).map_err(|e| crate::Error::Generic(e.to_string())) } -} -fn default_checkpoint_interval() -> u64 { - 10 + pub(crate) fn get_column_mapping_mode(&self) -> ColumnMappingMode { + self.column_mapping_mode.unwrap_or(ColumnMappingMode::None) + } } #[derive(Debug, PartialEq, Eq, Clone, Deserialize)] @@ -235,108 +230,6 @@ impl TryFrom for DataSkippingNumIndexedCols { } } -// /// Delta configuration error -// #[derive(thiserror::Error, Debug, PartialEq, Eq)] -// pub enum DeltaConfigError { -// /// Error returned when configuration validation failed. -// #[error("Validation failed - {0}")] -// Validation(String), -// } -// -// impl From for Error { -// fn from(e: DeltaConfigError) -> Self { -// Error::InvalidConfiguration(e.to_string()) -// } -// } - -// /// The shortest duration for Delta Lake to keep logically deleted data files before deleting -// /// them physically. This is to prevent failures in stale readers after compactions or partition -// /// overwrites. -// /// -// /// This value should be large enough to ensure that: -// /// -// /// * It is larger than the longest possible duration of a job if you run VACUUM when there are -// /// concurrent readers or writers accessing the Delta table. -// /// * If you run a streaming query that reads from the table, that query does not stop for -// /// longer than this value. Otherwise, the query may not be able to restart, as it must still -// /// read old files. -// pub fn deleted_file_retention_duration(&self) -> Duration { -// static DEFAULT_FILE_RETENTION_DURATION: LazyLock = -// LazyLock::new(|| parse_interval("interval 1 weeks").unwrap()); -// self.0 -// .get(DeltaTableProperty::DeletedFileRetentionDuration.as_ref()) -// .and_then(|v| parse_interval(v).ok()) -// .unwrap_or_else(|| DEFAULT_FILE_RETENTION_DURATION.to_owned()) -// } -// -// /// How long the history for a Delta table is kept. -// /// -// /// Each time a checkpoint is written, Delta Lake automatically cleans up log entries older -// /// than the retention interval. If you set this property to a large enough value, many log -// /// entries are retained. This should not impact performance as operations against the log are -// /// constant time. Operations on history are parallel but will become more expensive as the log -// /// size increases. -// pub fn log_retention_duration(&self) -> Duration { -// static DEFAULT_LOG_RETENTION_DURATION: LazyLock = -// LazyLock::new(|| parse_interval("interval 30 days").unwrap()); -// self.0 -// .get(DeltaTableProperty::LogRetentionDuration.as_ref()) -// .and_then(|v| parse_interval(v).ok()) -// .unwrap_or_else(|| DEFAULT_LOG_RETENTION_DURATION.to_owned()) -// } -// -// /// The degree to which a transaction must be isolated from modifications made by concurrent -// /// transactions. -// /// -// /// Valid values are `Serializable` and `WriteSerializable`. -// pub fn isolation_level(&self) -> IsolationLevel { -// self.0 -// .get(DeltaTableProperty::IsolationLevel.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(DeltaTableProperty::CheckpointPolicy.as_ref()) -// .and_then(|v| v.parse().ok()) -// .unwrap_or_default() -// } -// -// /// Return the column mapping mode according to delta.columnMapping.mode -// pub fn column_mapping_mode(&self) -> ColumnMappingMode { -// self.0 -// .get(DeltaTableProperty::ColumnMappingMode.as_ref()) -// .and_then(|v| v.parse().ok()) -// .unwrap_or_default() -// } -// -// /// Return the check constraints on the current table -// pub fn get_constraints(&self) -> Vec { -// self.0 -// .iter() -// .filter_map(|(field, value)| { -// if field.starts_with("delta.constraints") { -// field -// .splitn(3, '.') -// .last() -// .map(|n| Constraint::new(n, value)) -// } else { -// None -// } -// }) -// .collect() -// } -// -// /// Column names on which Delta Lake collects statistics to enhance data skipping functionality. -// /// This property takes precedence over [num_indexed_cols](Self::num_indexed_cols). -// pub fn stats_columns(&self) -> Option> { -// self.0 -// .get(DeltaTableProperty::DataSkippingStatsColumns.as_ref()) -// .map(|v| v.split(',').collect()) -// } - /// The isolation level applied during transaction #[derive(Deserialize, Debug, Copy, Clone, PartialEq, Eq)] #[serde(rename_all = "camelCase")] @@ -361,16 +254,15 @@ pub enum IsolationLevel { SnapshotIsolation, } -// Spark assumes Serializable as default isolation level -// https://github.com/delta-io/delta/blob/abb171c8401200e7772b27e3be6ea8682528ac72/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala#L1023 +// Delta-Spark default isolation level is Serializable impl Default for IsolationLevel { fn default() -> Self { Self::Serializable } } -#[derive(Deserialize, Debug, Clone, PartialEq, Eq)] /// The checkpoint policy applied when writing checkpoints +#[derive(Deserialize, Debug, Clone, PartialEq, Eq)] #[serde(rename_all = "camelCase")] pub enum CheckpointPolicy { /// classic Delta Lake checkpoints @@ -388,24 +280,9 @@ impl Default for CheckpointPolicy { #[cfg(test)] mod tests { use super::*; - use crate::actions::{Format, Metadata}; - use crate::schema::StructType; + use crate::expressions::column_name; use std::collections::HashMap; - fn dummy_metadata() -> Metadata { - let schema = StructType::new(Vec::new()); - Metadata { - id: "id".into(), - name: None, - description: None, - format: Format::default(), - schema_string: serde_json::to_string(&schema).unwrap(), - partition_columns: Vec::new(), - configuration: HashMap::new(), - created_time: None, - } - } - #[test] fn fail_known_keys() { let properties = HashMap::from([("delta.appendOnly".to_string(), "wack".to_string())]); @@ -413,350 +290,83 @@ mod tests { assert!(TableProperties::deserialize(de).is_err()); } - // #[test] - // fn get_interval_from_metadata_test() { - // let md = dummy_metadata(); - // let config = TableConfig(&md.configuration); - - // // default 1 week - // assert_eq!( - // config.deleted_file_retention_duration().as_secs(), - // SECONDS_PER_WEEK, - // ); - - // // change to 2 day - // let mut md = dummy_metadata(); - // md.configuration.insert( - // DeltaTableProperty::DeletedFileRetentionDuration - // .as_ref() - // .to_string(), - // "interval 2 day".to_string(), - // ); - // let config = TableConfig(&md.configuration); - - // assert_eq!( - // config.deleted_file_retention_duration().as_secs(), - // 2 * SECONDS_PER_DAY, - // ); - // } - - // #[test] - // fn get_long_from_metadata_test() { - // let md = dummy_metadata(); - // let config = TableConfig(&md.configuration); - // assert_eq!(config.checkpoint_interval(), 10,) - // } - - // #[test] - // fn get_boolean_from_metadata_test() { - // let md = dummy_metadata(); - // let config = TableConfig(&md.configuration); - - // // default value is true - // assert!(config.enable_expired_log_cleanup()); - - // // change to false - // let mut md = dummy_metadata(); - // md.configuration.insert( - // DeltaTableProperty::EnableExpiredLogCleanup.as_ref().into(), - // "false".to_string(), - // ); - // let config = TableConfig(&md.configuration); - - // assert!(!config.enable_expired_log_cleanup()); - // } - - // #[test] - // fn parse_interval_test() { - // assert_eq!( - // parse_interval("interval 123 nanosecond").unwrap(), - // Duration::from_nanos(123) - // ); - - // assert_eq!( - // parse_interval("interval 123 nanoseconds").unwrap(), - // Duration::from_nanos(123) - // ); - - // assert_eq!( - // parse_interval("interval 123 microsecond").unwrap(), - // Duration::from_micros(123) - // ); - - // assert_eq!( - // parse_interval("interval 123 microseconds").unwrap(), - // Duration::from_micros(123) - // ); - - // assert_eq!( - // parse_interval("interval 123 millisecond").unwrap(), - // Duration::from_millis(123) - // ); - - // assert_eq!( - // parse_interval("interval 123 milliseconds").unwrap(), - // Duration::from_millis(123) - // ); - - // assert_eq!( - // parse_interval("interval 123 second").unwrap(), - // Duration::from_secs(123) - // ); - - // assert_eq!( - // parse_interval("interval 123 seconds").unwrap(), - // Duration::from_secs(123) - // ); - - // assert_eq!( - // parse_interval("interval 123 minute").unwrap(), - // Duration::from_secs(123 * 60) - // ); - - // assert_eq!( - // parse_interval("interval 123 minutes").unwrap(), - // Duration::from_secs(123 * 60) - // ); - - // assert_eq!( - // parse_interval("interval 123 hour").unwrap(), - // Duration::from_secs(123 * 3600) - // ); - - // assert_eq!( - // parse_interval("interval 123 hours").unwrap(), - // Duration::from_secs(123 * 3600) - // ); - - // assert_eq!( - // parse_interval("interval 123 day").unwrap(), - // Duration::from_secs(123 * 86400) - // ); - - // assert_eq!( - // parse_interval("interval 123 days").unwrap(), - // Duration::from_secs(123 * 86400) - // ); - - // assert_eq!( - // parse_interval("interval 123 week").unwrap(), - // Duration::from_secs(123 * 604800) - // ); - - // assert_eq!( - // parse_interval("interval 123 week").unwrap(), - // Duration::from_secs(123 * 604800) - // ); - // } - - // #[test] - // fn parse_interval_invalid_test() { - // assert_eq!( - // parse_interval("whatever").err().unwrap(), - // DeltaConfigError::Validation("'whatever' is not an interval".to_string()) - // ); - - // assert_eq!( - // parse_interval("interval").err().unwrap(), - // DeltaConfigError::Validation("'interval' is not an interval".to_string()) - // ); - - // assert_eq!( - // parse_interval("interval 2").err().unwrap(), - // DeltaConfigError::Validation("'interval 2' is not an interval".to_string()) - // ); - - // assert_eq!( - // parse_interval("interval 2 years").err().unwrap(), - // DeltaConfigError::Validation("Unknown unit 'years'".to_string()) - // ); - - // assert_eq!( - // parse_interval("interval two years").err().unwrap(), - // DeltaConfigError::Validation( - // "Cannot parse 'two' as integer: invalid digit found in string".to_string() - // ) - // ); - - // assert_eq!( - // parse_interval("interval -25 hours").err().unwrap(), - // DeltaConfigError::Validation( - // "interval 'interval -25 hours' cannot be negative".to_string() - // ) - // ); - // } - - // #[test] - // fn test_constraint() { - // let md = dummy_metadata(); - // let config = TableConfig(&md.configuration); - - // assert_eq!(config.get_constraints().len(), 0); - - // let mut md = dummy_metadata(); - // md.configuration.insert( - // "delta.constraints.name".to_string(), - // "name = 'foo'".to_string(), - // ); - // md.configuration - // .insert("delta.constraints.age".to_string(), "age > 10".to_string()); - // let config = TableConfig(&md.configuration); - - // let constraints = config.get_constraints(); - // assert_eq!(constraints.len(), 2); - // assert!(constraints.contains(&Constraint::new("name", "name = 'foo'"))); - // assert!(constraints.contains(&Constraint::new("age", "age > 10"))); - // } - - // #[test] - // fn test_roundtrip_config_key() { - // let cases = [ - // (DeltaTableProperty::AppendOnly, "delta.appendOnly"), - // ( - // DeltaTableProperty::AutoOptimizeAutoCompact, - // "delta.autoOptimize.autoCompact", - // ), - // ( - // DeltaTableProperty::AutoOptimizeOptimizeWrite, - // "delta.autoOptimize.optimizeWrite", - // ), - // ( - // DeltaTableProperty::CheckpointInterval, - // "delta.checkpointInterval", - // ), - // ( - // DeltaTableProperty::CheckpointWriteStatsAsJson, - // "delta.checkpoint.writeStatsAsJson", - // ), - // ( - // DeltaTableProperty::CheckpointWriteStatsAsStruct, - // "delta.checkpoint.writeStatsAsStruct", - // ), - // ( - // DeltaTableProperty::ColumnMappingMode, - // "delta.columnMapping.mode", - // ), - // ( - // DeltaTableProperty::DataSkippingNumIndexedCols, - // "delta.dataSkippingNumIndexedCols", - // ), - // ( - // DeltaTableProperty::DataSkippingStatsColumns, - // "delta.dataSkippingStatsColumns", - // ), - // ( - // DeltaTableProperty::DeletedFileRetentionDuration, - // "delta.deletedFileRetentionDuration", - // ), - // ( - // DeltaTableProperty::EnableChangeDataFeed, - // "delta.enableChangeDataFeed", - // ), - // ( - // DeltaTableProperty::EnableDeletionVectors, - // "delta.enableDeletionVectors", - // ), - // (DeltaTableProperty::IsolationLevel, "delta.isolationLevel"), - // ( - // DeltaTableProperty::LogRetentionDuration, - // "delta.logRetentionDuration", - // ), - // ( - // DeltaTableProperty::MinReaderVersion, - // "delta.minReaderVersion", - // ), - // ( - // DeltaTableProperty::MinWriterVersion, - // "delta.minWriterVersion", - // ), - // ( - // DeltaTableProperty::RandomizeFilePrefixes, - // "delta.randomizeFilePrefixes", - // ), - // ( - // DeltaTableProperty::RandomPrefixLength, - // "delta.randomPrefixLength", - // ), - // ( - // DeltaTableProperty::SetTransactionRetentionDuration, - // "delta.setTransactionRetentionDuration", - // ), - // ( - // DeltaTableProperty::EnableExpiredLogCleanup, - // "delta.enableExpiredLogCleanup", - // ), - // (DeltaTableProperty::TargetFileSize, "delta.targetFileSize"), - // ( - // DeltaTableProperty::TuneFileSizesForRewrites, - // "delta.tuneFileSizesForRewrites", - // ), - // ( - // DeltaTableProperty::CheckpointPolicy, - // "delta.checkpointPolicy", - // ), - // ]; - - // assert_eq!(DeltaTableProperty::VARIANTS.len(), cases.len()); - - // for (key, expected) in cases { - // assert_eq!(key.as_ref(), expected); - - // let serialized = serde_json::to_string(&key).unwrap(); - // assert_eq!(serialized, format!("\"{}\"", expected)); - - // let deserialized: DeltaTableProperty = serde_json::from_str(&serialized).unwrap(); - // assert_eq!(deserialized, key); - - // let from_str: DeltaTableProperty = expected.parse().unwrap(); - // assert_eq!(from_str, key); - // } - // } - - // #[test] - // fn test_default_config() { - // let md = dummy_metadata(); - // let config = TableConfig(&md.configuration); - - // assert_eq!(config.append_only(), false); - // // assert_eq!(config.auto_optimize_auto_compact(), false); - // assert_eq!(config.auto_optimize_optimize_write(), false); - // assert_eq!(config.checkpoint_interval(), 10); - // assert_eq!(config.write_stats_as_json(), true); - // assert_eq!(config.write_stats_as_struct(), false); - // assert_eq!(config.target_file_size(), 104857600); - // assert_eq!(config.enable_change_data_feed(), false); - // assert_eq!(config.enable_deletion_vectors(), false); - // assert_eq!(config.num_indexed_cols(), 32); - // assert_eq!(config.enable_expired_log_cleanup(), true); - // } + #[test] + fn allow_unknown_keys() { + let properties = + HashMap::from([("some_random_unknown_key".to_string(), "test".to_string())]); + let de = StringMapDeserializer::new(&properties); + let actual = TableProperties::deserialize(de).unwrap(); + let expected = TableProperties::default(); + assert_eq!(actual, expected); + } + + #[test] + fn test_empty_table_properties() { + let map = HashMap::new(); + let de = StringMapDeserializer::new(&map); + let actual = TableProperties::deserialize(de).unwrap(); + let default_table_properties = TableProperties::default(); + assert_eq!(actual, default_table_properties); + } #[test] - fn it_works() { - let properties = HashMap::from([("delta.appendOnly".to_string(), "true".to_string())]); + fn test_parse_table_properties() { + let properties = [ + ("delta.appendOnly", "true"), + ("delta.autoOptimize.optimizeWrite", "true"), + ("delta.autoOptimize.autoCompact", "true"), + ("delta.checkpointInterval", "101"), + ("delta.checkpoint.writeStatsAsJson", "true"), + ("delta.checkpoint.writeStatsAsStruct", "true"), + ("delta.columnMapping.mode", "id"), + ("delta.dataSkippingNumIndexedCols", "-1"), + ("delta.dataSkippingStatsColumns", "col1,col2"), + ("delta.deletedFileRetentionDuration", "interval 1 second"), + ("delta.enableChangeDataFeed", "true"), + ("delta.enableDeletionVectors", "true"), + ("delta.isolationLevel", "snapshotIsolation"), + ("delta.logRetentionDuration", "interval 2 seconds"), + ("delta.enableExpiredLogCleanup", "true"), + ("delta.randomizeFilePrefixes", "true"), + ("delta.randomPrefixLength", "1001"), + ( + "delta.setTransactionRetentionDuration", + "interval 60 seconds", + ), + ("delta.targetFileSize", "1000000000"), + ("delta.tuneFileSizesForRewrites", "true"), + ("delta.checkpointPolicy", "v2"), + ("delta.enableRowTracking", "true"), + ]; + let properties = properties + .into_iter() + .map(|(k, v)| (k.to_string(), v.to_string())) + .collect(); let de = StringMapDeserializer::new(&properties); let actual = TableProperties::deserialize(de).unwrap(); let expected = TableProperties { - append_only: true, - optimize_write: false, - auto_compact: false, - checkpoint_interval: 10, - checkpoint_write_stats_as_json: false, - checkpoint_write_stats_as_struct: false, - column_mapping_mode: ColumnMappingMode::None, - data_skipping_num_indexed_cols: None, - data_skipping_stats_columns: vec![], - deleted_file_retention_duration: None, - enable_change_data_feed: false, - enable_deletion_vectors: false, - isolation_level: IsolationLevel::Serializable, - log_retention_duration: None, - enable_expired_log_cleanup: false, - randomize_file_prefixes: false, - random_prefix_length: None, - set_transaction_retention_duration: None, - target_file_size: None, - tune_file_sizes_for_rewrites: None, - checkpoint_policy: CheckpointPolicy::Classic, + append_only: Some(true), + optimize_write: Some(true), + auto_compact: Some(true), + checkpoint_interval: Some(101), + checkpoint_write_stats_as_json: Some(true), + checkpoint_write_stats_as_struct: Some(true), + column_mapping_mode: Some(ColumnMappingMode::Id), + data_skipping_num_indexed_cols: Some(DataSkippingNumIndexedCols::AllColumns), + data_skipping_stats_columns: Some(vec![column_name!("col1"), column_name!("col2")]), + deleted_file_retention_duration: Some(Duration::new(1, 0)), + enable_change_data_feed: Some(true), + enable_deletion_vectors: Some(true), + isolation_level: Some(IsolationLevel::SnapshotIsolation), + log_retention_duration: Some(Duration::new(2, 0)), + enable_expired_log_cleanup: Some(true), + randomize_file_prefixes: Some(true), + random_prefix_length: Some(1001), + set_transaction_retention_duration: Some(Duration::new(60, 0)), + target_file_size: Some(1_000_000_000), + tune_file_sizes_for_rewrites: Some(true), + checkpoint_policy: Some(CheckpointPolicy::V2), + enable_row_tracking: Some(true), }; assert_eq!(actual, expected); } diff --git a/kernel/src/table_properties/deserialize.rs b/kernel/src/table_properties/deserialize.rs index 247593e98..16a4535ff 100644 --- a/kernel/src/table_properties/deserialize.rs +++ b/kernel/src/table_properties/deserialize.rs @@ -73,50 +73,59 @@ impl<'de> MapAccess<'de> for HashMapMapAccess<'de> { } } -pub(crate) fn deserialize_pos_int<'de, D>(deserializer: D) -> Result +pub(crate) fn deserialize_option<'de, D, T>(deserializer: D) -> Result, D::Error> where - D: de::Deserializer<'de>, + D: Deserializer<'de>, + T: Deserialize<'de>, { - let s: String = de::Deserialize::deserialize(deserializer)?; + T::deserialize(deserializer).map(Some) +} + +pub(crate) fn deserialize_pos_int<'de, D>(deserializer: D) -> Result, D::Error> +where + D: Deserializer<'de>, +{ + let s: String = Deserialize::deserialize(deserializer)?; let n: u64 = s.parse().unwrap(); // FIXME if n == 0 { panic!("FIXME"); // return Err("something"); } - Ok(n) + Ok(Some(n)) } -pub(crate) fn deserialize_bool<'de, D>(deserializer: D) -> Result +pub(crate) fn deserialize_bool<'de, D>(deserializer: D) -> Result, D::Error> where - D: de::Deserializer<'de>, + D: Deserializer<'de>, { - let s: String = de::Deserialize::deserialize(deserializer)?; + let s: String = Deserialize::deserialize(deserializer)?; match s.as_str() { - "true" => Ok(true), - "false" => Ok(false), + "true" => Ok(Some(true)), + "false" => Ok(Some(false)), _ => Err(de::Error::unknown_variant(&s, &["true", "false"])), } } -pub(crate) fn deserialize_column_names<'de, D>(deserializer: D) -> Result, D::Error> +pub(crate) fn deserialize_column_names<'de, D>( + deserializer: D, +) -> Result>, D::Error> where - D: de::Deserializer<'de>, + D: Deserializer<'de>, { - let s: String = de::Deserialize::deserialize(deserializer)?; - Ok(s.split(',') - .map(|name: &str| ColumnName::new([name])) - .collect()) + let s: String = Deserialize::deserialize(deserializer)?; + Ok(Some( + s.split(',') + .map(|name: &str| ColumnName::new([name])) + .collect(), + )) } pub(crate) fn deserialize_interval<'de, D>(deserializer: D) -> Result, D::Error> where D: Deserializer<'de>, { - let opt = Option::::deserialize(deserializer)?; - match opt { - Some(s) => parse_interval(&s).map(Some).map_err(de::Error::custom), - None => Ok(None), - } + let s = String::deserialize(deserializer)?; + parse_interval(&s).map(Some).map_err(de::Error::custom) } fn parse_interval(value: &str) -> Result { @@ -155,3 +164,124 @@ fn parse_int(value: &str) -> Result { .parse() .map_err(|e| format!("Cannot parse '{value}' as integer: {e}")) } + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_parse_interval() { + assert_eq!( + parse_interval("interval 123 nanosecond").unwrap(), + Duration::from_nanos(123) + ); + + assert_eq!( + parse_interval("interval 123 nanoseconds").unwrap(), + Duration::from_nanos(123) + ); + + assert_eq!( + parse_interval("interval 123 microsecond").unwrap(), + Duration::from_micros(123) + ); + + assert_eq!( + parse_interval("interval 123 microseconds").unwrap(), + Duration::from_micros(123) + ); + + assert_eq!( + parse_interval("interval 123 millisecond").unwrap(), + Duration::from_millis(123) + ); + + assert_eq!( + parse_interval("interval 123 milliseconds").unwrap(), + Duration::from_millis(123) + ); + + assert_eq!( + parse_interval("interval 123 second").unwrap(), + Duration::from_secs(123) + ); + + assert_eq!( + parse_interval("interval 123 seconds").unwrap(), + Duration::from_secs(123) + ); + + assert_eq!( + parse_interval("interval 123 minute").unwrap(), + Duration::from_secs(123 * 60) + ); + + assert_eq!( + parse_interval("interval 123 minutes").unwrap(), + Duration::from_secs(123 * 60) + ); + + assert_eq!( + parse_interval("interval 123 hour").unwrap(), + Duration::from_secs(123 * 3600) + ); + + assert_eq!( + parse_interval("interval 123 hours").unwrap(), + Duration::from_secs(123 * 3600) + ); + + assert_eq!( + parse_interval("interval 123 day").unwrap(), + Duration::from_secs(123 * 86400) + ); + + assert_eq!( + parse_interval("interval 123 days").unwrap(), + Duration::from_secs(123 * 86400) + ); + + assert_eq!( + parse_interval("interval 123 week").unwrap(), + Duration::from_secs(123 * 604800) + ); + + assert_eq!( + parse_interval("interval 123 week").unwrap(), + Duration::from_secs(123 * 604800) + ); + } + + #[test] + fn test_invalid_parse_interval() { + assert_eq!( + parse_interval("whatever").err().unwrap(), + "'whatever' is not an interval".to_string() + ); + + assert_eq!( + parse_interval("interval").err().unwrap(), + "'interval' is not an interval".to_string() + ); + + assert_eq!( + parse_interval("interval 2").err().unwrap(), + "'interval 2' is not an interval".to_string() + ); + + assert_eq!( + parse_interval("interval 2 years").err().unwrap(), + "Unknown unit 'years'".to_string() + ); + + assert_eq!( + parse_interval("interval two years").err().unwrap(), + "Cannot parse 'two' as integer: invalid digit found in string".to_string() + ); + + assert_eq!( + parse_interval("interval -25 hours").err().unwrap(), + "interval 'interval -25 hours' cannot be negative".to_string() + ); + } +} From 02d50eedf7e18c5d4e1e181017d628f7b676f66a Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 8 Nov 2024 12:58:13 -0800 Subject: [PATCH 04/20] errors, comments, cleanup --- ffi/src/lib.rs | 2 + kernel/src/actions/mod.rs | 4 +- kernel/src/error.rs | 7 +++ kernel/src/snapshot.rs | 7 +-- kernel/src/table_properties.rs | 68 +++++++++++----------- kernel/src/table_properties/deserialize.rs | 65 +-------------------- 6 files changed, 47 insertions(+), 106 deletions(-) diff --git a/ffi/src/lib.rs b/ffi/src/lib.rs index 290c3a09d..5e39dd732 100644 --- a/ffi/src/lib.rs +++ b/ffi/src/lib.rs @@ -334,6 +334,7 @@ pub enum KernelError { InvalidCommitInfo, FileAlreadyExists, MissingCommitInfo, + InvalidTableProperties, } impl From for KernelError { @@ -384,6 +385,7 @@ impl From for KernelError { Error::InvalidCommitInfo(_) => KernelError::InvalidCommitInfo, Error::FileAlreadyExists(_) => KernelError::FileAlreadyExists, Error::MissingCommitInfo => KernelError::MissingCommitInfo, + Error::InvalidTableProperties(_) => KernelError::InvalidTableProperties, } } } diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 985bda04b..73d09ad97 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -116,8 +116,8 @@ impl Metadata { } /// Parse the metadata configuration HashMap into a TableProperties struct. - pub fn get_table_properties(&self) -> DeltaResult { - TableProperties::new(&self.configuration) + pub fn parse_table_properties(&self) -> DeltaResult { + TableProperties::new(self.configuration.clone()) } } diff --git a/kernel/src/error.rs b/kernel/src/error.rs index 6396f5b8f..82488539c 100644 --- a/kernel/src/error.rs +++ b/kernel/src/error.rs @@ -171,6 +171,10 @@ pub enum Error { /// The file already exists at the path, prohibiting a non-overwrite write #[error("File already exists: {0}")] FileAlreadyExists(String), + + /// The `metadata` actions's `configuration` field was unable to parse into `TableProperties` + #[error("Invalid table properties: {0}")] + InvalidTableProperties(String), } // Convenience constructors for Error types that take a String argument @@ -222,6 +226,9 @@ impl Error { pub(crate) fn invalid_log_path(msg: impl ToString) -> Self { Self::InvalidLogPath(msg.to_string()) } + pub(crate) fn invalid_table_properties(msg: impl ToString) -> Self { + Self::InvalidTableProperties(msg.to_string()) + } pub fn internal_error(msg: impl ToString) -> Self { Self::InternalError(msg.to_string()).with_backtrace() diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 129a3ed93..cf2779dd4 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -230,12 +230,7 @@ impl Snapshot { .read_metadata(engine)? .ok_or(Error::MissingMetadata)?; let schema = metadata.schema()?; - // FIXME - // let column_mapping_mode = match metadata.configuration.get(COLUMN_MAPPING_MODE_KEY) { - // Some(mode) if protocol.min_reader_version >= 2 => mode.as_str().try_into(), - // _ => Ok(ColumnMappingMode::None), - // }?; - let table_properties = metadata.get_table_properties()?; + let table_properties = metadata.parse_table_properties()?; Ok(Self { table_root: location, log_segment, diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index 80aed9932..9db47a625 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -13,11 +13,12 @@ use std::collections::HashMap; use std::time::Duration; +use serde::de::{self, value::MapDeserializer}; use serde::Deserialize; use crate::expressions::ColumnName; use crate::table_features::ColumnMappingMode; -use crate::DeltaResult; +use crate::{DeltaResult, Error}; mod deserialize; use deserialize::*; @@ -136,14 +137,6 @@ pub struct TableProperties { #[serde(deserialize_with = "deserialize_bool")] pub enable_expired_log_cleanup: Option, - // /// The minimum required protocol reader version for a reader that allows to read from this Delta table. - // #[serde(rename = "delta.minReaderVersion")] - // pub min_reader_version: Option, - - // /// The minimum required protocol writer version for a writer that allows to write to this Delta table. - // #[serde(rename = "delta.minWriterVersion")] - // pub min_writer_version: Option, - // /// true for Delta to generate a random prefix for a file path instead of partition information. /// /// For example, this may improve Amazon S3 performance when Delta Lake needs to send very high @@ -168,19 +161,12 @@ pub struct TableProperties { /// The target file size in bytes or higher units for file tuning. For example, 104857600 /// (bytes) or 100mb. - /// - /// TODO: kernel doesn't govern file writes? should we pass this through just with a note that - /// we don't pay attention to it? Scenario: engine calls - /// snapshot.table_properties().target_file_size in order to know how big _it_ should write - /// parquet files. #[serde(rename = "delta.targetFileSize")] #[serde(deserialize_with = "deserialize_pos_int")] pub target_file_size: Option, /// The target file size in bytes or higher units for file tuning. For example, 104857600 /// (bytes) or 100mb. - /// - /// See TODO above #[serde(rename = "delta.tuneFileSizesForRewrites")] #[serde(deserialize_with = "deserialize_bool")] pub tune_file_sizes_for_rewrites: Option, @@ -190,17 +176,20 @@ pub struct TableProperties { #[serde(deserialize_with = "deserialize_option")] pub checkpoint_policy: Option, - /// TODO + /// whether to enable row tracking during writes. #[serde(rename = "delta.enableRowTracking")] #[serde(deserialize_with = "deserialize_bool")] pub enable_row_tracking: Option, + + /// any unrecognized properties are passed through and ignored by the parser + #[serde(flatten)] + pub unknown_properties: HashMap, } impl TableProperties { - pub(crate) fn new(config_map: &HashMap) -> DeltaResult { - let deserializer = StringMapDeserializer::new(config_map); - // FIXME error - TableProperties::deserialize(deserializer).map_err(|e| crate::Error::Generic(e.to_string())) + pub(crate) fn new(config_map: HashMap) -> DeltaResult { + let deserializer = MapDeserializer::<_, de::value::Error>::new(config_map.into_iter()); + TableProperties::deserialize(deserializer).map_err(|e| Error::invalid_table_properties(e)) } pub(crate) fn get_column_mapping_mode(&self) -> ColumnMappingMode { @@ -216,16 +205,23 @@ pub enum DataSkippingNumIndexedCols { } impl TryFrom for DataSkippingNumIndexedCols { - type Error = String; + type Error = Error; fn try_from(value: String) -> Result { - let num: i64 = value - .parse() - .map_err(|_| format!("couldn't parse {value} as i64"))?; // FIXME + let num: i64 = value.parse().map_err(|_| { + Error::invalid_table_properties( + "couldn't parse DataSkippingNumIndexedCols to an integer", + ) + })?; match num { -1 => Ok(DataSkippingNumIndexedCols::AllColumns), - x if x > -1 => Ok(DataSkippingNumIndexedCols::NumColumns(x as u64)), - _ => Err(format!("Invalid value: {}", value)), + x => Ok(DataSkippingNumIndexedCols::NumColumns( + x.try_into().map_err(|_| { + Error::invalid_table_properties( + "couldn't parse DataSkippingNumIndexedCols to positive integer", + ) + })?, + )), } } } @@ -286,7 +282,7 @@ mod tests { #[test] fn fail_known_keys() { let properties = HashMap::from([("delta.appendOnly".to_string(), "wack".to_string())]); - let de = StringMapDeserializer::new(&properties); + let de = MapDeserializer::<_, de::value::Error>::new(properties.clone().into_iter()); assert!(TableProperties::deserialize(de).is_err()); } @@ -294,16 +290,19 @@ mod tests { fn allow_unknown_keys() { let properties = HashMap::from([("some_random_unknown_key".to_string(), "test".to_string())]); - let de = StringMapDeserializer::new(&properties); + let de = MapDeserializer::<_, de::value::Error>::new(properties.clone().into_iter()); let actual = TableProperties::deserialize(de).unwrap(); - let expected = TableProperties::default(); + let expected = TableProperties { + unknown_properties: properties, + ..Default::default() + }; assert_eq!(actual, expected); } #[test] fn test_empty_table_properties() { - let map = HashMap::new(); - let de = StringMapDeserializer::new(&map); + let map: HashMap = HashMap::new(); + let de = MapDeserializer::<_, de::value::Error>::new(map.into_iter()); let actual = TableProperties::deserialize(de).unwrap(); let default_table_properties = TableProperties::default(); assert_eq!(actual, default_table_properties); @@ -338,11 +337,11 @@ mod tests { ("delta.checkpointPolicy", "v2"), ("delta.enableRowTracking", "true"), ]; - let properties = properties + let properties: HashMap<_, _> = properties .into_iter() .map(|(k, v)| (k.to_string(), v.to_string())) .collect(); - let de = StringMapDeserializer::new(&properties); + let de = MapDeserializer::<_, de::value::Error>::new(properties.clone().into_iter()); let actual = TableProperties::deserialize(de).unwrap(); let expected = TableProperties { append_only: Some(true), @@ -367,6 +366,7 @@ mod tests { tune_file_sizes_for_rewrites: Some(true), checkpoint_policy: Some(CheckpointPolicy::V2), enable_row_tracking: Some(true), + unknown_properties: HashMap::new(), }; assert_eq!(actual, expected); } diff --git a/kernel/src/table_properties/deserialize.rs b/kernel/src/table_properties/deserialize.rs index 16a4535ff..ffb3681ad 100644 --- a/kernel/src/table_properties/deserialize.rs +++ b/kernel/src/table_properties/deserialize.rs @@ -1,9 +1,8 @@ use crate::expressions::ColumnName; -use std::collections::HashMap; use std::time::Duration; -use serde::de::{self, DeserializeSeed, Deserializer, MapAccess, Visitor}; +use serde::de::{self, Deserializer}; use serde::Deserialize; const SECONDS_PER_MINUTE: u64 = 60; @@ -11,68 +10,6 @@ const SECONDS_PER_HOUR: u64 = 60 * SECONDS_PER_MINUTE; const SECONDS_PER_DAY: u64 = 24 * SECONDS_PER_HOUR; const SECONDS_PER_WEEK: u64 = 7 * SECONDS_PER_DAY; -pub(crate) struct StringMapDeserializer<'de> { - iter: std::collections::hash_map::Iter<'de, String, String>, -} - -impl<'de> StringMapDeserializer<'de> { - pub(crate) fn new(map: &'de HashMap) -> Self { - StringMapDeserializer { iter: map.iter() } - } -} - -impl<'de> Deserializer<'de> for StringMapDeserializer<'de> { - type Error = de::value::Error; - - fn deserialize_any(self, visitor: V) -> Result - where - V: Visitor<'de>, - { - visitor.visit_map(HashMapMapAccess { - iter: self.iter, - value: None, - }) - } - - serde::forward_to_deserialize_any! { - bool i8 i16 i32 i64 u8 u16 u32 u64 f32 f64 char str string bytes - byte_buf option unit unit_struct newtype_struct seq tuple tuple_struct - map struct enum identifier ignored_any - } -} - -struct HashMapMapAccess<'de> { - iter: std::collections::hash_map::Iter<'de, String, String>, - value: Option<&'de String>, -} - -impl<'de> MapAccess<'de> for HashMapMapAccess<'de> { - type Error = de::value::Error; - - fn next_key_seed(&mut self, seed: K) -> Result, Self::Error> - where - K: DeserializeSeed<'de>, - { - match self.iter.next() { - Some((key, value)) => { - self.value = Some(value); - let de = de::value::StrDeserializer::new(key); - seed.deserialize(de).map(Some) - } - None => Ok(None), - } - } - - fn next_value_seed(&mut self, seed: V) -> Result - where - V: DeserializeSeed<'de>, - { - let value = self.value.take().unwrap(); // FIXME - let de = de::value::StrDeserializer::new(value); - seed.deserialize(de) - } -} - pub(crate) fn deserialize_option<'de, D, T>(deserializer: D) -> Result, D::Error> where D: Deserializer<'de>, From 9f8afa4350488866c22bb2e4b6d6d83f66653b49 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 8 Nov 2024 14:50:22 -0800 Subject: [PATCH 05/20] use new col name list parsing --- kernel/src/table_properties/deserialize.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/kernel/src/table_properties/deserialize.rs b/kernel/src/table_properties/deserialize.rs index ffb3681ad..b30d7d7fd 100644 --- a/kernel/src/table_properties/deserialize.rs +++ b/kernel/src/table_properties/deserialize.rs @@ -51,9 +51,7 @@ where { let s: String = Deserialize::deserialize(deserializer)?; Ok(Some( - s.split(',') - .map(|name: &str| ColumnName::new([name])) - .collect(), + ColumnName::parse_column_name_list(&s).map_err(de::Error::custom)?, )) } From 42e602842ad85a1862890cf400ff72fb0cac5ab5 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 18 Nov 2024 12:29:04 -0800 Subject: [PATCH 06/20] docs --- kernel/src/actions/mod.rs | 2 +- kernel/src/snapshot.rs | 4 +--- kernel/src/table_properties.rs | 6 ++++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 73d09ad97..d0a810a15 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -100,7 +100,7 @@ pub struct Metadata { pub partition_columns: Vec, /// The time when this metadata action is created, in milliseconds since the Unix epoch pub created_time: Option, - /// Configuration options for the metadata action. These are parsed into `TableProperties`. + /// Configuration options for the metadata action. These are parsed into [`TableProperties`]. pub configuration: HashMap, } diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index b62dd458e..7365a77dc 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -161,9 +161,7 @@ impl Snapshot { &self.protocol } - /// Get the [column mapping - /// mode](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#column-mapping) at this - /// `Snapshot`s version. + /// Get the [`TableProperties`] for this [`Snapshot`]. pub fn table_properties(&self) -> &TableProperties { &self.table_properties } diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index 9db47a625..40b9ff7c7 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -31,8 +31,10 @@ pub const DEFAULT_NUM_INDEX_COLS: i32 = 32; #[derive(Deserialize, Debug, Clone, Eq, PartialEq, Default)] #[serde(default)] pub struct TableProperties { - /// true for this Delta table to be append-only. If append-only, - /// existing records cannot be deleted, and existing values cannot be updated. + /// true for this Delta table to be append-only. If append-only, existing records cannot be + /// deleted, and existing values cannot be updated. See [append-only tables] in the protocol. + /// + /// [append-only tables]: https://github.com/delta-io/delta/blob/master/PROTOCOL.md#append-only-tables #[serde(rename = "delta.appendOnly")] #[serde(deserialize_with = "deserialize_bool")] pub append_only: Option, From 82370b44d0e5002188ee3efc9989ee3680ef25f9 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 18 Nov 2024 12:37:28 -0800 Subject: [PATCH 07/20] remove derive --- kernel/src/expressions/column_names.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/kernel/src/expressions/column_names.rs b/kernel/src/expressions/column_names.rs index 1584250a9..0ea7a7067 100644 --- a/kernel/src/expressions/column_names.rs +++ b/kernel/src/expressions/column_names.rs @@ -6,10 +6,8 @@ use std::hash::{Hash, Hasher}; use std::iter::Peekable; use std::ops::Deref; -use serde::Deserialize; - /// A (possibly nested) column name. -#[derive(Debug, Clone, PartialEq, PartialOrd, Eq, Ord, Deserialize)] +#[derive(Debug, Clone, PartialEq, PartialOrd, Eq, Ord)] pub struct ColumnName { path: Vec, } From 00b9d8ebbaadb729a3a652779954505ea39af151 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 18 Nov 2024 13:44:20 -0800 Subject: [PATCH 08/20] make deserializer work on hashmap ref --- kernel/src/actions/mod.rs | 2 +- kernel/src/table_properties.rs | 9 +++++++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 99281b4cc..0cd19312d 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -125,7 +125,7 @@ impl Metadata { /// Parse the metadata configuration HashMap into a TableProperties struct. pub fn parse_table_properties(&self) -> DeltaResult { - TableProperties::new(self.configuration.clone()) + TableProperties::new(self.configuration.iter()) } } diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index 40b9ff7c7..e3c076c53 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -189,8 +189,13 @@ pub struct TableProperties { } impl TableProperties { - pub(crate) fn new(config_map: HashMap) -> DeltaResult { - let deserializer = MapDeserializer::<_, de::value::Error>::new(config_map.into_iter()); + pub(crate) fn new<'a, I>(items: I) -> DeltaResult + where + I: Iterator + 'a, + { + let deserializer = MapDeserializer::<_, de::value::Error>::new( + items.map(|(k, v)| (k.as_str(), v.as_str())), + ); TableProperties::deserialize(deserializer).map_err(|e| Error::invalid_table_properties(e)) } From f748f876ee062ca7d77c0ad2c6a91cd5e086ad9f Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 18 Nov 2024 16:47:01 -0800 Subject: [PATCH 09/20] fix column mapping mode check --- acceptance/tests/dat_reader.rs | 5 ++ kernel/src/scan/mod.rs | 4 +- kernel/src/snapshot.rs | 5 ++ kernel/src/table_features/column_mapping.rs | 61 +++++++++++++++++++++ kernel/src/table_features/mod.rs | 4 +- kernel/src/table_properties.rs | 9 +-- 6 files changed, 76 insertions(+), 12 deletions(-) diff --git a/acceptance/tests/dat_reader.rs b/acceptance/tests/dat_reader.rs index ee0e1513f..321aa26b3 100644 --- a/acceptance/tests/dat_reader.rs +++ b/acceptance/tests/dat_reader.rs @@ -12,6 +12,11 @@ fn reader_test(path: &Path) -> datatest_stable::Result<()> { path.parent().unwrap().to_str().unwrap() ); + // TODO(zach): skip iceberg_compat_v1 test until DAT is fixed + if path.ends_with("iceberg_compat_v1/test_case_info.json") { + return Ok(()); + } + tokio::runtime::Builder::new_current_thread() .enable_all() .build()? diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index feea8cf81..161515964 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -95,7 +95,7 @@ impl ScanBuilder { let (all_fields, read_fields, have_partition_cols) = get_state_info( logical_schema.as_ref(), &self.snapshot.metadata().partition_columns, - self.snapshot.table_properties().get_column_mapping_mode(), + self.snapshot.column_mapping_mode, )?; let physical_schema = Arc::new(StructType::new(read_fields)); @@ -251,7 +251,7 @@ impl Scan { partition_columns: self.snapshot.metadata().partition_columns.clone(), logical_schema: self.logical_schema.clone(), read_schema: self.physical_schema.clone(), - column_mapping_mode: self.snapshot.table_properties().get_column_mapping_mode(), + column_mapping_mode: self.snapshot.column_mapping_mode, } } diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 7673b7ef0..1de56170c 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -11,6 +11,7 @@ use crate::log_segment::LogSegment; use crate::scan::ScanBuilder; use crate::schema::Schema; use crate::table_properties::TableProperties; +use crate::table_features::{ColumnMappingMode, column_mapping_mode}; use crate::{DeltaResult, Engine, Error, FileSystemClient, Version}; const LAST_CHECKPOINT_FILE_NAME: &str = "_last_checkpoint"; @@ -26,6 +27,7 @@ pub struct Snapshot { protocol: Protocol, schema: Schema, table_properties: TableProperties, + pub(crate) column_mapping_mode: ColumnMappingMode, } impl Drop for Snapshot { @@ -77,6 +79,8 @@ impl Snapshot { let (metadata, protocol) = log_segment.read_metadata(engine)?; let schema = metadata.schema()?; let table_properties = metadata.parse_table_properties()?; + let column_mapping_mode = column_mapping_mode(&protocol, &table_properties); + println!("column_mapping_mode: {:?}", column_mapping_mode); Ok(Self { table_root: location, log_segment, @@ -84,6 +88,7 @@ impl Snapshot { protocol, schema, table_properties, + column_mapping_mode }) } diff --git a/kernel/src/table_features/column_mapping.rs b/kernel/src/table_features/column_mapping.rs index 9365ccd4b..45ad6fd14 100644 --- a/kernel/src/table_features/column_mapping.rs +++ b/kernel/src/table_features/column_mapping.rs @@ -3,6 +3,9 @@ use std::str::FromStr; use serde::{Deserialize, Serialize}; +use super::ReaderFeatures; +use crate::actions::Protocol; +use crate::table_properties::TableProperties; use crate::{DeltaResult, Error}; /// Modes of column mapping a table can be in @@ -17,6 +20,23 @@ pub enum ColumnMappingMode { Name, } +/// Determine the column mapping mode for a table based on the [`Protocol`] and [`TableProperties`] +pub(crate) fn column_mapping_mode( + protocol: &Protocol, + table_properties: &TableProperties, +) -> ColumnMappingMode { + match table_properties.column_mapping_mode { + Some(mode) if protocol.min_reader_version() == 2 => mode, + Some(mode) + if protocol.min_reader_version() == 3 + && protocol.has_reader_feature(&ReaderFeatures::ColumnMapping) => + { + mode + } + _ => ColumnMappingMode::None, + } +} + impl TryFrom<&str> for ColumnMappingMode { type Error = Error; @@ -53,3 +73,44 @@ impl AsRef for ColumnMappingMode { } } } + +#[cfg(test)] +mod tests { + use super::*; + use std::collections::HashMap; + + #[test] + fn test_column_mapping_mode() { + let table_properties: HashMap<_, _> = + [("delta.columnMapping.mode".to_string(), "id".to_string())] + .into_iter() + .collect(); + let table_properties = TableProperties::new(table_properties.iter()).unwrap(); + + let protocol = Protocol::try_new(2, 5, None::>, None::>).unwrap(); + assert_eq!( + column_mapping_mode(&protocol, &table_properties), + ColumnMappingMode::Id + ); + + let empty_features = Some::<[String; 0]>([]); + let protocol = + Protocol::try_new(3, 7, empty_features.clone(), empty_features.clone()).unwrap(); + assert_eq!( + column_mapping_mode(&protocol, &table_properties), + ColumnMappingMode::None + ); + + let protocol = Protocol::try_new( + 3, + 7, + Some([ReaderFeatures::DeletionVectors]), + empty_features, + ) + .unwrap(); + assert_eq!( + column_mapping_mode(&protocol, &table_properties), + ColumnMappingMode::None + ); + } +} diff --git a/kernel/src/table_features/mod.rs b/kernel/src/table_features/mod.rs index fe44dbb61..fc9f1eec8 100644 --- a/kernel/src/table_features/mod.rs +++ b/kernel/src/table_features/mod.rs @@ -2,10 +2,10 @@ use std::collections::HashSet; use std::sync::LazyLock; use serde::{Deserialize, Serialize}; - -pub use column_mapping::ColumnMappingMode; use strum::{AsRefStr, Display as StrumDisplay, EnumString, VariantNames}; +pub use column_mapping::ColumnMappingMode; +pub(crate) use column_mapping::column_mapping_mode; mod column_mapping; /// Reader features communicate capabilities that must be implemented in order to correctly read a diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index e3c076c53..f96966277 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -198,10 +198,6 @@ impl TableProperties { ); TableProperties::deserialize(deserializer).map_err(|e| Error::invalid_table_properties(e)) } - - pub(crate) fn get_column_mapping_mode(&self) -> ColumnMappingMode { - self.column_mapping_mode.unwrap_or(ColumnMappingMode::None) - } } #[derive(Debug, PartialEq, Eq, Clone, Deserialize)] @@ -344,10 +340,7 @@ mod tests { ("delta.checkpointPolicy", "v2"), ("delta.enableRowTracking", "true"), ]; - let properties: HashMap<_, _> = properties - .into_iter() - .map(|(k, v)| (k.to_string(), v.to_string())) - .collect(); + let properties: HashMap<_, _> = properties.into_iter().collect(); let de = MapDeserializer::<_, de::value::Error>::new(properties.clone().into_iter()); let actual = TableProperties::deserialize(de).unwrap(); let expected = TableProperties { From af0809237c00e93e91196c3121ef6ea7bb583d50 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 18 Nov 2024 21:55:48 -0800 Subject: [PATCH 10/20] testing, errors, docs, cleanup --- kernel/src/error.rs | 5 + kernel/src/snapshot.rs | 1 - kernel/src/table_properties.rs | 1 + kernel/src/table_properties/deserialize.rs | 122 ++++++++++++++++----- 4 files changed, 98 insertions(+), 31 deletions(-) diff --git a/kernel/src/error.rs b/kernel/src/error.rs index 9edff6617..1ceb15ce6 100644 --- a/kernel/src/error.rs +++ b/kernel/src/error.rs @@ -7,6 +7,7 @@ use std::{ }; use crate::schema::DataType; +use crate::table_properties::ParseIntervalError; /// A [`std::result::Result`] that has the kernel [`Error`] as the error variant pub type DeltaResult = std::result::Result; @@ -183,6 +184,10 @@ pub enum Error { /// Some functionality is currently unsupported #[error("Unsupported: {0}")] Unsupported(String), + + /// Parsing error when attempting to deserizlize an interval + #[error(transparent)] + ParseIntervalError(#[from] ParseIntervalError), } // Convenience constructors for Error types that take a String argument diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 1de56170c..85d1b45b4 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -80,7 +80,6 @@ impl Snapshot { let schema = metadata.schema()?; let table_properties = metadata.parse_table_properties()?; let column_mapping_mode = column_mapping_mode(&protocol, &table_properties); - println!("column_mapping_mode: {:?}", column_mapping_mode); Ok(Self { table_root: location, log_segment, diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index f96966277..b0c14247a 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -22,6 +22,7 @@ use crate::{DeltaResult, Error}; mod deserialize; use deserialize::*; +pub use deserialize::ParseIntervalError; /// Default num index cols pub const DEFAULT_NUM_INDEX_COLS: i32 = 32; diff --git a/kernel/src/table_properties/deserialize.rs b/kernel/src/table_properties/deserialize.rs index b30d7d7fd..494fb8a9f 100644 --- a/kernel/src/table_properties/deserialize.rs +++ b/kernel/src/table_properties/deserialize.rs @@ -1,3 +1,8 @@ +//! For now we just use simple functions to deserialize table properties from strings. This allows +//! us to relatively simply implement the functionality described in the protocol and expose +//! 'simple' types to the user in the [`TableProperties`] struct. E.g. we can expose a `bool` +//! directly instead of a `BoolConfig` type that we implement `Deserialize` for. + use crate::expressions::ColumnName; use std::time::Duration; @@ -10,6 +15,7 @@ const SECONDS_PER_HOUR: u64 = 60 * SECONDS_PER_MINUTE; const SECONDS_PER_DAY: u64 = 24 * SECONDS_PER_HOUR; const SECONDS_PER_WEEK: u64 = 7 * SECONDS_PER_DAY; +/// Transparently pass through the option deserialization to the inner type. pub(crate) fn deserialize_option<'de, D, T>(deserializer: D) -> Result, D::Error> where D: Deserializer<'de>, @@ -18,19 +24,20 @@ where T::deserialize(deserializer).map(Some) } +/// Deserialize a string representing a positive integer into an `Option`. pub(crate) fn deserialize_pos_int<'de, D>(deserializer: D) -> Result, D::Error> where D: Deserializer<'de>, { let s: String = Deserialize::deserialize(deserializer)?; - let n: u64 = s.parse().unwrap(); // FIXME + let n: u64 = s.parse().map_err(de::Error::custom)?; if n == 0 { - panic!("FIXME"); - // return Err("something"); + return Err(de::Error::custom("expected a positive integer")); } Ok(Some(n)) } +/// Deserialize a string representing a boolean into an `Option`. pub(crate) fn deserialize_bool<'de, D>(deserializer: D) -> Result, D::Error> where D: Deserializer<'de>, @@ -43,6 +50,7 @@ where } } +/// Deserialize a comma-separated list of column names into an `Option>`. pub(crate) fn deserialize_column_names<'de, D>( deserializer: D, ) -> Result>, D::Error> @@ -55,6 +63,7 @@ where )) } +/// Deserialize an interval string of the form "interval 5 days" into an `Option`. pub(crate) fn deserialize_interval<'de, D>(deserializer: D) -> Result, D::Error> where D: Deserializer<'de>, @@ -63,21 +72,60 @@ where parse_interval(&s).map(Some).map_err(de::Error::custom) } -fn parse_interval(value: &str) -> Result { - let not_an_interval = || format!("'{value}' is not an interval"); +#[derive(thiserror::Error, Debug)] +pub enum ParseIntervalError { + /// The input string is not a valid interval + #[error("'{0}' is not an interval")] + NotAnInterval(String), + /// Couldn't parse the input string as an integer + #[error("Unable to parse '{0}' as an integer")] + ParseIntError(String), + /// Negative intervals aren't supported + #[error("Interval '{0}' cannot be negative")] + NegativeInterval(String), + /// Unsupported interval + #[error("Unsupported interval '{0}'")] + UnsupportedInterval(String), + /// Unknown unit + #[error("Unknown interval unit '{0}'")] + UnknownUnit(String), +} - if !value.starts_with("interval ") { - return Err(not_an_interval()); - } +/// This is effectively a simpler version of spark's `CalendarInterval` parser. See spark's +/// `stringToInterval`: +/// https://github.com/apache/spark/blob/5a57efdcee9e6569d8de4272bda258788cf349e3/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkIntervalUtils.scala#L134 +/// +/// Notably we don't support months nor years, nor do we support fractional values, and negative +/// intervals aren't supported. +/// +/// For now this is adapted from delta-rs' `parse_interval` function: +/// https://github.com/delta-io/delta-rs/blob/d4f18b3ae9d616e771b5d0e0fa498d0086fd91eb/crates/core/src/table/config.rs#L474 +/// +/// See issue delta-kernel-rs/#507 for details: https://github.com/delta-io/delta-kernel-rs/issues/507 +fn parse_interval(value: &str) -> Result { let mut it = value.split_whitespace(); - let _ = it.next(); // skip "interval" - let number = parse_int(it.next().ok_or_else(not_an_interval)?)?; - if number < 0 { - return Err(format!("interval '{value}' cannot be negative")); + if it.next() != Some("interval") { + return Err(ParseIntervalError::NotAnInterval(value.to_string())); } - let number = number as u64; + let number = it + .next() + .ok_or_else(|| ParseIntervalError::NotAnInterval(value.to_string()))?; + let number: i64 = number + .parse() + .map_err(|_| ParseIntervalError::ParseIntError(number.into()))?; - let duration = match it.next().ok_or_else(not_an_interval)? { + // TODO(zach): spark allows negative intervals, but we don't + if number < 0 { + return Err(ParseIntervalError::NegativeInterval(value.to_string())); + } + let number: u64 = number + .try_into() + .expect("interval is non-negative and was i64, so it fits in u64"); + + let duration = match it + .next() + .ok_or_else(|| ParseIntervalError::NotAnInterval(value.to_string()))? + { "nanosecond" | "nanoseconds" => Duration::from_nanos(number), "microsecond" | "microseconds" => Duration::from_micros(number), "millisecond" | "milliseconds" => Duration::from_millis(number), @@ -86,20 +134,17 @@ fn parse_interval(value: &str) -> Result { "hour" | "hours" => Duration::from_secs(number * SECONDS_PER_HOUR), "day" | "days" => Duration::from_secs(number * SECONDS_PER_DAY), "week" | "weeks" => Duration::from_secs(number * SECONDS_PER_WEEK), + unit @ "month" | unit @ "months" => { + return Err(ParseIntervalError::UnsupportedInterval(unit.to_string())); + } unit => { - return Err(format!("Unknown unit '{unit}'")); + return Err(ParseIntervalError::UnknownUnit(unit.to_string())); } }; Ok(duration) } -fn parse_int(value: &str) -> Result { - value - .parse() - .map_err(|e| format!("Cannot parse '{value}' as integer: {e}")) -} - #[cfg(test)] mod tests { use super::*; @@ -190,33 +235,50 @@ mod tests { #[test] fn test_invalid_parse_interval() { assert_eq!( - parse_interval("whatever").err().unwrap(), + parse_interval("whatever").err().unwrap().to_string(), "'whatever' is not an interval".to_string() ); assert_eq!( - parse_interval("interval").err().unwrap(), + parse_interval("interval").err().unwrap().to_string(), "'interval' is not an interval".to_string() ); assert_eq!( - parse_interval("interval 2").err().unwrap(), + parse_interval("interval 2").err().unwrap().to_string(), "'interval 2' is not an interval".to_string() ); assert_eq!( - parse_interval("interval 2 years").err().unwrap(), - "Unknown unit 'years'".to_string() + parse_interval("interval 2 months") + .err() + .unwrap() + .to_string(), + "Unsupported interval 'months'".to_string() + ); + + assert_eq!( + parse_interval("interval 2 years") + .err() + .unwrap() + .to_string(), + "Unknown interval unit 'years'".to_string() ); assert_eq!( - parse_interval("interval two years").err().unwrap(), - "Cannot parse 'two' as integer: invalid digit found in string".to_string() + parse_interval("interval two years") + .err() + .unwrap() + .to_string(), + "Unable to parse 'two' as an integer".to_string() ); assert_eq!( - parse_interval("interval -25 hours").err().unwrap(), - "interval 'interval -25 hours' cannot be negative".to_string() + parse_interval("interval -25 hours") + .err() + .unwrap() + .to_string(), + "Interval 'interval -25 hours' cannot be negative".to_string() ); } } From 458779452736e2f9d035289b2f366c3940e2cdbb Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 18 Nov 2024 22:19:18 -0800 Subject: [PATCH 11/20] cleanup --- ffi/src/lib.rs | 2 ++ kernel/src/actions/mod.rs | 2 +- kernel/src/lib.rs | 2 +- kernel/src/snapshot.rs | 4 ++-- kernel/src/table_features/mod.rs | 2 +- kernel/src/table_properties.rs | 6 +++--- 6 files changed, 10 insertions(+), 8 deletions(-) diff --git a/ffi/src/lib.rs b/ffi/src/lib.rs index 9659fc3e7..104f43f54 100644 --- a/ffi/src/lib.rs +++ b/ffi/src/lib.rs @@ -384,6 +384,7 @@ pub enum KernelError { MissingCommitInfo, InvalidTableProperties, UnsupportedError, + ParseIntervalError, } impl From for KernelError { @@ -437,6 +438,7 @@ impl From for KernelError { Error::MissingCommitInfo => KernelError::MissingCommitInfo, Error::InvalidTableProperties(_) => KernelError::InvalidTableProperties, Error::Unsupported(_) => KernelError::UnsupportedError, + Error::ParseIntervalError(_) => KernelError::ParseIntervalError, } } } diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 0cd19312d..b6dfd8f53 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -14,9 +14,9 @@ use crate::schema::{SchemaRef, StructType}; use crate::table_features::{ ReaderFeatures, WriterFeatures, SUPPORTED_READER_FEATURES, SUPPORTED_WRITER_FEATURES, }; +use crate::table_properties::TableProperties; use crate::utils::require; use crate::{DeltaResult, EngineData, Error}; -use crate::table_properties::TableProperties; use visitors::{AddVisitor, MetadataVisitor, ProtocolVisitor}; use delta_kernel_derive::Schema; diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index 0d9dfa860..28335c801 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -63,11 +63,11 @@ pub mod actions; pub mod engine_data; pub mod error; pub mod expressions; -pub mod table_features; pub mod scan; pub mod schema; pub mod snapshot; pub mod table; +pub mod table_features; pub mod table_properties; pub mod transaction; diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 85d1b45b4..9ed36b28f 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -10,8 +10,8 @@ use crate::actions::{Metadata, Protocol}; use crate::log_segment::LogSegment; use crate::scan::ScanBuilder; use crate::schema::Schema; +use crate::table_features::{column_mapping_mode, ColumnMappingMode}; use crate::table_properties::TableProperties; -use crate::table_features::{ColumnMappingMode, column_mapping_mode}; use crate::{DeltaResult, Engine, Error, FileSystemClient, Version}; const LAST_CHECKPOINT_FILE_NAME: &str = "_last_checkpoint"; @@ -87,7 +87,7 @@ impl Snapshot { protocol, schema, table_properties, - column_mapping_mode + column_mapping_mode, }) } diff --git a/kernel/src/table_features/mod.rs b/kernel/src/table_features/mod.rs index fc9f1eec8..72d8032ad 100644 --- a/kernel/src/table_features/mod.rs +++ b/kernel/src/table_features/mod.rs @@ -4,8 +4,8 @@ use std::sync::LazyLock; use serde::{Deserialize, Serialize}; use strum::{AsRefStr, Display as StrumDisplay, EnumString, VariantNames}; -pub use column_mapping::ColumnMappingMode; pub(crate) use column_mapping::column_mapping_mode; +pub use column_mapping::ColumnMappingMode; mod column_mapping; /// Reader features communicate capabilities that must be implemented in order to correctly read a diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index b0c14247a..8d939ada0 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -21,8 +21,8 @@ use crate::table_features::ColumnMappingMode; use crate::{DeltaResult, Error}; mod deserialize; -use deserialize::*; pub use deserialize::ParseIntervalError; +use deserialize::*; /// Default num index cols pub const DEFAULT_NUM_INDEX_COLS: i32 = 32; @@ -86,7 +86,7 @@ pub struct TableProperties { /// A comma-separated list of column names on which Delta Lake collects statistics to enhance /// data skipping functionality. This property takes precedence over - /// [DataSkippingNumIndexedCols](DeltaConfigKey::DataSkippingNumIndexedCols). + /// `delta.dataSkippingNumIndexedCols`. #[serde(rename = "delta.dataSkippingStatsColumns")] #[serde(deserialize_with = "deserialize_column_names")] pub data_skipping_stats_columns: Option>, @@ -197,7 +197,7 @@ impl TableProperties { let deserializer = MapDeserializer::<_, de::value::Error>::new( items.map(|(k, v)| (k.as_str(), v.as_str())), ); - TableProperties::deserialize(deserializer).map_err(|e| Error::invalid_table_properties(e)) + TableProperties::deserialize(deserializer).map_err(Error::invalid_table_properties) } } From 1e7d2868a4ee44062b82f3ba130e9967ea0065a6 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Wed, 20 Nov 2024 10:09:23 -0800 Subject: [PATCH 12/20] fix skipping dat test --- acceptance/src/data.rs | 3 ++- acceptance/tests/dat_reader.rs | 5 ----- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/acceptance/src/data.rs b/acceptance/src/data.rs index a6a50ef77..e341d2e52 100644 --- a/acceptance/src/data.rs +++ b/acceptance/src/data.rs @@ -61,7 +61,8 @@ pub fn sort_record_batch(batch: RecordBatch) -> DeltaResult { Ok(RecordBatch::try_new(batch.schema(), columns)?) } -static SKIPPED_TESTS: &[&str; 0] = &[]; +// TODO(zach): skip iceberg_compat_v1 test until DAT is fixed +static SKIPPED_TESTS: &[&str; 1] = &["iceberg_compat_v1"]; // Ensure that two schema have the same field names, and dict_id/ordering. // We ignore: diff --git a/acceptance/tests/dat_reader.rs b/acceptance/tests/dat_reader.rs index 321aa26b3..ee0e1513f 100644 --- a/acceptance/tests/dat_reader.rs +++ b/acceptance/tests/dat_reader.rs @@ -12,11 +12,6 @@ fn reader_test(path: &Path) -> datatest_stable::Result<()> { path.parent().unwrap().to_str().unwrap() ); - // TODO(zach): skip iceberg_compat_v1 test until DAT is fixed - if path.ends_with("iceberg_compat_v1/test_case_info.json") { - return Ok(()); - } - tokio::runtime::Builder::new_current_thread() .enable_all() .build()? From bd9ac7aa23812b62fc0c1c3856a13f9257b47425 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 21 Nov 2024 14:01:24 -0800 Subject: [PATCH 13/20] address feedback, cleanup --- kernel/src/table_properties.rs | 25 +++------- kernel/src/table_properties/deserialize.rs | 56 ++++++++++------------ 2 files changed, 32 insertions(+), 49 deletions(-) diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index 8d939ada0..4d493098b 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -55,7 +55,7 @@ pub struct TableProperties { /// E.g. if checkpoint interval = 10, then a checkpoint should be written every 10 commits. #[serde(rename = "delta.checkpointInterval")] #[serde(deserialize_with = "deserialize_pos_int")] - pub checkpoint_interval: Option, + pub checkpoint_interval: Option, /// true for Delta Lake to write file statistics in checkpoints in JSON format for the stats column. #[serde(rename = "delta.checkpoint.writeStatsAsJson")] @@ -152,7 +152,7 @@ pub struct TableProperties { /// generates for random prefixes. #[serde(rename = "delta.randomPrefixLength")] #[serde(deserialize_with = "deserialize_pos_int")] - pub random_prefix_length: Option, + pub random_prefix_length: Option, /// The shortest duration within which new snapshots will retain transaction identifiers (for /// example, SetTransactions). When a new snapshot sees a transaction identifier older than or @@ -166,7 +166,7 @@ pub struct TableProperties { /// (bytes) or 100mb. #[serde(rename = "delta.targetFileSize")] #[serde(deserialize_with = "deserialize_pos_int")] - pub target_file_size: Option, + pub target_file_size: Option, /// The target file size in bytes or higher units for file tuning. For example, 104857600 /// (bytes) or 100mb. @@ -231,7 +231,7 @@ impl TryFrom for DataSkippingNumIndexedCols { } /// The isolation level applied during transaction -#[derive(Deserialize, Debug, Copy, Clone, PartialEq, Eq)] +#[derive(Deserialize, Debug, Default, Copy, Clone, PartialEq, Eq)] #[serde(rename_all = "camelCase")] pub enum IsolationLevel { /// The strongest isolation level. It ensures that committed write operations @@ -239,6 +239,7 @@ pub enum IsolationLevel { /// exists a serial sequence of executing them one-at-a-time that generates /// the same outcome as that seen in the table. For the write operations, /// the serial sequence is exactly the same as that seen in the table’s history. + #[default] Serializable, /// A weaker isolation level than Serializable. It ensures only that the write @@ -254,29 +255,17 @@ pub enum IsolationLevel { SnapshotIsolation, } -// Delta-Spark default isolation level is Serializable -impl Default for IsolationLevel { - fn default() -> Self { - Self::Serializable - } -} - /// The checkpoint policy applied when writing checkpoints -#[derive(Deserialize, Debug, Clone, PartialEq, Eq)] +#[derive(Deserialize, Debug, Default, Clone, PartialEq, Eq)] #[serde(rename_all = "camelCase")] pub enum CheckpointPolicy { /// classic Delta Lake checkpoints + #[default] Classic, /// v2 checkpoints V2, } -impl Default for CheckpointPolicy { - fn default() -> Self { - Self::Classic - } -} - #[cfg(test)] mod tests { use super::*; diff --git a/kernel/src/table_properties/deserialize.rs b/kernel/src/table_properties/deserialize.rs index 494fb8a9f..868a5a9b1 100644 --- a/kernel/src/table_properties/deserialize.rs +++ b/kernel/src/table_properties/deserialize.rs @@ -4,6 +4,7 @@ //! directly instead of a `BoolConfig` type that we implement `Deserialize` for. use crate::expressions::ColumnName; +use crate::utils::require; use std::time::Duration; @@ -25,23 +26,20 @@ where } /// Deserialize a string representing a positive integer into an `Option`. -pub(crate) fn deserialize_pos_int<'de, D>(deserializer: D) -> Result, D::Error> -where - D: Deserializer<'de>, -{ +pub(crate) fn deserialize_pos_int<'de, D: Deserializer<'de>>( + deserializer: D, +) -> Result, D::Error> { let s: String = Deserialize::deserialize(deserializer)?; - let n: u64 = s.parse().map_err(de::Error::custom)?; - if n == 0 { - return Err(de::Error::custom("expected a positive integer")); - } + // parse to i64 (then check n > 0) since java doesn't even allow u64 + let n: i64 = s.parse().map_err(de::Error::custom)?; + require!(n > 0, de::Error::custom("expected a positive integer")); Ok(Some(n)) } /// Deserialize a string representing a boolean into an `Option`. -pub(crate) fn deserialize_bool<'de, D>(deserializer: D) -> Result, D::Error> -where - D: Deserializer<'de>, -{ +pub(crate) fn deserialize_bool<'de, D: Deserializer<'de>>( + deserializer: D, +) -> Result, D::Error> { let s: String = Deserialize::deserialize(deserializer)?; match s.as_str() { "true" => Ok(Some(true)), @@ -51,23 +49,18 @@ where } /// Deserialize a comma-separated list of column names into an `Option>`. -pub(crate) fn deserialize_column_names<'de, D>( +pub(crate) fn deserialize_column_names<'de, D: Deserializer<'de>>( deserializer: D, -) -> Result>, D::Error> -where - D: Deserializer<'de>, -{ +) -> Result>, D::Error> { let s: String = Deserialize::deserialize(deserializer)?; - Ok(Some( - ColumnName::parse_column_name_list(&s).map_err(de::Error::custom)?, - )) + let column_names = ColumnName::parse_column_name_list(&s).map_err(de::Error::custom)?; + Ok(Some(column_names)) } /// Deserialize an interval string of the form "interval 5 days" into an `Option`. -pub(crate) fn deserialize_interval<'de, D>(deserializer: D) -> Result, D::Error> -where - D: Deserializer<'de>, -{ +pub(crate) fn deserialize_interval<'de, D: Deserializer<'de>>( + deserializer: D, +) -> Result, D::Error> { let s = String::deserialize(deserializer)?; parse_interval(&s).map(Some).map_err(de::Error::custom) } @@ -115,12 +108,13 @@ fn parse_interval(value: &str) -> Result { .map_err(|_| ParseIntervalError::ParseIntError(number.into()))?; // TODO(zach): spark allows negative intervals, but we don't - if number < 0 { - return Err(ParseIntervalError::NegativeInterval(value.to_string())); - } - let number: u64 = number - .try_into() - .expect("interval is non-negative and was i64, so it fits in u64"); + require!( + number >= 0, + ParseIntervalError::NegativeInterval(value.to_string()) + ); + + // convert to u64 since Duration expects it + let number = number as u64; // non-negative i64 => always safe let duration = match it .next() @@ -134,7 +128,7 @@ fn parse_interval(value: &str) -> Result { "hour" | "hours" => Duration::from_secs(number * SECONDS_PER_HOUR), "day" | "days" => Duration::from_secs(number * SECONDS_PER_DAY), "week" | "weeks" => Duration::from_secs(number * SECONDS_PER_WEEK), - unit @ "month" | unit @ "months" => { + unit @ ("month" | "months") => { return Err(ParseIntervalError::UnsupportedInterval(unit.to_string())); } unit => { From ff786237d94fb8137a66910f05536245efaf9f73 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Thu, 21 Nov 2024 16:21:23 -0800 Subject: [PATCH 14/20] remove unused const --- kernel/src/table_properties.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index 4d493098b..6a50d4f39 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -24,9 +24,6 @@ mod deserialize; pub use deserialize::ParseIntervalError; use deserialize::*; -/// Default num index cols -pub const DEFAULT_NUM_INDEX_COLS: i32 = 32; - /// Delta table properties. These are parsed from the 'configuration' map in the most recent /// 'Metadata' action of a table. #[derive(Deserialize, Debug, Clone, Eq, PartialEq, Default)] From b667a15ac5917cf78c5a08f65ed4a7e723afd6e6 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 22 Nov 2024 15:02:44 -0800 Subject: [PATCH 15/20] no more serde --- kernel/src/actions/mod.rs | 6 +- kernel/src/snapshot.rs | 2 +- kernel/src/table_features/column_mapping.rs | 51 +------ kernel/src/table_properties.rs | 123 ++++----------- kernel/src/table_properties/deserialize.rs | 158 ++++++++++++++------ 5 files changed, 154 insertions(+), 186 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 4a2c3ffae..a95655032 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -125,8 +125,10 @@ impl Metadata { } /// Parse the metadata configuration HashMap into a TableProperties struct. - pub fn parse_table_properties(&self) -> DeltaResult { - TableProperties::new(self.configuration.iter()) + /// Note that this method doesn't return a result since any items that fail to parse are + /// propagated through to the `TableProperties.unknown_properties` field. + pub fn parse_table_properties(&self) -> TableProperties { + TableProperties::from(self.configuration.iter()) } } diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index f256202c7..a8fef17e8 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -83,7 +83,7 @@ impl Snapshot { protocol.ensure_read_supported()?; let schema = metadata.schema()?; - let table_properties = metadata.parse_table_properties()?; + let table_properties = metadata.parse_table_properties(); let column_mapping_mode = column_mapping_mode(&protocol, &table_properties); Ok(Self { table_root: location, diff --git a/kernel/src/table_features/column_mapping.rs b/kernel/src/table_features/column_mapping.rs index 45ad6fd14..b62e200cc 100644 --- a/kernel/src/table_features/column_mapping.rs +++ b/kernel/src/table_features/column_mapping.rs @@ -1,15 +1,14 @@ //! Code to handle column mapping, including modes and schema transforms -use std::str::FromStr; - -use serde::{Deserialize, Serialize}; - use super::ReaderFeatures; use crate::actions::Protocol; use crate::table_properties::TableProperties; -use crate::{DeltaResult, Error}; + +use serde::{Deserialize, Serialize}; +use strum::EnumString; /// Modes of column mapping a table can be in -#[derive(Serialize, Deserialize, Debug, Copy, Clone, PartialEq, Eq)] +#[derive(Debug, Default, EnumString, Serialize, Deserialize, Copy, Clone, PartialEq, Eq)] +#[strum(serialize_all = "camelCase")] #[serde(rename_all = "camelCase")] pub enum ColumnMappingMode { /// No column mapping is applied @@ -17,6 +16,7 @@ pub enum ColumnMappingMode { /// Columns are mapped by their field_id in parquet Id, /// Columns are mapped to a physical name + #[default] Name, } @@ -37,43 +37,6 @@ pub(crate) fn column_mapping_mode( } } -impl TryFrom<&str> for ColumnMappingMode { - type Error = Error; - - fn try_from(s: &str) -> DeltaResult { - match s.to_ascii_lowercase().as_str() { - "none" => Ok(Self::None), - "id" => Ok(Self::Id), - "name" => Ok(Self::Name), - _ => Err(Error::invalid_column_mapping_mode(s)), - } - } -} - -impl FromStr for ColumnMappingMode { - type Err = Error; - - fn from_str(s: &str) -> Result { - s.try_into() - } -} - -impl Default for ColumnMappingMode { - fn default() -> Self { - Self::None - } -} - -impl AsRef for ColumnMappingMode { - fn as_ref(&self) -> &str { - match self { - Self::None => "none", - Self::Id => "id", - Self::Name => "name", - } - } -} - #[cfg(test)] mod tests { use super::*; @@ -85,7 +48,7 @@ mod tests { [("delta.columnMapping.mode".to_string(), "id".to_string())] .into_iter() .collect(); - let table_properties = TableProperties::new(table_properties.iter()).unwrap(); + let table_properties = TableProperties::from(table_properties.iter()); let protocol = Protocol::try_new(2, 5, None::>, None::>).unwrap(); assert_eq!( diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index 6a50d4f39..98cf2bbfe 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -13,62 +13,47 @@ use std::collections::HashMap; use std::time::Duration; -use serde::de::{self, value::MapDeserializer}; -use serde::Deserialize; - use crate::expressions::ColumnName; use crate::table_features::ColumnMappingMode; -use crate::{DeltaResult, Error}; +use crate::Error; + +use strum::EnumString; mod deserialize; pub use deserialize::ParseIntervalError; -use deserialize::*; /// Delta table properties. These are parsed from the 'configuration' map in the most recent /// 'Metadata' action of a table. -#[derive(Deserialize, Debug, Clone, Eq, PartialEq, Default)] -#[serde(default)] +/// +/// Reference: https://github.com/delta-io/delta/blob/master/spark/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala +#[derive(Debug, Clone, Eq, PartialEq, Default)] pub struct TableProperties { /// true for this Delta table to be append-only. If append-only, existing records cannot be /// deleted, and existing values cannot be updated. See [append-only tables] in the protocol. /// /// [append-only tables]: https://github.com/delta-io/delta/blob/master/PROTOCOL.md#append-only-tables - #[serde(rename = "delta.appendOnly")] - #[serde(deserialize_with = "deserialize_bool")] pub append_only: Option, /// true for Delta Lake to automatically optimize the layout of the files for this Delta table. - #[serde(rename = "delta.autoOptimize.autoCompact")] - #[serde(deserialize_with = "deserialize_bool")] pub auto_compact: Option, /// true for Delta Lake to automatically optimize the layout of the files for this Delta table /// during writes. - #[serde(rename = "delta.autoOptimize.optimizeWrite")] - #[serde(deserialize_with = "deserialize_bool")] pub optimize_write: Option, /// Interval (expressed as number of commits) after which a new checkpoint should be created. /// E.g. if checkpoint interval = 10, then a checkpoint should be written every 10 commits. - #[serde(rename = "delta.checkpointInterval")] - #[serde(deserialize_with = "deserialize_pos_int")] pub checkpoint_interval: Option, /// true for Delta Lake to write file statistics in checkpoints in JSON format for the stats column. - #[serde(rename = "delta.checkpoint.writeStatsAsJson")] - #[serde(deserialize_with = "deserialize_bool")] pub checkpoint_write_stats_as_json: Option, /// true for Delta Lake to write file statistics to checkpoints in struct format for the /// stats_parsed column and to write partition values as a struct for partitionValues_parsed. - #[serde(rename = "delta.checkpoint.writeStatsAsStruct")] - #[serde(deserialize_with = "deserialize_bool")] pub checkpoint_write_stats_as_struct: Option, /// Whether column mapping is enabled for Delta table columns and the corresponding /// Parquet columns that use different names. - #[serde(rename = "delta.columnMapping.mode")] - #[serde(deserialize_with = "deserialize_option")] pub column_mapping_mode: Option, /// The number of columns for Delta Lake to collect statistics about for data skipping. @@ -77,15 +62,11 @@ pub struct TableProperties { /// of the Delta table. Specifically, it changes the behavior of future statistics collection /// (such as during appends and optimizations) as well as data skipping (such as ignoring column /// statistics beyond this number, even when such statistics exist). - #[serde(rename = "delta.dataSkippingNumIndexedCols")] - #[serde(deserialize_with = "deserialize_option")] pub data_skipping_num_indexed_cols: Option, /// A comma-separated list of column names on which Delta Lake collects statistics to enhance /// data skipping functionality. This property takes precedence over /// `delta.dataSkippingNumIndexedCols`. - #[serde(rename = "delta.dataSkippingStatsColumns")] - #[serde(deserialize_with = "deserialize_column_names")] pub data_skipping_stats_columns: Option>, /// The shortest duration for Delta Lake to keep logically deleted data files before deleting @@ -99,26 +80,18 @@ pub struct TableProperties { /// * If you run a streaming query that reads from the table, that query does not stop for /// longer than this value. Otherwise, the query may not be able to restart, as it must still /// read old files. - #[serde(rename = "delta.deletedFileRetentionDuration")] - #[serde(deserialize_with = "deserialize_interval")] pub deleted_file_retention_duration: Option, /// true to enable change data feed. - #[serde(rename = "delta.enableChangeDataFeed")] - #[serde(deserialize_with = "deserialize_bool")] pub enable_change_data_feed: Option, /// true to enable deletion vectors and predictive I/O for updates. - #[serde(rename = "delta.enableDeletionVectors")] - #[serde(deserialize_with = "deserialize_bool")] pub enable_deletion_vectors: Option, /// The degree to which a transaction must be isolated from modifications made by concurrent /// transactions. /// /// Valid values are `Serializable` and `WriteSerializable`. - #[serde(rename = "delta.isolationLevel")] - #[serde(deserialize_with = "deserialize_option")] pub isolation_level: Option, /// How long the history for a Delta table is kept. @@ -128,99 +101,63 @@ pub struct TableProperties { /// entries are retained. This should not impact performance as operations against the log are /// constant time. Operations on history are parallel but will become more expensive as the log /// size increases. - #[serde(rename = "delta.logRetentionDuration")] - #[serde(deserialize_with = "deserialize_interval")] pub log_retention_duration: Option, - /// TODO docs - #[serde(rename = "delta.enableExpiredLogCleanup")] - #[serde(deserialize_with = "deserialize_bool")] + /// Whether to clean up expired checkpoints/commits in the delta log. pub enable_expired_log_cleanup: Option, /// true for Delta to generate a random prefix for a file path instead of partition information. /// /// For example, this may improve Amazon S3 performance when Delta Lake needs to send very high /// volumes of Amazon S3 calls to better partition across S3 servers. - #[serde(rename = "delta.randomizeFilePrefixes")] - #[serde(deserialize_with = "deserialize_bool")] pub randomize_file_prefixes: Option, /// When delta.randomizeFilePrefixes is set to true, the number of characters that Delta /// generates for random prefixes. - #[serde(rename = "delta.randomPrefixLength")] - #[serde(deserialize_with = "deserialize_pos_int")] pub random_prefix_length: Option, /// The shortest duration within which new snapshots will retain transaction identifiers (for /// example, SetTransactions). When a new snapshot sees a transaction identifier older than or /// equal to the duration specified by this property, the snapshot considers it expired and /// ignores it. The SetTransaction identifier is used when making the writes idempotent. - #[serde(rename = "delta.setTransactionRetentionDuration")] - #[serde(deserialize_with = "deserialize_interval")] pub set_transaction_retention_duration: Option, /// The target file size in bytes or higher units for file tuning. For example, 104857600 /// (bytes) or 100mb. - #[serde(rename = "delta.targetFileSize")] - #[serde(deserialize_with = "deserialize_pos_int")] pub target_file_size: Option, /// The target file size in bytes or higher units for file tuning. For example, 104857600 /// (bytes) or 100mb. - #[serde(rename = "delta.tuneFileSizesForRewrites")] - #[serde(deserialize_with = "deserialize_bool")] pub tune_file_sizes_for_rewrites: Option, /// 'classic' for classic Delta Lake checkpoints. 'v2' for v2 checkpoints. - #[serde(rename = "delta.checkpointPolicy")] - #[serde(deserialize_with = "deserialize_option")] pub checkpoint_policy: Option, /// whether to enable row tracking during writes. - #[serde(rename = "delta.enableRowTracking")] - #[serde(deserialize_with = "deserialize_bool")] pub enable_row_tracking: Option, /// any unrecognized properties are passed through and ignored by the parser - #[serde(flatten)] pub unknown_properties: HashMap, } -impl TableProperties { - pub(crate) fn new<'a, I>(items: I) -> DeltaResult - where - I: Iterator + 'a, - { - let deserializer = MapDeserializer::<_, de::value::Error>::new( - items.map(|(k, v)| (k.as_str(), v.as_str())), - ); - TableProperties::deserialize(deserializer).map_err(Error::invalid_table_properties) - } -} - -#[derive(Debug, PartialEq, Eq, Clone, Deserialize)] -#[serde(try_from = "String")] +#[derive(Debug, Copy, Clone, PartialEq, Eq)] pub enum DataSkippingNumIndexedCols { AllColumns, NumColumns(u64), } -impl TryFrom for DataSkippingNumIndexedCols { +impl TryFrom<&str> for DataSkippingNumIndexedCols { type Error = Error; - fn try_from(value: String) -> Result { + fn try_from(value: &str) -> Result { let num: i64 = value.parse().map_err(|_| { - Error::invalid_table_properties( - "couldn't parse DataSkippingNumIndexedCols to an integer", - ) + Error::generic("couldn't parse DataSkippingNumIndexedCols to an integer") })?; match num { -1 => Ok(DataSkippingNumIndexedCols::AllColumns), x => Ok(DataSkippingNumIndexedCols::NumColumns( x.try_into().map_err(|_| { - Error::invalid_table_properties( - "couldn't parse DataSkippingNumIndexedCols to positive integer", - ) + Error::generic("couldn't parse DataSkippingNumIndexedCols to positive integer") })?, )), } @@ -228,8 +165,8 @@ impl TryFrom for DataSkippingNumIndexedCols { } /// The isolation level applied during transaction -#[derive(Deserialize, Debug, Default, Copy, Clone, PartialEq, Eq)] -#[serde(rename_all = "camelCase")] +#[derive(Debug, EnumString, Default, Copy, Clone, PartialEq, Eq)] +#[strum(serialize_all = "camelCase")] pub enum IsolationLevel { /// The strongest isolation level. It ensures that committed write operations /// and all reads are Serializable. Operations are allowed as long as there @@ -253,8 +190,8 @@ pub enum IsolationLevel { } /// The checkpoint policy applied when writing checkpoints -#[derive(Deserialize, Debug, Default, Clone, PartialEq, Eq)] -#[serde(rename_all = "camelCase")] +#[derive(Debug, EnumString, Default, Clone, PartialEq, Eq)] +#[strum(serialize_all = "camelCase")] pub enum CheckpointPolicy { /// classic Delta Lake checkpoints #[default] @@ -266,24 +203,29 @@ pub enum CheckpointPolicy { #[cfg(test)] mod tests { use super::*; + use crate::expressions::column_name; use std::collections::HashMap; #[test] - fn fail_known_keys() { + fn known_key_unknown_val() { let properties = HashMap::from([("delta.appendOnly".to_string(), "wack".to_string())]); - let de = MapDeserializer::<_, de::value::Error>::new(properties.clone().into_iter()); - assert!(TableProperties::deserialize(de).is_err()); + let table_properties = TableProperties::from(properties.iter()); + let unknown_properties = + HashMap::from([("delta.appendOnly".to_string(), "wack".to_string())]); + let expected = TableProperties { + unknown_properties, + ..Default::default() + }; + assert_eq!(table_properties, expected); } #[test] fn allow_unknown_keys() { - let properties = - HashMap::from([("some_random_unknown_key".to_string(), "test".to_string())]); - let de = MapDeserializer::<_, de::value::Error>::new(properties.clone().into_iter()); - let actual = TableProperties::deserialize(de).unwrap(); + let properties = [("some_random_unknown_key".to_string(), "test".to_string())]; + let actual = TableProperties::from(properties.clone().into_iter()); let expected = TableProperties { - unknown_properties: properties, + unknown_properties: HashMap::from(properties), ..Default::default() }; assert_eq!(actual, expected); @@ -292,8 +234,7 @@ mod tests { #[test] fn test_empty_table_properties() { let map: HashMap = HashMap::new(); - let de = MapDeserializer::<_, de::value::Error>::new(map.into_iter()); - let actual = TableProperties::deserialize(de).unwrap(); + let actual = TableProperties::from(map.iter()); let default_table_properties = TableProperties::default(); assert_eq!(actual, default_table_properties); } @@ -327,9 +268,7 @@ mod tests { ("delta.checkpointPolicy", "v2"), ("delta.enableRowTracking", "true"), ]; - let properties: HashMap<_, _> = properties.into_iter().collect(); - let de = MapDeserializer::<_, de::value::Error>::new(properties.clone().into_iter()); - let actual = TableProperties::deserialize(de).unwrap(); + let actual = TableProperties::from(properties.into_iter()); let expected = TableProperties { append_only: Some(true), optimize_write: Some(true), diff --git a/kernel/src/table_properties/deserialize.rs b/kernel/src/table_properties/deserialize.rs index 868a5a9b1..0ed28ad03 100644 --- a/kernel/src/table_properties/deserialize.rs +++ b/kernel/src/table_properties/deserialize.rs @@ -2,67 +2,131 @@ //! us to relatively simply implement the functionality described in the protocol and expose //! 'simple' types to the user in the [`TableProperties`] struct. E.g. we can expose a `bool` //! directly instead of a `BoolConfig` type that we implement `Deserialize` for. +use std::time::Duration; +use super::*; use crate::expressions::ColumnName; +use crate::table_features::ColumnMappingMode; use crate::utils::require; -use std::time::Duration; - -use serde::de::{self, Deserializer}; -use serde::Deserialize; - const SECONDS_PER_MINUTE: u64 = 60; const SECONDS_PER_HOUR: u64 = 60 * SECONDS_PER_MINUTE; const SECONDS_PER_DAY: u64 = 24 * SECONDS_PER_HOUR; const SECONDS_PER_WEEK: u64 = 7 * SECONDS_PER_DAY; -/// Transparently pass through the option deserialization to the inner type. -pub(crate) fn deserialize_option<'de, D, T>(deserializer: D) -> Result, D::Error> +impl From for TableProperties where - D: Deserializer<'de>, - T: Deserialize<'de>, + I: IntoIterator, + K: AsRef, + V: AsRef, { - T::deserialize(deserializer).map(Some) + fn from(unparsed: I) -> Self { + let mut props = TableProperties::default(); + for (k, v) in unparsed { + let parsed = + match k.as_ref() { + "delta.appendOnly" => { + parse_bool(v.as_ref()).map(|val| props.append_only = Some(val)) + } + "delta.autoOptimize.autoCompact" => { + parse_bool(v.as_ref()).map(|val| props.auto_compact = Some(val)) + } + "delta.autoOptimize.optimizeWrite" => { + parse_bool(v.as_ref()).map(|val| props.optimize_write = Some(val)) + } + "delta.checkpointInterval" => parse_positive_int(v.as_ref()) + .map(|val| props.checkpoint_interval = Some(val)), + "delta.checkpoint.writeStatsAsJson" => parse_bool(v.as_ref()) + .map(|val| props.checkpoint_write_stats_as_json = Some(val)), + "delta.checkpoint.writeStatsAsStruct" => parse_bool(v.as_ref()) + .map(|val| props.checkpoint_write_stats_as_struct = Some(val)), + "delta.columnMapping.mode" => ColumnMappingMode::try_from(v.as_ref()) + .map(|val| props.column_mapping_mode = Some(val)) + .ok(), + "delta.dataSkippingNumIndexedCols" => { + DataSkippingNumIndexedCols::try_from(v.as_ref()) + .map(|val| props.data_skipping_num_indexed_cols = Some(val)) + .ok() + } + "delta.dataSkippingStatsColumns" => parse_column_names(v.as_ref()) + .map(|val| props.data_skipping_stats_columns = Some(val)), + "delta.deletedFileRetentionDuration" => parse_interval(v.as_ref()) + .map(|val| props.deleted_file_retention_duration = Some(val)), + "delta.enableChangeDataFeed" => { + parse_bool(v.as_ref()).map(|val| props.enable_change_data_feed = Some(val)) + } + "delta.enableDeletionVectors" => { + parse_bool(v.as_ref()).map(|val| props.enable_deletion_vectors = Some(val)) + } + "delta.isolationLevel" => IsolationLevel::try_from(v.as_ref()) + .map(|val| props.isolation_level = Some(val)) + .ok(), + "delta.logRetentionDuration" => parse_interval(v.as_ref()) + .map(|val| props.log_retention_duration = Some(val)), + "delta.enableExpiredLogCleanup" => parse_bool(v.as_ref()) + .map(|val| props.enable_expired_log_cleanup = Some(val)), + "delta.randomizeFilePrefixes" => { + parse_bool(v.as_ref()).map(|val| props.randomize_file_prefixes = Some(val)) + } + "delta.randomPrefixLength" => parse_positive_int(v.as_ref()) + .map(|val| props.random_prefix_length = Some(val)), + "delta.setTransactionRetentionDuration" => parse_interval(v.as_ref()) + .map(|val| props.set_transaction_retention_duration = Some(val)), + "delta.targetFileSize" => { + parse_positive_int(v.as_ref()).map(|val| props.target_file_size = Some(val)) + } + "delta.tuneFileSizesForRewrites" => parse_bool(v.as_ref()) + .map(|val| props.tune_file_sizes_for_rewrites = Some(val)), + "delta.checkpointPolicy" => CheckpointPolicy::try_from(v.as_ref()) + .map(|val| props.checkpoint_policy = Some(val)) + .ok(), + "delta.enableRowTracking" => { + parse_bool(v.as_ref()).map(|val| props.enable_row_tracking = Some(val)) + } + _ => None, + }; + if parsed.is_none() { + props + .unknown_properties + .insert(k.as_ref().to_string(), v.as_ref().to_string()); + } + } + props + } } -/// Deserialize a string representing a positive integer into an `Option`. -pub(crate) fn deserialize_pos_int<'de, D: Deserializer<'de>>( - deserializer: D, -) -> Result, D::Error> { - let s: String = Deserialize::deserialize(deserializer)?; +/// Deserialize a string representing a positive integer into an `Option`. Returns `Some` if +/// successfully parses, and `None` otherwise. +pub(crate) fn parse_positive_int(s: &str) -> Option { // parse to i64 (then check n > 0) since java doesn't even allow u64 - let n: i64 = s.parse().map_err(de::Error::custom)?; - require!(n > 0, de::Error::custom("expected a positive integer")); - Ok(Some(n)) + let n: i64 = s.parse().ok()?; + if n > 0 { + Some(n) + } else { + None + } } -/// Deserialize a string representing a boolean into an `Option`. -pub(crate) fn deserialize_bool<'de, D: Deserializer<'de>>( - deserializer: D, -) -> Result, D::Error> { - let s: String = Deserialize::deserialize(deserializer)?; - match s.as_str() { - "true" => Ok(Some(true)), - "false" => Ok(Some(false)), - _ => Err(de::Error::unknown_variant(&s, &["true", "false"])), +/// Deserialize a string representing a boolean into an `Option`. Returns `Some` if +/// successfully parses, and `None` otherwise. +pub(crate) fn parse_bool(s: &str) -> Option { + match s { + "true" => Some(true), + "false" => Some(false), + _ => None, } } -/// Deserialize a comma-separated list of column names into an `Option>`. -pub(crate) fn deserialize_column_names<'de, D: Deserializer<'de>>( - deserializer: D, -) -> Result>, D::Error> { - let s: String = Deserialize::deserialize(deserializer)?; - let column_names = ColumnName::parse_column_name_list(&s).map_err(de::Error::custom)?; - Ok(Some(column_names)) +/// Deserialize a comma-separated list of column names into an `Option>`. Returns +/// `Some` if successfully parses, and `None` otherwise. +pub(crate) fn parse_column_names(s: &str) -> Option> { + ColumnName::parse_column_name_list(&s).ok() } /// Deserialize an interval string of the form "interval 5 days" into an `Option`. -pub(crate) fn deserialize_interval<'de, D: Deserializer<'de>>( - deserializer: D, -) -> Result, D::Error> { - let s = String::deserialize(deserializer)?; - parse_interval(&s).map(Some).map_err(de::Error::custom) +/// Returns `Some` if successfully parses, and `None` otherwise. +pub(crate) fn parse_interval(s: &str) -> Option { + parse_interval_impl(s).ok() } #[derive(thiserror::Error, Debug)] @@ -95,7 +159,7 @@ pub enum ParseIntervalError { /// https://github.com/delta-io/delta-rs/blob/d4f18b3ae9d616e771b5d0e0fa498d0086fd91eb/crates/core/src/table/config.rs#L474 /// /// See issue delta-kernel-rs/#507 for details: https://github.com/delta-io/delta-kernel-rs/issues/507 -fn parse_interval(value: &str) -> Result { +fn parse_interval_impl(value: &str) -> Result { let mut it = value.split_whitespace(); if it.next() != Some("interval") { return Err(ParseIntervalError::NotAnInterval(value.to_string())); @@ -229,22 +293,22 @@ mod tests { #[test] fn test_invalid_parse_interval() { assert_eq!( - parse_interval("whatever").err().unwrap().to_string(), + parse_interval_impl("whatever").err().unwrap().to_string(), "'whatever' is not an interval".to_string() ); assert_eq!( - parse_interval("interval").err().unwrap().to_string(), + parse_interval_impl("interval").err().unwrap().to_string(), "'interval' is not an interval".to_string() ); assert_eq!( - parse_interval("interval 2").err().unwrap().to_string(), + parse_interval_impl("interval 2").err().unwrap().to_string(), "'interval 2' is not an interval".to_string() ); assert_eq!( - parse_interval("interval 2 months") + parse_interval_impl("interval 2 months") .err() .unwrap() .to_string(), @@ -252,7 +316,7 @@ mod tests { ); assert_eq!( - parse_interval("interval 2 years") + parse_interval_impl("interval 2 years") .err() .unwrap() .to_string(), @@ -260,7 +324,7 @@ mod tests { ); assert_eq!( - parse_interval("interval two years") + parse_interval_impl("interval two years") .err() .unwrap() .to_string(), @@ -268,7 +332,7 @@ mod tests { ); assert_eq!( - parse_interval("interval -25 hours") + parse_interval_impl("interval -25 hours") .err() .unwrap() .to_string(), From b3cdc613874f7c105244210db85a261c1e501206 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 22 Nov 2024 15:05:11 -0800 Subject: [PATCH 16/20] cleanup --- ffi/src/lib.rs | 2 -- kernel/src/error.rs | 7 ------- kernel/src/table_properties.rs | 2 +- kernel/src/table_properties/deserialize.rs | 2 +- 4 files changed, 2 insertions(+), 11 deletions(-) diff --git a/ffi/src/lib.rs b/ffi/src/lib.rs index 104f43f54..8bc33647e 100644 --- a/ffi/src/lib.rs +++ b/ffi/src/lib.rs @@ -382,7 +382,6 @@ pub enum KernelError { InvalidCommitInfo, FileAlreadyExists, MissingCommitInfo, - InvalidTableProperties, UnsupportedError, ParseIntervalError, } @@ -436,7 +435,6 @@ impl From for KernelError { Error::InvalidCommitInfo(_) => KernelError::InvalidCommitInfo, Error::FileAlreadyExists(_) => KernelError::FileAlreadyExists, Error::MissingCommitInfo => KernelError::MissingCommitInfo, - Error::InvalidTableProperties(_) => KernelError::InvalidTableProperties, Error::Unsupported(_) => KernelError::UnsupportedError, Error::ParseIntervalError(_) => KernelError::ParseIntervalError, } diff --git a/kernel/src/error.rs b/kernel/src/error.rs index 1ceb15ce6..4ac3adf03 100644 --- a/kernel/src/error.rs +++ b/kernel/src/error.rs @@ -177,10 +177,6 @@ pub enum Error { #[error("File already exists: {0}")] FileAlreadyExists(String), - /// The `metadata` actions's `configuration` field was unable to parse into `TableProperties` - #[error("Invalid table properties: {0}")] - InvalidTableProperties(String), - /// Some functionality is currently unsupported #[error("Unsupported: {0}")] Unsupported(String), @@ -239,9 +235,6 @@ impl Error { pub(crate) fn invalid_log_path(msg: impl ToString) -> Self { Self::InvalidLogPath(msg.to_string()) } - pub(crate) fn invalid_table_properties(msg: impl ToString) -> Self { - Self::InvalidTableProperties(msg.to_string()) - } pub fn internal_error(msg: impl ToString) -> Self { Self::InternalError(msg.to_string()).with_backtrace() diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index 98cf2bbfe..bcb58a756 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -25,7 +25,7 @@ pub use deserialize::ParseIntervalError; /// Delta table properties. These are parsed from the 'configuration' map in the most recent /// 'Metadata' action of a table. /// -/// Reference: https://github.com/delta-io/delta/blob/master/spark/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala +/// Reference: #[derive(Debug, Clone, Eq, PartialEq, Default)] pub struct TableProperties { /// true for this Delta table to be append-only. If append-only, existing records cannot be diff --git a/kernel/src/table_properties/deserialize.rs b/kernel/src/table_properties/deserialize.rs index 0ed28ad03..777a30988 100644 --- a/kernel/src/table_properties/deserialize.rs +++ b/kernel/src/table_properties/deserialize.rs @@ -120,7 +120,7 @@ pub(crate) fn parse_bool(s: &str) -> Option { /// Deserialize a comma-separated list of column names into an `Option>`. Returns /// `Some` if successfully parses, and `None` otherwise. pub(crate) fn parse_column_names(s: &str) -> Option> { - ColumnName::parse_column_name_list(&s).ok() + ColumnName::parse_column_name_list(s).ok() } /// Deserialize an interval string of the form "interval 5 days" into an `Option`. From d8a293358233af3f7d7209be61471f39330e321e Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 22 Nov 2024 15:30:30 -0800 Subject: [PATCH 17/20] add back col mapping mode fn --- kernel/src/snapshot.rs | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 154f79be1..30063da3e 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -131,6 +131,13 @@ impl Snapshot { &self.table_properties } + /// Get the [column mapping + /// mode](https://github.com/delta-io/delta/blob/master/PROTOCOL.md#column-mapping) at this + /// `Snapshot`s version. + pub fn column_mapping_mode(&self) -> ColumnMappingMode { + self.column_mapping_mode + } + /// Create a [`ScanBuilder`] for an `Arc`. pub fn scan_builder(self: Arc) -> ScanBuilder { ScanBuilder::new(self) From d1ce73d844bad0dbd139e447cb58b05af49ca40d Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Fri, 22 Nov 2024 22:29:55 -0800 Subject: [PATCH 18/20] address ryan review --- kernel/src/actions/mod.rs | 4 +- kernel/src/error.rs | 2 +- kernel/src/table_properties.rs | 2 +- kernel/src/table_properties/deserialize.rs | 131 +++++++++------------ 4 files changed, 59 insertions(+), 80 deletions(-) diff --git a/kernel/src/actions/mod.rs b/kernel/src/actions/mod.rs index 346c2622b..ac169e9cc 100644 --- a/kernel/src/actions/mod.rs +++ b/kernel/src/actions/mod.rs @@ -133,8 +133,8 @@ impl Metadata { } /// Parse the metadata configuration HashMap into a TableProperties struct. - /// Note that this method doesn't return a result since any items that fail to parse are - /// propagated through to the `TableProperties.unknown_properties` field. + /// Note that parsing is infallible -- any items that fail to parse are simply propagated + /// through to the `TableProperties.unknown_properties` field. pub fn parse_table_properties(&self) -> TableProperties { TableProperties::from(self.configuration.iter()) } diff --git a/kernel/src/error.rs b/kernel/src/error.rs index 4b5892c8e..b110e3cbc 100644 --- a/kernel/src/error.rs +++ b/kernel/src/error.rs @@ -182,7 +182,7 @@ pub enum Error { #[error("Unsupported: {0}")] Unsupported(String), - /// Parsing error when attempting to deserizlize an interval + /// Parsing error when attempting to deserialize an interval #[error(transparent)] ParseIntervalError(#[from] ParseIntervalError), diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index bcb58a756..2bbf0821d 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -222,7 +222,7 @@ mod tests { #[test] fn allow_unknown_keys() { - let properties = [("some_random_unknown_key".to_string(), "test".to_string())]; + let properties = [("unknown_properties".to_string(), "two words".to_string())]; let actual = TableProperties::from(properties.clone().into_iter()); let expected = TableProperties { unknown_properties: HashMap::from(properties), diff --git a/kernel/src/table_properties/deserialize.rs b/kernel/src/table_properties/deserialize.rs index 777a30988..dba80ed42 100644 --- a/kernel/src/table_properties/deserialize.rs +++ b/kernel/src/table_properties/deserialize.rs @@ -17,94 +17,73 @@ const SECONDS_PER_WEEK: u64 = 7 * SECONDS_PER_DAY; impl From for TableProperties where I: IntoIterator, - K: AsRef, - V: AsRef, + K: AsRef + Into, + V: AsRef + Into, { fn from(unparsed: I) -> Self { let mut props = TableProperties::default(); - for (k, v) in unparsed { - let parsed = - match k.as_ref() { - "delta.appendOnly" => { - parse_bool(v.as_ref()).map(|val| props.append_only = Some(val)) - } - "delta.autoOptimize.autoCompact" => { - parse_bool(v.as_ref()).map(|val| props.auto_compact = Some(val)) - } - "delta.autoOptimize.optimizeWrite" => { - parse_bool(v.as_ref()).map(|val| props.optimize_write = Some(val)) - } - "delta.checkpointInterval" => parse_positive_int(v.as_ref()) - .map(|val| props.checkpoint_interval = Some(val)), - "delta.checkpoint.writeStatsAsJson" => parse_bool(v.as_ref()) - .map(|val| props.checkpoint_write_stats_as_json = Some(val)), - "delta.checkpoint.writeStatsAsStruct" => parse_bool(v.as_ref()) - .map(|val| props.checkpoint_write_stats_as_struct = Some(val)), - "delta.columnMapping.mode" => ColumnMappingMode::try_from(v.as_ref()) - .map(|val| props.column_mapping_mode = Some(val)) - .ok(), - "delta.dataSkippingNumIndexedCols" => { - DataSkippingNumIndexedCols::try_from(v.as_ref()) - .map(|val| props.data_skipping_num_indexed_cols = Some(val)) - .ok() - } - "delta.dataSkippingStatsColumns" => parse_column_names(v.as_ref()) - .map(|val| props.data_skipping_stats_columns = Some(val)), - "delta.deletedFileRetentionDuration" => parse_interval(v.as_ref()) - .map(|val| props.deleted_file_retention_duration = Some(val)), - "delta.enableChangeDataFeed" => { - parse_bool(v.as_ref()).map(|val| props.enable_change_data_feed = Some(val)) - } - "delta.enableDeletionVectors" => { - parse_bool(v.as_ref()).map(|val| props.enable_deletion_vectors = Some(val)) - } - "delta.isolationLevel" => IsolationLevel::try_from(v.as_ref()) - .map(|val| props.isolation_level = Some(val)) - .ok(), - "delta.logRetentionDuration" => parse_interval(v.as_ref()) - .map(|val| props.log_retention_duration = Some(val)), - "delta.enableExpiredLogCleanup" => parse_bool(v.as_ref()) - .map(|val| props.enable_expired_log_cleanup = Some(val)), - "delta.randomizeFilePrefixes" => { - parse_bool(v.as_ref()).map(|val| props.randomize_file_prefixes = Some(val)) - } - "delta.randomPrefixLength" => parse_positive_int(v.as_ref()) - .map(|val| props.random_prefix_length = Some(val)), - "delta.setTransactionRetentionDuration" => parse_interval(v.as_ref()) - .map(|val| props.set_transaction_retention_duration = Some(val)), - "delta.targetFileSize" => { - parse_positive_int(v.as_ref()).map(|val| props.target_file_size = Some(val)) - } - "delta.tuneFileSizesForRewrites" => parse_bool(v.as_ref()) - .map(|val| props.tune_file_sizes_for_rewrites = Some(val)), - "delta.checkpointPolicy" => CheckpointPolicy::try_from(v.as_ref()) - .map(|val| props.checkpoint_policy = Some(val)) - .ok(), - "delta.enableRowTracking" => { - parse_bool(v.as_ref()).map(|val| props.enable_row_tracking = Some(val)) - } - _ => None, - }; - if parsed.is_none() { - props - .unknown_properties - .insert(k.as_ref().to_string(), v.as_ref().to_string()); - } - } + let unparsed = unparsed.into_iter().filter(|(k, v)| { + // Only keep elements that fail to parse + try_parse(&mut props, k.as_ref(), v.as_ref()).is_none() + }); + props.unknown_properties = unparsed.map(|(k, v)| (k.into(), v.into())).collect(); props } } +// attempt to parse a key-value pair into a `TableProperties` struct. Returns Some(()) if the key +// was successfully parsed, and None otherwise. +fn try_parse(props: &mut TableProperties, k: &str, v: &str) -> Option<()> { + match k { + "delta.appendOnly" => props.append_only = Some(parse_bool(v)?), + "delta.autoOptimize.autoCompact" => props.auto_compact = Some(parse_bool(v)?), + "delta.autoOptimize.optimizeWrite" => props.optimize_write = Some(parse_bool(v)?), + "delta.checkpointInterval" => props.checkpoint_interval = Some(parse_positive_int(v)?), + "delta.checkpoint.writeStatsAsJson" => { + props.checkpoint_write_stats_as_json = Some(parse_bool(v)?) + } + "delta.checkpoint.writeStatsAsStruct" => { + props.checkpoint_write_stats_as_struct = Some(parse_bool(v)?) + } + "delta.columnMapping.mode" => { + props.column_mapping_mode = ColumnMappingMode::try_from(v).ok() + } + "delta.dataSkippingNumIndexedCols" => { + props.data_skipping_num_indexed_cols = DataSkippingNumIndexedCols::try_from(v).ok() + } + "delta.dataSkippingStatsColumns" => { + props.data_skipping_stats_columns = Some(parse_column_names(v)?) + } + "delta.deletedFileRetentionDuration" => { + props.deleted_file_retention_duration = Some(parse_interval(v)?) + } + "delta.enableChangeDataFeed" => props.enable_change_data_feed = Some(parse_bool(v)?), + "delta.enableDeletionVectors" => props.enable_deletion_vectors = Some(parse_bool(v)?), + "delta.isolationLevel" => props.isolation_level = IsolationLevel::try_from(v).ok(), + "delta.logRetentionDuration" => props.log_retention_duration = Some(parse_interval(v)?), + "delta.enableExpiredLogCleanup" => props.enable_expired_log_cleanup = Some(parse_bool(v)?), + "delta.randomizeFilePrefixes" => props.randomize_file_prefixes = Some(parse_bool(v)?), + "delta.randomPrefixLength" => props.random_prefix_length = Some(parse_positive_int(v)?), + "delta.setTransactionRetentionDuration" => { + props.set_transaction_retention_duration = Some(parse_interval(v)?) + } + "delta.targetFileSize" => props.target_file_size = Some(parse_positive_int(v)?), + "delta.tuneFileSizesForRewrites" => { + props.tune_file_sizes_for_rewrites = Some(parse_bool(v)?) + } + "delta.checkpointPolicy" => props.checkpoint_policy = CheckpointPolicy::try_from(v).ok(), + "delta.enableRowTracking" => props.enable_row_tracking = Some(parse_bool(v)?), + _ => return None, + } + Some(()) +} + /// Deserialize a string representing a positive integer into an `Option`. Returns `Some` if /// successfully parses, and `None` otherwise. pub(crate) fn parse_positive_int(s: &str) -> Option { // parse to i64 (then check n > 0) since java doesn't even allow u64 let n: i64 = s.parse().ok()?; - if n > 0 { - Some(n) - } else { - None - } + (n > 0).then_some(n) } /// Deserialize a string representing a boolean into an `Option`. Returns `Some` if From 6d1b466aebc4ad82b56cdb70b57a755f35f3a390 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 25 Nov 2024 08:34:10 -0800 Subject: [PATCH 19/20] use NonZero --- kernel/src/table_properties.rs | 13 ++++---- kernel/src/table_properties/deserialize.rs | 38 ++++++++++++++++++++-- 2 files changed, 42 insertions(+), 9 deletions(-) diff --git a/kernel/src/table_properties.rs b/kernel/src/table_properties.rs index 2bbf0821d..949bd5ac4 100644 --- a/kernel/src/table_properties.rs +++ b/kernel/src/table_properties.rs @@ -11,6 +11,7 @@ //! table. use std::collections::HashMap; +use std::num::NonZero; use std::time::Duration; use crate::expressions::ColumnName; @@ -43,7 +44,7 @@ pub struct TableProperties { /// Interval (expressed as number of commits) after which a new checkpoint should be created. /// E.g. if checkpoint interval = 10, then a checkpoint should be written every 10 commits. - pub checkpoint_interval: Option, + pub checkpoint_interval: Option>, /// true for Delta Lake to write file statistics in checkpoints in JSON format for the stats column. pub checkpoint_write_stats_as_json: Option, @@ -114,7 +115,7 @@ pub struct TableProperties { /// When delta.randomizeFilePrefixes is set to true, the number of characters that Delta /// generates for random prefixes. - pub random_prefix_length: Option, + pub random_prefix_length: Option>, /// The shortest duration within which new snapshots will retain transaction identifiers (for /// example, SetTransactions). When a new snapshot sees a transaction identifier older than or @@ -124,7 +125,7 @@ pub struct TableProperties { /// The target file size in bytes or higher units for file tuning. For example, 104857600 /// (bytes) or 100mb. - pub target_file_size: Option, + pub target_file_size: Option>, /// The target file size in bytes or higher units for file tuning. For example, 104857600 /// (bytes) or 100mb. @@ -273,7 +274,7 @@ mod tests { append_only: Some(true), optimize_write: Some(true), auto_compact: Some(true), - checkpoint_interval: Some(101), + checkpoint_interval: Some(NonZero::new(101).unwrap()), checkpoint_write_stats_as_json: Some(true), checkpoint_write_stats_as_struct: Some(true), column_mapping_mode: Some(ColumnMappingMode::Id), @@ -286,9 +287,9 @@ mod tests { log_retention_duration: Some(Duration::new(2, 0)), enable_expired_log_cleanup: Some(true), randomize_file_prefixes: Some(true), - random_prefix_length: Some(1001), + random_prefix_length: Some(NonZero::new(1001).unwrap()), set_transaction_retention_duration: Some(Duration::new(60, 0)), - target_file_size: Some(1_000_000_000), + target_file_size: Some(NonZero::new(1_000_000_000).unwrap()), tune_file_sizes_for_rewrites: Some(true), checkpoint_policy: Some(CheckpointPolicy::V2), enable_row_tracking: Some(true), diff --git a/kernel/src/table_properties/deserialize.rs b/kernel/src/table_properties/deserialize.rs index dba80ed42..99b252f9a 100644 --- a/kernel/src/table_properties/deserialize.rs +++ b/kernel/src/table_properties/deserialize.rs @@ -2,6 +2,7 @@ //! us to relatively simply implement the functionality described in the protocol and expose //! 'simple' types to the user in the [`TableProperties`] struct. E.g. we can expose a `bool` //! directly instead of a `BoolConfig` type that we implement `Deserialize` for. +use std::num::NonZero; use std::time::Duration; use super::*; @@ -9,6 +10,8 @@ use crate::expressions::ColumnName; use crate::table_features::ColumnMappingMode; use crate::utils::require; +use tracing::warn; + const SECONDS_PER_MINUTE: u64 = 60; const SECONDS_PER_HOUR: u64 = 60 * SECONDS_PER_MINUTE; const SECONDS_PER_DAY: u64 = 24 * SECONDS_PER_HOUR; @@ -80,10 +83,10 @@ fn try_parse(props: &mut TableProperties, k: &str, v: &str) -> Option<()> { /// Deserialize a string representing a positive integer into an `Option`. Returns `Some` if /// successfully parses, and `None` otherwise. -pub(crate) fn parse_positive_int(s: &str) -> Option { +pub(crate) fn parse_positive_int(s: &str) -> Option> { // parse to i64 (then check n > 0) since java doesn't even allow u64 let n: i64 = s.parse().ok()?; - (n > 0).then_some(n) + NonZero::new(n.try_into().ok()?) } /// Deserialize a string representing a boolean into an `Option`. Returns `Some` if @@ -99,7 +102,9 @@ pub(crate) fn parse_bool(s: &str) -> Option { /// Deserialize a comma-separated list of column names into an `Option>`. Returns /// `Some` if successfully parses, and `None` otherwise. pub(crate) fn parse_column_names(s: &str) -> Option> { - ColumnName::parse_column_name_list(s).ok() + ColumnName::parse_column_name_list(s) + .map_err(|e| warn!("column name list failed to parse: {e}")) + .ok() } /// Deserialize an interval string of the form "interval 5 days" into an `Option`. @@ -185,6 +190,33 @@ fn parse_interval_impl(value: &str) -> Result { #[cfg(test)] mod tests { use super::*; + use crate::expressions::column_name; + + #[test] + fn test_parse_column_names() { + assert_eq!( + parse_column_names("`col 1`, col.a2,col3").unwrap(), + vec![ + ColumnName::new(["col 1"]), + column_name!("col.a2"), + column_name!("col3") + ] + ); + } + + #[test] + fn test_parse_bool() { + assert_eq!(parse_bool("true").unwrap(), true); + assert_eq!(parse_bool("false").unwrap(), false); + assert_eq!(parse_bool("whatever"), None); + } + + #[test] + fn test_parse_positive_int() { + assert_eq!(parse_positive_int("123").unwrap().get(), 123); + assert_eq!(parse_positive_int("0"), None); + assert_eq!(parse_positive_int("-123"), None); + } #[test] fn test_parse_interval() { From 437b8dbf8a3a24381be4196a408ab4240293bd64 Mon Sep 17 00:00:00 2001 From: Zach Schuermann Date: Mon, 25 Nov 2024 08:46:34 -0800 Subject: [PATCH 20/20] clippy --- kernel/src/table_properties/deserialize.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kernel/src/table_properties/deserialize.rs b/kernel/src/table_properties/deserialize.rs index 99b252f9a..e22befdb2 100644 --- a/kernel/src/table_properties/deserialize.rs +++ b/kernel/src/table_properties/deserialize.rs @@ -206,8 +206,8 @@ mod tests { #[test] fn test_parse_bool() { - assert_eq!(parse_bool("true").unwrap(), true); - assert_eq!(parse_bool("false").unwrap(), false); + assert!(parse_bool("true").unwrap()); + assert!(!parse_bool("false").unwrap()); assert_eq!(parse_bool("whatever"), None); }