From c7913dc089980cf211bb00639792d9457918a0ce Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 28 Oct 2024 15:15:10 -0700 Subject: [PATCH 01/88] Move log segment into separate module Log segment is going to be used by both snapshot and table changes. It makes sense to separate it into its own module --- kernel/src/lib.rs | 1 + kernel/src/log_segment.rs | 125 +++++++++++++++++++++++++++++++++++++ kernel/src/scan/mod.rs | 70 ++++++++++----------- kernel/src/snapshot.rs | 128 ++++---------------------------------- 4 files changed, 173 insertions(+), 151 deletions(-) create mode 100644 kernel/src/log_segment.rs diff --git a/kernel/src/lib.rs b/kernel/src/lib.rs index 2f686a3ad..c60c5b9d7 100644 --- a/kernel/src/lib.rs +++ b/kernel/src/lib.rs @@ -69,6 +69,7 @@ pub mod path; #[cfg(not(feature = "developer-visibility"))] pub(crate) mod path; +pub mod log_segment; pub mod scan; pub mod schema; pub mod snapshot; diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs new file mode 100644 index 000000000..b000cb837 --- /dev/null +++ b/kernel/src/log_segment.rs @@ -0,0 +1,125 @@ +//! Represents a segment of a delta log + +use crate::expressions::column_expr; +use std::sync::{Arc, LazyLock}; +use url::Url; + +use crate::{ + actions::{get_log_schema, Metadata, Protocol, METADATA_NAME, PROTOCOL_NAME}, + schema::SchemaRef, + DeltaResult, Engine, EngineData, Error, ExpressionRef, FileMeta, +}; +use itertools::Itertools; + +#[derive(Debug)] +#[cfg_attr(feature = "developer-visibility", visibility::make(pub))] +#[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] +pub(crate) struct LogSegment { + log_root: Url, + /// Reverse order sorted commit files in the log segment + commit_files: Vec, + /// checkpoint files in the log segment. + checkpoint_files: Vec, +} + +impl LogSegment { + pub(crate) fn new( + log_root: Url, + commit_files: Vec, + checkpoint_files: Vec, + ) -> Self { + LogSegment { + log_root, + commit_files, + checkpoint_files, + } + } + pub(crate) fn log_root(&self) -> &Url { + &self.log_root + } + /// Read a stream of log data from this log segment. + /// + /// The log files will be read from most recent to oldest. + /// The boolean flags indicates whether the data was read from + /// a commit file (true) or a checkpoint file (false). + /// + /// `read_schema` is the schema to read the log files with. This can be used + /// to project the log files to a subset of the columns. + /// + /// `meta_predicate` is an optional expression to filter the log files with. It is _NOT_ the + /// query's predicate, but rather a predicate for filtering log files themselves. + #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] + #[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] + pub fn replay( + &self, + engine: &dyn Engine, + commit_read_schema: SchemaRef, + checkpoint_read_schema: SchemaRef, + meta_predicate: Option, + ) -> DeltaResult, bool)>> + Send> { + let json_client = engine.get_json_handler(); + let commit_stream = json_client + .read_json_files( + &self.commit_files, + commit_read_schema, + meta_predicate.clone(), + )? + .map_ok(|batch| (batch, true)); + + let parquet_client = engine.get_parquet_handler(); + let checkpoint_stream = parquet_client + .read_parquet_files( + &self.checkpoint_files, + checkpoint_read_schema, + meta_predicate, + )? + .map_ok(|batch| (batch, false)); + + let batches = commit_stream.chain(checkpoint_stream); + + Ok(batches) + } + + pub(crate) fn read_metadata(&self, engine: &dyn Engine) -> DeltaResult<(Metadata, Protocol)> { + let data_batches = self.replay_for_metadata(engine)?; + let mut metadata_opt: Option = None; + let mut protocol_opt: Option = None; + for batch in data_batches { + let (batch, _) = batch?; + if metadata_opt.is_none() { + metadata_opt = crate::actions::Metadata::try_new_from_data(batch.as_ref())?; + } + if protocol_opt.is_none() { + protocol_opt = crate::actions::Protocol::try_new_from_data(batch.as_ref())?; + } + if metadata_opt.is_some() && protocol_opt.is_some() { + // we've found both, we can stop + break; + } + } + match (metadata_opt, protocol_opt) { + (Some(m), Some(p)) => Ok((m, p)), + (None, Some(_)) => Err(Error::MissingMetadata), + (Some(_), None) => Err(Error::MissingProtocol), + _ => Err(Error::MissingMetadataAndProtocol), + } + } + + // Factored out to facilitate testing + pub(crate) fn replay_for_metadata( + &self, + engine: &dyn Engine, + ) -> DeltaResult, bool)>> + Send> { + let schema = get_log_schema().project(&[PROTOCOL_NAME, METADATA_NAME])?; + // filter out log files that do not contain metadata or protocol information + use crate::Expression as Expr; + static META_PREDICATE: LazyLock> = LazyLock::new(|| { + Some(Arc::new(Expr::or( + column_expr!("metaData.id").is_not_null(), + column_expr!("protocol.minReaderVersion").is_not_null(), + ))) + }); + // read the same protocol and metadata schema for both commits and checkpoints + self.replay(engine, schema.clone(), schema, META_PREDICATE.clone()) + } +} diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index 78a3c74ae..5cb21a928 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -473,42 +473,42 @@ fn transform_to_logical_internal( have_partition_cols: bool, ) -> DeltaResult> { let read_schema = global_state.read_schema.clone(); - if have_partition_cols || global_state.column_mapping_mode != ColumnMappingMode::None { - // need to add back partition cols and/or fix-up mapped columns - let all_fields = all_fields - .iter() - .map(|field| match field { - ColumnType::Partition(field_idx) => { - let field = global_state - .logical_schema - .fields - .get_index(*field_idx) - .ok_or_else(|| { - Error::generic("logical schema did not contain expected field, can't transform data") - })?.1; - let name = field.physical_name(global_state.column_mapping_mode)?; - let value_expression = parse_partition_value( - partition_values.get(name), - field.data_type(), - )?; - Ok::(value_expression.into()) - } - ColumnType::Selected(field_name) => Ok(field_name.clone().into()), - }) - .try_collect()?; - let read_expression = Expression::Struct(all_fields); - let result = engine - .get_expression_handler() - .get_evaluator( - read_schema, - read_expression, - global_state.logical_schema.clone().into(), - ) - .evaluate(data.as_ref())?; - Ok(result) - } else { - Ok(data) + if !have_partition_cols && global_state.column_mapping_mode == ColumnMappingMode::None { + return Ok(data); } + // need to add back partition cols and/or fix-up mapped columns + let all_fields = all_fields + .iter() + .map(|field| match field { + ColumnType::Partition(field_idx) => { + let field = global_state + .logical_schema + .fields + .get_index(*field_idx) + .ok_or_else(|| { + Error::generic( + "logical schema did not contain expected field, can't transform data", + ) + })? + .1; + let name = field.physical_name(global_state.column_mapping_mode)?; + let value_expression = + parse_partition_value(partition_values.get(name), field.data_type())?; + Ok::(value_expression.into()) + } + ColumnType::Selected(field_name) => Ok(field_name.clone().into()), + }) + .try_collect()?; + let read_expression = Expression::Struct(all_fields); + let result = engine + .get_expression_handler() + .get_evaluator( + read_schema, + read_expression, + global_state.logical_schema.clone().into(), + ) + .evaluate(data.as_ref())?; + Ok(result) } // some utils that are used in file_stream.rs and state.rs tests diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 93d808585..d62503c4c 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -3,124 +3,22 @@ //! use std::cmp::Ordering; -use std::sync::{Arc, LazyLock}; +use std::sync::Arc; -use itertools::Itertools; use serde::{Deserialize, Serialize}; use tracing::{debug, warn}; use url::Url; -use crate::actions::{get_log_schema, Metadata, Protocol, METADATA_NAME, PROTOCOL_NAME}; -use crate::expressions::column_expr; +use crate::actions::{Metadata, Protocol}; use crate::features::{ColumnMappingMode, COLUMN_MAPPING_MODE_KEY}; +use crate::log_segment::LogSegment; use crate::path::ParsedLogPath; use crate::scan::ScanBuilder; -use crate::schema::{Schema, SchemaRef}; +use crate::schema::Schema; use crate::utils::require; -use crate::{DeltaResult, Engine, Error, FileMeta, FileSystemClient, Version}; -use crate::{EngineData, Expression, ExpressionRef}; +use crate::{DeltaResult, Engine, Error, FileSystemClient, Version}; const LAST_CHECKPOINT_FILE_NAME: &str = "_last_checkpoint"; - -#[derive(Debug)] -#[cfg_attr(feature = "developer-visibility", visibility::make(pub))] -#[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] -struct LogSegment { - log_root: Url, - /// Reverse order sorted commit files in the log segment - pub(crate) commit_files: Vec, - /// checkpoint files in the log segment. - pub(crate) checkpoint_files: Vec, -} - -impl LogSegment { - /// Read a stream of log data from this log segment. - /// - /// The log files will be read from most recent to oldest. - /// The boolean flags indicates whether the data was read from - /// a commit file (true) or a checkpoint file (false). - /// - /// `read_schema` is the schema to read the log files with. This can be used - /// to project the log files to a subset of the columns. - /// - /// `meta_predicate` is an optional expression to filter the log files with. It is _NOT_ the - /// query's predicate, but rather a predicate for filtering log files themselves. - #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] - #[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] - fn replay( - &self, - engine: &dyn Engine, - commit_read_schema: SchemaRef, - checkpoint_read_schema: SchemaRef, - meta_predicate: Option, - ) -> DeltaResult, bool)>> + Send> { - let json_client = engine.get_json_handler(); - let commit_stream = json_client - .read_json_files( - &self.commit_files, - commit_read_schema, - meta_predicate.clone(), - )? - .map_ok(|batch| (batch, true)); - - let parquet_client = engine.get_parquet_handler(); - let checkpoint_stream = parquet_client - .read_parquet_files( - &self.checkpoint_files, - checkpoint_read_schema, - meta_predicate, - )? - .map_ok(|batch| (batch, false)); - - let batches = commit_stream.chain(checkpoint_stream); - - Ok(batches) - } - - fn read_metadata(&self, engine: &dyn Engine) -> DeltaResult> { - let data_batches = self.replay_for_metadata(engine)?; - let mut metadata_opt: Option = None; - let mut protocol_opt: Option = None; - for batch in data_batches { - let (batch, _) = batch?; - if metadata_opt.is_none() { - metadata_opt = crate::actions::Metadata::try_new_from_data(batch.as_ref())?; - } - if protocol_opt.is_none() { - protocol_opt = crate::actions::Protocol::try_new_from_data(batch.as_ref())?; - } - if metadata_opt.is_some() && protocol_opt.is_some() { - // we've found both, we can stop - break; - } - } - match (metadata_opt, protocol_opt) { - (Some(m), Some(p)) => Ok(Some((m, p))), - (None, Some(_)) => Err(Error::MissingMetadata), - (Some(_), None) => Err(Error::MissingProtocol), - _ => Err(Error::MissingMetadataAndProtocol), - } - } - - // Factored out to facilitate testing - fn replay_for_metadata( - &self, - engine: &dyn Engine, - ) -> DeltaResult, bool)>> + Send> { - let schema = get_log_schema().project(&[PROTOCOL_NAME, METADATA_NAME])?; - // filter out log files that do not contain metadata or protocol information - use Expression as Expr; - static META_PREDICATE: LazyLock> = LazyLock::new(|| { - Some(Arc::new(Expr::or( - column_expr!("metaData.id").is_not_null(), - column_expr!("protocol.minReaderVersion").is_not_null(), - ))) - }); - // read the same protocol and metadata schema for both commits and checkpoints - self.replay(engine, schema.clone(), schema, META_PREDICATE.clone()) - } -} - // TODO expose methods for accessing the files of a table (with file pruning). /// In-memory representation of a specific snapshot of a Delta table. While a `DeltaTable` exists /// throughout time, `Snapshot`s represent a view of a table at a specific point in time; they @@ -146,7 +44,7 @@ impl Drop for Snapshot { impl std::fmt::Debug for Snapshot { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("Snapshot") - .field("path", &self.log_segment.log_root.as_str()) + .field("path", &self.log_segment.log_root()) .field("version", &self.version) .field("metadata", &self.metadata) .finish() @@ -204,17 +102,17 @@ impl Snapshot { ); } - let log_segment = LogSegment { - log_root: log_url, - commit_files: commit_files + let log_segment = LogSegment::new( + log_url, + commit_files .into_iter() .map(|log_path| log_path.location) .collect(), - checkpoint_files: checkpoint_files + checkpoint_files .into_iter() .map(|log_path| log_path.location) .collect(), - }; + ); Self::try_new_from_log_segment(table_root, log_segment, version_eff, engine) } @@ -226,9 +124,7 @@ impl Snapshot { version: Version, engine: &dyn Engine, ) -> DeltaResult { - let (metadata, protocol) = log_segment - .read_metadata(engine)? - .ok_or(Error::MissingMetadata)?; + let (metadata, protocol) = log_segment.read_metadata(engine)?; 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(), From 6b331acdb57d1a4cc854e266f166646da5f93446 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 28 Oct 2024 16:13:15 -0700 Subject: [PATCH 02/88] Fix tests, make fields pub --- kernel/src/log_segment.rs | 74 +++++++++++++++++++++++++++++---------- kernel/src/snapshot.rs | 56 ++++------------------------- 2 files changed, 62 insertions(+), 68 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index b000cb837..bd407ffb1 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -15,28 +15,14 @@ use itertools::Itertools; #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] #[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] pub(crate) struct LogSegment { - log_root: Url, + pub log_root: Url, /// Reverse order sorted commit files in the log segment - commit_files: Vec, + pub commit_files: Vec, /// checkpoint files in the log segment. - checkpoint_files: Vec, + pub checkpoint_files: Vec, } impl LogSegment { - pub(crate) fn new( - log_root: Url, - commit_files: Vec, - checkpoint_files: Vec, - ) -> Self { - LogSegment { - log_root, - commit_files, - checkpoint_files, - } - } - pub(crate) fn log_root(&self) -> &Url { - &self.log_root - } /// Read a stream of log data from this log segment. /// /// The log files will be read from most recent to oldest. @@ -106,7 +92,7 @@ impl LogSegment { } // Factored out to facilitate testing - pub(crate) fn replay_for_metadata( + fn replay_for_metadata( &self, engine: &dyn Engine, ) -> DeltaResult, bool)>> + Send> { @@ -123,3 +109,55 @@ impl LogSegment { self.replay(engine, schema.clone(), schema, META_PREDICATE.clone()) } } + +#[cfg(test)] +mod tests { + use std::path::PathBuf; + + use itertools::Itertools; + + use crate::{engine::sync::SyncEngine, Table}; + + // NOTE: In addition to testing the meta-predicate for metadata replay, this test also verifies + // that the parquet reader properly infers nullcount = rowcount for missing columns. The two + // checkpoint part files that contain transaction app ids have truncated schemas that would + // otherwise fail skipping due to their missing nullcount stat: + // + // Row group 0: count: 1 total(compressed): 111 B total(uncompressed):107 B + // -------------------------------------------------------------------------------- + // type nulls min / max + // txn.appId BINARY 0 "3ae45b72-24e1-865a-a211-3..." / "3ae45b72-24e1-865a-a211-3..." + // txn.version INT64 0 "4390" / "4390" + #[test] + fn test_replay_for_metadata() { + let path = std::fs::canonicalize(PathBuf::from("./tests/data/parquet_row_group_skipping/")); + let url = url::Url::from_directory_path(path.unwrap()).unwrap(); + let engine = SyncEngine::new(); + + let table = Table::new(url); + let snapshot = table.snapshot(&engine, None).unwrap(); + let data: Vec<_> = snapshot + .log_segment + .replay_for_metadata(&engine) + .unwrap() + .try_collect() + .unwrap(); + + // The checkpoint has five parts, each containing one action: + // 1. txn (physically missing P&M columns) + // 2. metaData + // 3. protocol + // 4. add + // 5. txn (physically missing P&M columns) + // + // The parquet reader should skip parts 1, 3, and 5. Note that the actual `read_metadata` + // always skips parts 4 and 5 because it terminates the iteration after finding both P&M. + // + // NOTE: Each checkpoint part is a single-row file -- guaranteed to produce one row group. + // + // WARNING: https://github.com/delta-incubator/delta-kernel-rs/issues/434 -- We currently + // read parts 1 and 5 (4 in all instead of 2) because row group skipping is disabled for + // missing columns, but can still skip part 3 because has valid nullcount stats for P&M. + assert_eq!(data.len(), 4); + } +} diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index d62503c4c..6aa8f20b4 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -44,7 +44,7 @@ impl Drop for Snapshot { impl std::fmt::Debug for Snapshot { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("Snapshot") - .field("path", &self.log_segment.log_root()) + .field("path", &self.log_segment.log_root.as_str()) .field("version", &self.version) .field("metadata", &self.metadata) .finish() @@ -102,17 +102,17 @@ impl Snapshot { ); } - let log_segment = LogSegment::new( - log_url, - commit_files + let log_segment = LogSegment { + log_root: log_url, + commit_files: commit_files .into_iter() .map(|log_path| log_path.location) .collect(), - checkpoint_files + checkpoint_files: checkpoint_files .into_iter() .map(|log_path| log_path.location) .collect(), - ); + }; Self::try_new_from_log_segment(table_root, log_segment, version_eff, engine) } @@ -377,7 +377,6 @@ mod tests { use crate::engine::default::filesystem::ObjectStoreFileSystemClient; use crate::engine::sync::SyncEngine; use crate::schema::StructType; - use crate::Table; #[test] fn test_snapshot_read_metadata() { @@ -552,49 +551,6 @@ mod tests { assert!(invalid.is_none()) } - // NOTE: In addition to testing the meta-predicate for metadata replay, this test also verifies - // that the parquet reader properly infers nullcount = rowcount for missing columns. The two - // checkpoint part files that contain transaction app ids have truncated schemas that would - // otherwise fail skipping due to their missing nullcount stat: - // - // Row group 0: count: 1 total(compressed): 111 B total(uncompressed):107 B - // -------------------------------------------------------------------------------- - // type nulls min / max - // txn.appId BINARY 0 "3ae45b72-24e1-865a-a211-3..." / "3ae45b72-24e1-865a-a211-3..." - // txn.version INT64 0 "4390" / "4390" - #[test] - fn test_replay_for_metadata() { - let path = std::fs::canonicalize(PathBuf::from("./tests/data/parquet_row_group_skipping/")); - let url = url::Url::from_directory_path(path.unwrap()).unwrap(); - let engine = SyncEngine::new(); - - let table = Table::new(url); - let snapshot = table.snapshot(&engine, None).unwrap(); - let data: Vec<_> = snapshot - .log_segment - .replay_for_metadata(&engine) - .unwrap() - .try_collect() - .unwrap(); - - // The checkpoint has five parts, each containing one action: - // 1. txn (physically missing P&M columns) - // 2. metaData - // 3. protocol - // 4. add - // 5. txn (physically missing P&M columns) - // - // The parquet reader should skip parts 1, 3, and 5. Note that the actual `read_metadata` - // always skips parts 4 and 5 because it terminates the iteration after finding both P&M. - // - // NOTE: Each checkpoint part is a single-row file -- guaranteed to produce one row group. - // - // WARNING: https://github.com/delta-incubator/delta-kernel-rs/issues/434 -- We currently - // read parts 1 and 5 (4 in all instead of 2) because row group skipping is disabled for - // missing columns, but can still skip part 3 because has valid nullcount stats for P&M. - assert_eq!(data.len(), 4); - } - #[test_log::test] fn test_read_table_with_checkpoint() { let path = std::fs::canonicalize(PathBuf::from( From f1f988606abb629c0de8e337755210c6ea8a95c9 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 28 Oct 2024 16:23:02 -0700 Subject: [PATCH 03/88] Improve comments --- kernel/src/log_segment.rs | 3 ++- kernel/src/table.rs | 13 +++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index bd407ffb1..0d16f2fbb 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -66,6 +66,7 @@ impl LogSegment { Ok(batches) } + // Get the most up-to-date Protocol and Metadata actions pub(crate) fn read_metadata(&self, engine: &dyn Engine) -> DeltaResult<(Metadata, Protocol)> { let data_batches = self.replay_for_metadata(engine)?; let mut metadata_opt: Option = None; @@ -91,7 +92,7 @@ impl LogSegment { } } - // Factored out to facilitate testing + // Replay the commit log, projecting rows to only contain Protocol and Metadata action columns. fn replay_for_metadata( &self, engine: &dyn Engine, diff --git a/kernel/src/table.rs b/kernel/src/table.rs index d7aba9894..328c0552e 100644 --- a/kernel/src/table.rs +++ b/kernel/src/table.rs @@ -6,6 +6,7 @@ use std::path::PathBuf; use url::Url; use crate::snapshot::Snapshot; +// use crate::table_changes::TableChanges; use crate::transaction::Transaction; use crate::{DeltaResult, Engine, Error, Version}; @@ -78,6 +79,18 @@ impl Table { Snapshot::try_new(self.location.clone(), engine, version) } + // /// Create a [`TableChanges`] of the table corresponding to `version`. + // /// + // /// If no version is supplied, a snapshot for the latest version will be created. + // pub fn table_changes( + // &self, + // engine: &dyn Engine, + // start_version: Version, + // end_version: Option, + // ) -> DeltaResult { + // TableChanges::try_new(self.location().clone(), engine, start_version, end_version) + // } + /// Create a new write transaction for this table. pub fn new_transaction(&self, engine: &dyn Engine) -> DeltaResult { Ok(Transaction::new(self.snapshot(engine, None)?)) From 81221133ca51ffc21bad9669cdb62b64adedeec4 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 28 Oct 2024 16:25:27 -0700 Subject: [PATCH 04/88] Remove table changes --- kernel/src/table.rs | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/kernel/src/table.rs b/kernel/src/table.rs index 328c0552e..d7aba9894 100644 --- a/kernel/src/table.rs +++ b/kernel/src/table.rs @@ -6,7 +6,6 @@ use std::path::PathBuf; use url::Url; use crate::snapshot::Snapshot; -// use crate::table_changes::TableChanges; use crate::transaction::Transaction; use crate::{DeltaResult, Engine, Error, Version}; @@ -79,18 +78,6 @@ impl Table { Snapshot::try_new(self.location.clone(), engine, version) } - // /// Create a [`TableChanges`] of the table corresponding to `version`. - // /// - // /// If no version is supplied, a snapshot for the latest version will be created. - // pub fn table_changes( - // &self, - // engine: &dyn Engine, - // start_version: Version, - // end_version: Option, - // ) -> DeltaResult { - // TableChanges::try_new(self.location().clone(), engine, start_version, end_version) - // } - /// Create a new write transaction for this table. pub fn new_transaction(&self, engine: &dyn Engine) -> DeltaResult { Ok(Transaction::new(self.snapshot(engine, None)?)) From 471a8583a80d5ee2930d3e3713c1f912749cea0b Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 28 Oct 2024 16:57:10 -0700 Subject: [PATCH 05/88] change visibility --- kernel/src/log_segment.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 0d16f2fbb..b473f660a 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -1,4 +1,5 @@ -//! Represents a segment of a delta log +//! Represents a segment of a delta log. [`LogSegment`] wraps a set of checkpoint and commit +//! files. use crate::expressions::column_expr; use std::sync::{Arc, LazyLock}; @@ -36,7 +37,7 @@ impl LogSegment { /// query's predicate, but rather a predicate for filtering log files themselves. #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] #[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] - pub fn replay( + pub(crate) fn replay( &self, engine: &dyn Engine, commit_read_schema: SchemaRef, From 0b8463ae2572f20e9a484862a7baf0de4661e6e0 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 14:22:46 -0800 Subject: [PATCH 06/88] Remove old log segment --- kernel/src/scan/mod.rs | 2 +- kernel/src/snapshot.rs | 103 +---------------------------------------- 2 files changed, 3 insertions(+), 102 deletions(-) diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index 12afa0511..310258c30 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -9,7 +9,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::expressions::{Expression, ExpressionRef, Scalar}; use crate::features::ColumnMappingMode; use crate::scan::state::{DvInfo, Stats}; use crate::schema::{DataType, Schema, SchemaRef, StructField, StructType}; diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index ec757d48a..4240e49df 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -3,8 +3,9 @@ //! use std::cmp::Ordering; -use std::sync::Arc; +use std::sync::{Arc, LazyLock}; +use itertools::Itertools; use serde::{Deserialize, Serialize}; use tracing::{debug, warn}; use url::Url; @@ -19,106 +20,6 @@ use crate::utils::require; use crate::{DeltaResult, Engine, Error, FileSystemClient, Version}; const LAST_CHECKPOINT_FILE_NAME: &str = "_last_checkpoint"; - -#[derive(Debug)] -#[cfg_attr(feature = "developer-visibility", visibility::make(pub))] -#[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] -struct LogSegment { - log_root: Url, - /// Reverse order sorted commit files in the log segment - pub(crate) commit_files: Vec, - /// checkpoint files in the log segment. - pub(crate) checkpoint_files: Vec, -} - -impl LogSegment { - /// Read a stream of log data from this log segment. - /// - /// The log files will be read from most recent to oldest. - /// The boolean flags indicates whether the data was read from - /// a commit file (true) or a checkpoint file (false). - /// - /// `read_schema` is the schema to read the log files with. This can be used - /// to project the log files to a subset of the columns. - /// - /// `meta_predicate` is an optional expression to filter the log files with. It is _NOT_ the - /// query's predicate, but rather a predicate for filtering log files themselves. - #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] - #[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] - fn replay( - &self, - engine: &dyn Engine, - commit_read_schema: SchemaRef, - checkpoint_read_schema: SchemaRef, - meta_predicate: Option, - ) -> DeltaResult, bool)>> + Send> { - let json_client = engine.get_json_handler(); - let commit_stream = json_client - .read_json_files( - &self.commit_files, - commit_read_schema, - meta_predicate.clone(), - )? - .map_ok(|batch| (batch, true)); - - let parquet_client = engine.get_parquet_handler(); - let checkpoint_stream = parquet_client - .read_parquet_files( - &self.checkpoint_files, - checkpoint_read_schema, - meta_predicate, - )? - .map_ok(|batch| (batch, false)); - - let batches = commit_stream.chain(checkpoint_stream); - - Ok(batches) - } - - fn read_metadata(&self, engine: &dyn Engine) -> DeltaResult> { - let data_batches = self.replay_for_metadata(engine)?; - let mut metadata_opt: Option = None; - let mut protocol_opt: Option = None; - for batch in data_batches { - let (batch, _) = batch?; - if metadata_opt.is_none() { - metadata_opt = crate::actions::Metadata::try_new_from_data(batch.as_ref())?; - } - if protocol_opt.is_none() { - protocol_opt = crate::actions::Protocol::try_new_from_data(batch.as_ref())?; - } - if metadata_opt.is_some() && protocol_opt.is_some() { - // we've found both, we can stop - break; - } - } - match (metadata_opt, protocol_opt) { - (Some(m), Some(p)) => Ok(Some((m, p))), - (None, Some(_)) => Err(Error::MissingMetadata), - (Some(_), None) => Err(Error::MissingProtocol), - _ => Err(Error::MissingMetadataAndProtocol), - } - } - - // Factored out to facilitate testing - fn replay_for_metadata( - &self, - engine: &dyn Engine, - ) -> DeltaResult, bool)>> + Send> { - let schema = get_log_schema().project(&[PROTOCOL_NAME, METADATA_NAME])?; - // filter out log files that do not contain metadata or protocol information - use Expression as Expr; - static META_PREDICATE: LazyLock> = LazyLock::new(|| { - Some(Arc::new(Expr::or( - Expr::column([METADATA_NAME, "id"]).is_not_null(), - Expr::column([PROTOCOL_NAME, "minReaderVersion"]).is_not_null(), - ))) - }); - // read the same protocol and metadata schema for both commits and checkpoints - self.replay(engine, schema.clone(), schema, META_PREDICATE.clone()) - } -} - // TODO expose methods for accessing the files of a table (with file pruning). /// In-memory representation of a specific snapshot of a Delta table. While a `DeltaTable` exists /// throughout time, `Snapshot`s represent a view of a table at a specific point in time; they From 5300a7b2100b7efc901f66872bb31af6998b2261 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 14:34:21 -0800 Subject: [PATCH 07/88] fix failing tests --- kernel/src/scan/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index 310258c30..12afa0511 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -9,7 +9,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::{Expression, ExpressionRef, Scalar}; +use crate::expressions::{ColumnName, Expression, ExpressionRef, Scalar}; use crate::features::ColumnMappingMode; use crate::scan::state::{DvInfo, Stats}; use crate::schema::{DataType, Schema, SchemaRef, StructField, StructType}; From 81d0de0ad22e4fa2b1f97d519cc20a01f710feb1 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 14:44:24 -0800 Subject: [PATCH 08/88] Get rid of warnings --- kernel/src/snapshot.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 4240e49df..65ffe3918 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -3,9 +3,8 @@ //! use std::cmp::Ordering; -use std::sync::{Arc, LazyLock}; +use std::sync::Arc; -use itertools::Itertools; use serde::{Deserialize, Serialize}; use tracing::{debug, warn}; use url::Url; From 6b85932ab54804c1cab062bdc3f3424f83dfd49b Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 14:58:21 -0800 Subject: [PATCH 09/88] Fix failing tests --- kernel/src/log_segment.rs | 4 ++-- kernel/src/scan/mod.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index b473f660a..df65ee293 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -103,8 +103,8 @@ impl LogSegment { use crate::Expression as Expr; static META_PREDICATE: LazyLock> = LazyLock::new(|| { Some(Arc::new(Expr::or( - column_expr!("metaData.id").is_not_null(), - column_expr!("protocol.minReaderVersion").is_not_null(), + Expr::column([METADATA_NAME, "id"]).is_not_null(), + Expr::column([PROTOCOL_NAME, "minReaderVersion"]).is_not_null(), ))) }); // read the same protocol and metadata schema for both commits and checkpoints diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index 12afa0511..c1edf8ac6 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -495,7 +495,7 @@ fn transform_to_logical_internal( parse_partition_value(partition_values.get(name), field.data_type())?; Ok::(value_expression.into()) } - ColumnType::Selected(field_name) => Ok(field_name.clone().into()), + ColumnType::Selected(field_name) => Ok(ColumnName::new([field_name]).into()), }) .try_collect()?; let read_expression = Expression::Struct(all_fields); From 1384ea3f18fe7f5b22cc1bd0b00cd8249d7dc81b Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 15:00:31 -0800 Subject: [PATCH 10/88] Apply suggestions from code review Co-authored-by: Ryan Johnson --- kernel/src/log_segment.rs | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index df65ee293..a8f9dd213 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -14,7 +14,6 @@ use itertools::Itertools; #[derive(Debug)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] -#[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] pub(crate) struct LogSegment { pub log_root: Url, /// Reverse order sorted commit files in the log segment @@ -62,16 +61,14 @@ impl LogSegment { )? .map_ok(|batch| (batch, false)); - let batches = commit_stream.chain(checkpoint_stream); - - Ok(batches) + Ok(commit_stream.chain(checkpoint_stream)) } // Get the most up-to-date Protocol and Metadata actions pub(crate) fn read_metadata(&self, engine: &dyn Engine) -> DeltaResult<(Metadata, Protocol)> { let data_batches = self.replay_for_metadata(engine)?; - let mut metadata_opt: Option = None; - let mut protocol_opt: Option = None; + let mut metadata_opt = None; + let mut protocol_opt = None; for batch in data_batches { let (batch, _) = batch?; if metadata_opt.is_none() { @@ -80,15 +77,13 @@ impl LogSegment { if protocol_opt.is_none() { protocol_opt = crate::actions::Protocol::try_new_from_data(batch.as_ref())?; } - if metadata_opt.is_some() && protocol_opt.is_some() { - // we've found both, we can stop - break; + if let (Some(m), Some(p)) = (metadata_opt, protocol_opt) { + return Ok((m, p)) } } match (metadata_opt, protocol_opt) { - (Some(m), Some(p)) => Ok((m, p)), - (None, Some(_)) => Err(Error::MissingMetadata), - (Some(_), None) => Err(Error::MissingProtocol), + (_, Some(_)) => Err(Error::MissingMetadata), + (Some(_), _) => Err(Error::MissingProtocol), _ => Err(Error::MissingMetadataAndProtocol), } } From 01823260a3ea8d20d648b6462a982472c2ce692d Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 15:02:23 -0800 Subject: [PATCH 11/88] Address more pr comments --- kernel/src/log_segment.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index a8f9dd213..56cc86102 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -72,10 +72,10 @@ impl LogSegment { for batch in data_batches { let (batch, _) = batch?; if metadata_opt.is_none() { - metadata_opt = crate::actions::Metadata::try_new_from_data(batch.as_ref())?; + metadata_opt = Metadata::try_new_from_data(batch.as_ref())?; } if protocol_opt.is_none() { - protocol_opt = crate::actions::Protocol::try_new_from_data(batch.as_ref())?; + protocol_opt = Protocol::try_new_from_data(batch.as_ref())?; } if let (Some(m), Some(p)) = (metadata_opt, protocol_opt) { return Ok((m, p)) From d053a772eb1f294fc97950e9b120c8cefaaceb07 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 15:04:35 -0800 Subject: [PATCH 12/88] fix imports --- kernel/src/snapshot.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 65ffe3918..6aa8f20b4 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -9,7 +9,7 @@ use serde::{Deserialize, Serialize}; use tracing::{debug, warn}; use url::Url; -use crate::actions::{get_log_schema, Metadata, Protocol, METADATA_NAME, PROTOCOL_NAME}; +use crate::actions::{Metadata, Protocol}; use crate::features::{ColumnMappingMode, COLUMN_MAPPING_MODE_KEY}; use crate::log_segment::LogSegment; use crate::path::ParsedLogPath; From 52f57e52cd7efe77bd6da87e94e54756aa21e99d Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 15:07:25 -0800 Subject: [PATCH 13/88] rebase onto git changes --- kernel/src/log_segment.rs | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 56cc86102..5c6cd3757 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -1,7 +1,6 @@ //! Represents a segment of a delta log. [`LogSegment`] wraps a set of checkpoint and commit //! files. -use crate::expressions::column_expr; use std::sync::{Arc, LazyLock}; use url::Url; @@ -77,14 +76,16 @@ impl LogSegment { if protocol_opt.is_none() { protocol_opt = Protocol::try_new_from_data(batch.as_ref())?; } - if let (Some(m), Some(p)) = (metadata_opt, protocol_opt) { - return Ok((m, p)) + if metadata_opt.is_some() && protocol_opt.is_some() { + // we've found both, we can stop + break; } } match (metadata_opt, protocol_opt) { - (_, Some(_)) => Err(Error::MissingMetadata), - (Some(_), _) => Err(Error::MissingProtocol), - _ => Err(Error::MissingMetadataAndProtocol), + (Some(m), Some(p)) => Ok((m, p)), + (None, Some(_)) => Err(Error::MissingMetadata), + (Some(_), None) => Err(Error::MissingProtocol), + (None, None) => Err(Error::MissingMetadataAndProtocol), } } From aa6c9f46537a40d071b841d0cd1b9fc112417e4e Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 15:23:29 -0800 Subject: [PATCH 14/88] address nits --- kernel/src/scan/mod.rs | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/kernel/src/scan/mod.rs b/kernel/src/scan/mod.rs index c1edf8ac6..bcbef1f53 100644 --- a/kernel/src/scan/mod.rs +++ b/kernel/src/scan/mod.rs @@ -480,20 +480,16 @@ fn transform_to_logical_internal( .iter() .map(|field| match field { ColumnType::Partition(field_idx) => { - let field = global_state - .logical_schema - .fields - .get_index(*field_idx) - .ok_or_else(|| { - Error::generic( - "logical schema did not contain expected field, can't transform data", - ) - })? - .1; + let field = global_state.logical_schema.fields.get_index(*field_idx); + let Some((_, field)) = field else { + return Err(Error::generic( + "logical schema did not contain expected field, can't transform data", + )); + }; let name = field.physical_name(global_state.column_mapping_mode)?; let value_expression = parse_partition_value(partition_values.get(name), field.data_type())?; - Ok::(value_expression.into()) + Ok(value_expression.into()) } ColumnType::Selected(field_name) => Ok(ColumnName::new([field_name]).into()), }) From dca491cd382eb1bdb465420df75c89ce09b3cf47 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 15:27:55 -0800 Subject: [PATCH 15/88] fix visibility issue --- kernel/src/log_segment.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 5c6cd3757..74d3d9d87 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -34,7 +34,6 @@ impl LogSegment { /// `meta_predicate` is an optional expression to filter the log files with. It is _NOT_ the /// query's predicate, but rather a predicate for filtering log files themselves. #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] - #[cfg_attr(not(feature = "developer-visibility"), visibility::make(pub(crate)))] pub(crate) fn replay( &self, engine: &dyn Engine, From bf5cdd432bbf64cf27a7954541d98d922bfe1247 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Tue, 29 Oct 2024 22:41:27 -0700 Subject: [PATCH 16/88] Use LogSegmentBuilder --- kernel/src/log_segment.rs | 201 +++++++++++++++++++++++++++++++++++++- kernel/src/snapshot.rs | 51 ++-------- 2 files changed, 207 insertions(+), 45 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 74d3d9d87..9183593ee 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -1,7 +1,14 @@ //! Represents a segment of a delta log. [`LogSegment`] wraps a set of checkpoint and commit //! files. -use std::sync::{Arc, LazyLock}; +use crate::{ + path::ParsedLogPath, snapshot::CheckpointMetadata, utils::require, FileSystemClient, Version, +}; +use std::{ + cmp::Ordering, + sync::{Arc, LazyLock}, +}; +use tracing::warn; use url::Url; use crate::{ @@ -14,6 +21,7 @@ use itertools::Itertools; #[derive(Debug)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) struct LogSegment { + pub version: Version, pub log_root: Url, /// Reverse order sorted commit files in the log segment pub commit_files: Vec, @@ -107,6 +115,197 @@ impl LogSegment { } } +pub struct LogSegmentBuilder<'a> { + fs_client: Arc, + log_root: &'a Url, + checkpoint: Option, + version: Option, + no_checkpoint: bool, +} +impl<'a> LogSegmentBuilder<'a> { + pub fn new(fs_client: Arc, log_root: &'a Url) -> Self { + LogSegmentBuilder { + fs_client, + log_root, + checkpoint: None, + version: None, + no_checkpoint: false, + } + } + + pub fn with_checkpoint(mut self, checkpoint: CheckpointMetadata) -> Self { + let _ = self.checkpoint.insert(checkpoint); + self + } + pub fn with_version(mut self, version: Version) -> Self { + let _ = self.version.insert(version); + self + } + pub fn with_no_checkpoint(mut self) -> Self { + self.no_checkpoint = true; + self + } + pub fn build(self) -> DeltaResult { + let Self { + fs_client, + log_root, + checkpoint, + version, + no_checkpoint, + } = self; + let log_url = log_root.join("_delta_log/").unwrap(); + let (mut commit_files, checkpoint_files) = match (checkpoint, version) { + (Some(cp), None) => { + Self::list_log_files_with_checkpoint(&cp, fs_client.as_ref(), &log_url)? + } + (Some(cp), Some(version)) if cp.version >= version => { + Self::list_log_files_with_checkpoint(&cp, fs_client.as_ref(), &log_url)? + } + _ => Self::list_log_files(fs_client.as_ref(), &log_url)?, + }; + + // remove all files above requested version + if let Some(version) = version { + commit_files.retain(|log_path| log_path.version <= version); + } + // only keep commit files above the checkpoint we found + if let Some(checkpoint_file) = checkpoint_files.first() { + commit_files.retain(|log_path| checkpoint_file.version < log_path.version); + } + + // get the effective version from chosen files + let version_eff = commit_files + .first() + .or(checkpoint_files.first()) + .ok_or(Error::MissingVersion)? // TODO: A more descriptive error + .version; + + if let Some(v) = version { + require!( + version_eff == v, + Error::MissingVersion // TODO more descriptive error + ); + } + + Ok(LogSegment { + version: version_eff, + log_root: log_url, + commit_files: commit_files + .into_iter() + .map(|log_path| log_path.location) + .collect(), + checkpoint_files: checkpoint_files + .into_iter() + .map(|log_path| log_path.location) + .collect(), + }) + } + pub fn list_log_files_from_version( + fs_client: &dyn FileSystemClient, + log_root: &Url, + version: Option, + ) -> DeltaResult<(Vec, Vec, i64)> { + let begin_version = version.unwrap_or(0); + let version_prefix = format!("{:020}", begin_version); + let start_from = log_root.join(&version_prefix)?; + + let mut max_checkpoint_version = version.map_or(-1, |x| x as i64); + let mut checkpoint_files = vec![]; + // We expect 10 commit files per checkpoint, so start with that size. We could adjust this based + // on config at some point + let mut commit_files = Vec::with_capacity(10); + + for meta_res in fs_client.list_from(&start_from)? { + let meta = meta_res?; + let parsed_path = ParsedLogPath::try_from(meta)?; + // TODO this filters out .crc files etc which start with "." - how do we want to use these kind of files? + if let Some(parsed_path) = parsed_path { + if parsed_path.is_commit() { + commit_files.push(parsed_path); + } else if parsed_path.is_checkpoint() { + let path_version = parsed_path.version as i64; + match path_version.cmp(&max_checkpoint_version) { + Ordering::Greater => { + max_checkpoint_version = path_version; + checkpoint_files.clear(); + checkpoint_files.push(parsed_path); + } + Ordering::Equal => checkpoint_files.push(parsed_path), + Ordering::Less => {} + } + } + } + } + + debug_assert!( + commit_files + .windows(2) + .all(|cfs| cfs[0].version <= cfs[1].version), + "fs_client.list_from() didn't return a sorted listing! {:?}", + commit_files + ); + + // We assume listing returned ordered, we want reverse order + let commit_files = commit_files.into_iter().rev().collect(); + + Ok((commit_files, checkpoint_files, max_checkpoint_version)) + } + + /// List all log files after a given checkpoint. + pub fn list_log_files_with_checkpoint( + checkpoint_metadata: &CheckpointMetadata, + fs_client: &dyn FileSystemClient, + log_root: &Url, + ) -> DeltaResult<(Vec, Vec)> { + let (mut commit_files, checkpoint_files, max_checkpoint_version) = + Self::list_log_files_from_version( + fs_client, + log_root, + Some(checkpoint_metadata.version), + )?; + + if checkpoint_files.is_empty() { + // TODO: We could potentially recover here + return Err(Error::generic( + "Had a _last_checkpoint hint but didn't find any checkpoints", + )); + } + + if max_checkpoint_version != checkpoint_metadata.version as i64 { + warn!( + "_last_checkpoint hint is out of date. _last_checkpoint version: {}. Using actual most recent: {}", + checkpoint_metadata.version, + max_checkpoint_version + ); + // we (may) need to drop commits that are before the _actual_ last checkpoint (that + // is, commits between a stale _last_checkpoint and the _actual_ last checkpoint) + commit_files.retain(|parsed_path| parsed_path.version as i64 > max_checkpoint_version); + } else if checkpoint_files.len() != checkpoint_metadata.parts.unwrap_or(1) { + return Err(Error::Generic(format!( + "_last_checkpoint indicated that checkpoint should have {} parts, but it has {}", + checkpoint_metadata.parts.unwrap_or(1), + checkpoint_files.len() + ))); + } + Ok((commit_files, checkpoint_files)) + } + + /// List relevant log files. + /// + /// Relevant files are the max checkpoint found and all subsequent commits. + pub fn list_log_files( + fs_client: &dyn FileSystemClient, + log_root: &Url, + ) -> DeltaResult<(Vec, Vec)> { + let (mut commit_files, checkpoint_files, max_checkpoint_version) = + Self::list_log_files_from_version(fs_client, log_root, None)?; + + commit_files.retain(|f| f.version as i64 > max_checkpoint_version); + + Ok((commit_files, checkpoint_files)) + } +} + #[cfg(test)] mod tests { use std::path::PathBuf; diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 6aa8f20b4..1bf55fc5e 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -11,7 +11,7 @@ use url::Url; use crate::actions::{Metadata, Protocol}; use crate::features::{ColumnMappingMode, COLUMN_MAPPING_MODE_KEY}; -use crate::log_segment::LogSegment; +use crate::log_segment::{self, LogSegment, LogSegmentBuilder}; use crate::path::ParsedLogPath; use crate::scan::ScanBuilder; use crate::schema::Schema; @@ -67,53 +67,16 @@ impl Snapshot { let fs_client = engine.get_file_system_client(); let log_url = table_root.join("_delta_log/").unwrap(); - // List relevant files from log - let (mut commit_files, checkpoint_files) = - match (read_last_checkpoint(fs_client.as_ref(), &log_url)?, version) { - (Some(cp), None) => { - list_log_files_with_checkpoint(&cp, fs_client.as_ref(), &log_url)? - } - (Some(cp), Some(version)) if cp.version >= version => { - list_log_files_with_checkpoint(&cp, fs_client.as_ref(), &log_url)? - } - _ => list_log_files(fs_client.as_ref(), &log_url)?, - }; - - // remove all files above requested version + let mut builder = LogSegmentBuilder::new(fs_client.clone(), &table_root); if let Some(version) = version { - commit_files.retain(|log_path| log_path.version <= version); + builder = builder.with_version(version); } - // only keep commit files above the checkpoint we found - if let Some(checkpoint_file) = checkpoint_files.first() { - commit_files.retain(|log_path| checkpoint_file.version < log_path.version); + if let Some(checkpoint) = read_last_checkpoint(fs_client.as_ref(), &log_url)? { + builder = builder.with_checkpoint(checkpoint); } + let log_segment = builder.build()?; - // get the effective version from chosen files - let version_eff = commit_files - .first() - .or(checkpoint_files.first()) - .ok_or(Error::MissingVersion)? // TODO: A more descriptive error - .version; - - if let Some(v) = version { - require!( - version_eff == v, - Error::MissingVersion // TODO more descriptive error - ); - } - - let log_segment = LogSegment { - log_root: log_url, - commit_files: commit_files - .into_iter() - .map(|log_path| log_path.location) - .collect(), - checkpoint_files: checkpoint_files - .into_iter() - .map(|log_path| log_path.location) - .collect(), - }; - + let version_eff = log_segment.version; Self::try_new_from_log_segment(table_root, log_segment, version_eff, engine) } From 748bab9d62e830c2c7b3699c0f52437cf69f736e Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Tue, 29 Oct 2024 22:48:37 -0700 Subject: [PATCH 17/88] Introduce start and end versions --- kernel/src/log_segment.rs | 46 ++++++++++++++++++++++++--------------- kernel/src/snapshot.rs | 2 +- 2 files changed, 29 insertions(+), 19 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 9183593ee..b7ef223ee 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -119,7 +119,8 @@ pub struct LogSegmentBuilder<'a> { fs_client: Arc, log_root: &'a Url, checkpoint: Option, - version: Option, + start_version: Option, + end_version: Option, no_checkpoint: bool, } impl<'a> LogSegmentBuilder<'a> { @@ -128,7 +129,8 @@ impl<'a> LogSegmentBuilder<'a> { fs_client, log_root, checkpoint: None, - version: None, + start_version: None, + end_version: None, no_checkpoint: false, } } @@ -137,11 +139,16 @@ impl<'a> LogSegmentBuilder<'a> { let _ = self.checkpoint.insert(checkpoint); self } - pub fn with_version(mut self, version: Version) -> Self { - let _ = self.version.insert(version); + + pub fn with_start_version(mut self, version: Version) -> Self { + let _ = self.start_version.insert(version); + self + } + pub fn with_end_version(mut self, version: Version) -> Self { + let _ = self.end_version.insert(version); self } - pub fn with_no_checkpoint(mut self) -> Self { + pub fn set_omit_checkpoints(mut self) -> Self { self.no_checkpoint = true; self } @@ -150,11 +157,12 @@ impl<'a> LogSegmentBuilder<'a> { fs_client, log_root, checkpoint, - version, + start_version, + end_version, no_checkpoint, } = self; let log_url = log_root.join("_delta_log/").unwrap(); - let (mut commit_files, checkpoint_files) = match (checkpoint, version) { + let (mut commit_files, checkpoint_files) = match (checkpoint, end_version) { (Some(cp), None) => { Self::list_log_files_with_checkpoint(&cp, fs_client.as_ref(), &log_url)? } @@ -164,15 +172,6 @@ impl<'a> LogSegmentBuilder<'a> { _ => Self::list_log_files(fs_client.as_ref(), &log_url)?, }; - // remove all files above requested version - if let Some(version) = version { - commit_files.retain(|log_path| log_path.version <= version); - } - // only keep commit files above the checkpoint we found - if let Some(checkpoint_file) = checkpoint_files.first() { - commit_files.retain(|log_path| checkpoint_file.version < log_path.version); - } - // get the effective version from chosen files let version_eff = commit_files .first() @@ -180,11 +179,22 @@ impl<'a> LogSegmentBuilder<'a> { .ok_or(Error::MissingVersion)? // TODO: A more descriptive error .version; - if let Some(v) = version { + // remove all files above requested version + if let Some(end_version) = end_version { require!( - version_eff == v, + version_eff == end_version, Error::MissingVersion // TODO more descriptive error ); + commit_files.retain(|log_path| log_path.version <= end_version); + } + if let Some(start_version) = start_version { + commit_files.retain(|log_path| log_path.version >= start_version); + } + // only keep commit files above the checkpoint we found + if no_checkpoint { + commit_files.clear(); + } else if let Some(checkpoint_file) = checkpoint_files.first() { + commit_files.retain(|log_path| checkpoint_file.version < log_path.version); } Ok(LogSegment { diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 1bf55fc5e..999d9941b 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -69,7 +69,7 @@ impl Snapshot { let mut builder = LogSegmentBuilder::new(fs_client.clone(), &table_root); if let Some(version) = version { - builder = builder.with_version(version); + builder = builder.with_end_version(version); } if let Some(checkpoint) = read_last_checkpoint(fs_client.as_ref(), &log_url)? { builder = builder.with_checkpoint(checkpoint); From 2a6eb3e24935a380f9c1efe5f3d2df92b0b0e097 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 1 Nov 2024 13:47:26 -0700 Subject: [PATCH 18/88] Remove old code --- kernel/src/snapshot.rs | 130 +---------------------------------------- 1 file changed, 1 insertion(+), 129 deletions(-) diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 999d9941b..43276ef90 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -2,7 +2,6 @@ //! has schema etc.) //! -use std::cmp::Ordering; use std::sync::Arc; use serde::{Deserialize, Serialize}; @@ -11,11 +10,9 @@ use url::Url; use crate::actions::{Metadata, Protocol}; use crate::features::{ColumnMappingMode, COLUMN_MAPPING_MODE_KEY}; -use crate::log_segment::{self, LogSegment, LogSegmentBuilder}; -use crate::path::ParsedLogPath; +use crate::log_segment::{LogSegment, LogSegmentBuilder}; use crate::scan::ScanBuilder; use crate::schema::Schema; -use crate::utils::require; use crate::{DeltaResult, Engine, Error, FileSystemClient, Version}; const LAST_CHECKPOINT_FILE_NAME: &str = "_last_checkpoint"; @@ -199,131 +196,6 @@ fn read_last_checkpoint( } } -/// List all log files after a given checkpoint. -fn list_log_files_with_checkpoint( - checkpoint_metadata: &CheckpointMetadata, - fs_client: &dyn FileSystemClient, - log_root: &Url, -) -> DeltaResult<(Vec, Vec)> { - let version_prefix = format!("{:020}", checkpoint_metadata.version); - let start_from = log_root.join(&version_prefix)?; - - let mut max_checkpoint_version = checkpoint_metadata.version; - let mut checkpoint_files = vec![]; - // We expect 10 commit files per checkpoint, so start with that size. We could adjust this based - // on config at some point - let mut commit_files = Vec::with_capacity(10); - - for meta_res in fs_client.list_from(&start_from)? { - let meta = meta_res?; - let parsed_path = ParsedLogPath::try_from(meta)?; - // TODO this filters out .crc files etc which start with "." - how do we want to use these kind of files? - if let Some(parsed_path) = parsed_path { - if parsed_path.is_commit() { - commit_files.push(parsed_path); - } else if parsed_path.is_checkpoint() { - match parsed_path.version.cmp(&max_checkpoint_version) { - Ordering::Greater => { - max_checkpoint_version = parsed_path.version; - checkpoint_files.clear(); - checkpoint_files.push(parsed_path); - } - Ordering::Equal => checkpoint_files.push(parsed_path), - Ordering::Less => {} - } - } - } - } - - if checkpoint_files.is_empty() { - // TODO: We could potentially recover here - return Err(Error::generic( - "Had a _last_checkpoint hint but didn't find any checkpoints", - )); - } - - if max_checkpoint_version != checkpoint_metadata.version { - warn!( - "_last_checkpoint hint is out of date. _last_checkpoint version: {}. Using actual most recent: {}", - checkpoint_metadata.version, - max_checkpoint_version - ); - // we (may) need to drop commits that are before the _actual_ last checkpoint (that - // is, commits between a stale _last_checkpoint and the _actual_ last checkpoint) - commit_files.retain(|parsed_path| parsed_path.version > max_checkpoint_version); - } else if checkpoint_files.len() != checkpoint_metadata.parts.unwrap_or(1) { - return Err(Error::Generic(format!( - "_last_checkpoint indicated that checkpoint should have {} parts, but it has {}", - checkpoint_metadata.parts.unwrap_or(1), - checkpoint_files.len() - ))); - } - - debug_assert!( - commit_files - .windows(2) - .all(|cfs| cfs[0].version <= cfs[1].version), - "fs_client.list_from() didn't return a sorted listing! {:?}", - commit_files - ); - // We assume listing returned ordered, we want reverse order - let commit_files = commit_files.into_iter().rev().collect(); - - Ok((commit_files, checkpoint_files)) -} - -/// List relevant log files. -/// -/// Relevant files are the max checkpoint found and all subsequent commits. -fn list_log_files( - fs_client: &dyn FileSystemClient, - log_root: &Url, -) -> DeltaResult<(Vec, Vec)> { - let version_prefix = format!("{:020}", 0); - let start_from = log_root.join(&version_prefix)?; - - let mut max_checkpoint_version = -1_i64; - let mut commit_files = Vec::new(); - let mut checkpoint_files = Vec::with_capacity(10); - - let log_paths = fs_client - .list_from(&start_from)? - .flat_map(|file| file.and_then(ParsedLogPath::try_from).transpose()); - for log_path in log_paths { - let log_path = log_path?; - if log_path.is_checkpoint() { - let version = log_path.version as i64; - match version.cmp(&max_checkpoint_version) { - Ordering::Greater => { - max_checkpoint_version = version; - checkpoint_files.clear(); - checkpoint_files.push(log_path); - } - Ordering::Equal => { - checkpoint_files.push(log_path); - } - _ => {} - } - } else if log_path.is_commit() { - commit_files.push(log_path); - } - } - - commit_files.retain(|f| f.version as i64 > max_checkpoint_version); - - debug_assert!( - commit_files - .windows(2) - .all(|cfs| cfs[0].version <= cfs[1].version), - "fs_client.list_from() didn't return a sorted listing! {:?}", - commit_files - ); - // We assume listing returned ordered, we want reverse order - let commit_files = commit_files.into_iter().rev().collect(); - - Ok((commit_files, checkpoint_files)) -} - #[cfg(test)] mod tests { use super::*; From 841f17f0d1bfd6cdd30dbaf551ef0849796f4b06 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 1 Nov 2024 14:07:28 -0700 Subject: [PATCH 19/88] Fix failing tests --- kernel/src/log_segment.rs | 105 +++++++++++++++++++++++++++++++----- kernel/src/snapshot.rs | 68 +---------------------- kernel/src/table_changes.rs | 41 ++++++++++++++ 3 files changed, 134 insertions(+), 80 deletions(-) create mode 100644 kernel/src/table_changes.rs diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index b7ef223ee..f2fd088e3 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -172,6 +172,16 @@ impl<'a> LogSegmentBuilder<'a> { _ => Self::list_log_files(fs_client.as_ref(), &log_url)?, }; + // remove all files above requested version + if let Some(end_version) = end_version { + commit_files.retain(|log_path| log_path.version <= end_version); + } + + // only keep commit files above the checkpoint we found + if let Some(checkpoint_file) = checkpoint_files.first() { + commit_files.retain(|log_path| checkpoint_file.version < log_path.version); + } + // get the effective version from chosen files let version_eff = commit_files .first() @@ -179,22 +189,11 @@ impl<'a> LogSegmentBuilder<'a> { .ok_or(Error::MissingVersion)? // TODO: A more descriptive error .version; - // remove all files above requested version if let Some(end_version) = end_version { require!( version_eff == end_version, Error::MissingVersion // TODO more descriptive error ); - commit_files.retain(|log_path| log_path.version <= end_version); - } - if let Some(start_version) = start_version { - commit_files.retain(|log_path| log_path.version >= start_version); - } - // only keep commit files above the checkpoint we found - if no_checkpoint { - commit_files.clear(); - } else if let Some(checkpoint_file) = checkpoint_files.first() { - commit_files.retain(|log_path| checkpoint_file.version < log_path.version); } Ok(LogSegment { @@ -318,11 +317,23 @@ impl<'a> LogSegmentBuilder<'a> { #[cfg(test)] mod tests { - use std::path::PathBuf; + use std::{path::PathBuf, sync::Arc}; use itertools::Itertools; + use object_store::{memory::InMemory, path::Path, ObjectStore}; + use url::Url; - use crate::{engine::sync::SyncEngine, Table}; + use crate::{ + engine::{ + default::{ + executor::tokio::TokioBackgroundExecutor, filesystem::ObjectStoreFileSystemClient, + }, + sync::SyncEngine, + }, + log_segment::LogSegmentBuilder, + snapshot::CheckpointMetadata, + Table, + }; // NOTE: In addition to testing the meta-predicate for metadata replay, this test also verifies // that the parquet reader properly infers nullcount = rowcount for missing columns. The two @@ -366,4 +377,72 @@ mod tests { // missing columns, but can still skip part 3 because has valid nullcount stats for P&M. assert_eq!(data.len(), 4); } + + #[test] + fn test_read_log_with_out_of_date_last_checkpoint() { + let store = Arc::new(InMemory::new()); + + fn get_path(index: usize, suffix: &str) -> Path { + let path = format!("_delta_log/{index:020}.{suffix}"); + Path::from(path.as_str()) + } + let data = bytes::Bytes::from("kernel-data"); + + let checkpoint_metadata = CheckpointMetadata { + version: 3, + size: 10, + parts: None, + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + + // add log files to store + tokio::runtime::Runtime::new() + .expect("create tokio runtime") + .block_on(async { + for path in [ + get_path(0, "json"), + get_path(1, "checkpoint.parquet"), + get_path(2, "json"), + get_path(3, "checkpoint.parquet"), + get_path(4, "json"), + get_path(5, "checkpoint.parquet"), + get_path(6, "json"), + get_path(7, "json"), + ] { + store + .put(&path, data.clone().into()) + .await + .expect("put log file in store"); + } + let checkpoint_str = + serde_json::to_string(&checkpoint_metadata).expect("Serialize checkpoint"); + store + .put( + &Path::from("_delta_log/_last_checkpoint"), + checkpoint_str.into(), + ) + .await + .expect("Write _last_checkpoint"); + }); + + let client = ObjectStoreFileSystemClient::new( + store, + false, // don't have ordered listing + Path::from("/"), + Arc::new(TokioBackgroundExecutor::new()), + ); + + let url = Url::parse("memory:///_delta_log/").expect("valid url"); + let (commit_files, checkpoint_files) = + LogSegmentBuilder::list_log_files_with_checkpoint(&checkpoint_metadata, &client, &url) + .unwrap(); + assert_eq!(checkpoint_files.len(), 1); + assert_eq!(commit_files.len(), 2); + assert_eq!(checkpoint_files[0].version, 5); + assert_eq!(commit_files[0].version, 7); + assert_eq!(commit_files[1].version, 6); + } } diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 43276ef90..baf6045ce 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -211,6 +211,7 @@ mod tests { use crate::engine::default::executor::tokio::TokioBackgroundExecutor; use crate::engine::default::filesystem::ObjectStoreFileSystemClient; use crate::engine::sync::SyncEngine; + use crate::path::ParsedLogPath; use crate::schema::StructType; #[test] @@ -277,73 +278,6 @@ mod tests { assert!(cp.is_none()) } - #[test] - fn test_read_log_with_out_of_date_last_checkpoint() { - let store = Arc::new(InMemory::new()); - - fn get_path(index: usize, suffix: &str) -> Path { - let path = format!("_delta_log/{index:020}.{suffix}"); - Path::from(path.as_str()) - } - let data = bytes::Bytes::from("kernel-data"); - - let checkpoint_metadata = CheckpointMetadata { - version: 3, - size: 10, - parts: None, - size_in_bytes: None, - num_of_add_files: None, - checkpoint_schema: None, - checksum: None, - }; - - // add log files to store - tokio::runtime::Runtime::new() - .expect("create tokio runtime") - .block_on(async { - for path in [ - get_path(0, "json"), - get_path(1, "checkpoint.parquet"), - get_path(2, "json"), - get_path(3, "checkpoint.parquet"), - get_path(4, "json"), - get_path(5, "checkpoint.parquet"), - get_path(6, "json"), - get_path(7, "json"), - ] { - store - .put(&path, data.clone().into()) - .await - .expect("put log file in store"); - } - let checkpoint_str = - serde_json::to_string(&checkpoint_metadata).expect("Serialize checkpoint"); - store - .put( - &Path::from("_delta_log/_last_checkpoint"), - checkpoint_str.into(), - ) - .await - .expect("Write _last_checkpoint"); - }); - - let client = ObjectStoreFileSystemClient::new( - store, - false, // don't have ordered listing - Path::from("/"), - Arc::new(TokioBackgroundExecutor::new()), - ); - - let url = Url::parse("memory:///_delta_log/").expect("valid url"); - let (commit_files, checkpoint_files) = - list_log_files_with_checkpoint(&checkpoint_metadata, &client, &url).unwrap(); - assert_eq!(checkpoint_files.len(), 1); - assert_eq!(commit_files.len(), 2); - assert_eq!(checkpoint_files[0].version, 5); - assert_eq!(commit_files[0].version, 7); - assert_eq!(commit_files[1].version, 6); - } - fn valid_last_checkpoint() -> Vec { r#"{"size":8,"size_in_bytes":21857,"version":1}"#.as_bytes().to_vec() } diff --git a/kernel/src/table_changes.rs b/kernel/src/table_changes.rs new file mode 100644 index 000000000..7ac51f584 --- /dev/null +++ b/kernel/src/table_changes.rs @@ -0,0 +1,41 @@ +//! In-memory representation of a change data feed table. + +use url::Url; + +use crate::{ + log_segment::{LogSegment, LogSegmentBuilder}, + path::AsUrl, + schema::Schema, + snapshot::Snapshot, + DeltaResult, Engine, Version, +}; + +#[derive(Debug)] +pub struct TableChanges { + snapshot: Snapshot, + cdf_range: LogSegment, + schema: Schema, +} + +impl TableChanges { + pub fn try_new( + table_root: Url, + engine: &dyn Engine, + start_version: Version, + end_version: Option, + ) -> DeltaResult { + let fs_client = engine.get_file_system_client(); + let snapshot = Snapshot::try_new(table_root.as_url().clone(), engine, Some(start_version))?; + let mut builder = LogSegmentBuilder::new(fs_client, &table_root); + builder = builder.with_start_version(start_version); + if let Some(end_version) = end_version { + builder = builder.with_start_version(end_version); + } + let log_segment = builder.build()?; + + Ok(TableChanges { + snapshot, + cdf_range: log_segment, + }) + } +} From 1ce29d80838296e871ee0fe65e3fd4f928e185d8 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 11:58:39 -0800 Subject: [PATCH 20/88] Most up to date logsegment --- kernel/src/log_segment.rs | 58 ++++++++++++++++++++++++++++++++++----- 1 file changed, 51 insertions(+), 7 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index f2fd088e3..d81435e28 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -70,6 +70,23 @@ impl LogSegment { Ok(commit_stream.chain(checkpoint_stream)) } + pub(crate) fn replay_commits( + &self, + engine: &dyn Engine, + commit_read_schema: SchemaRef, + meta_predicate: Option, + ) -> DeltaResult< + impl Iterator< + Item = DeltaResult>> + Send>>, + >, + > { + let json_client = engine.get_json_handler(); + let commit_files = self.commit_files.clone(); + let commit_stream = commit_files.into_iter().map(move |file| { + json_client.read_json_files(&[file], commit_read_schema.clone(), meta_predicate.clone()) + }); + Ok(commit_stream) + } // Get the most up-to-date Protocol and Metadata actions pub(crate) fn read_metadata(&self, engine: &dyn Engine) -> DeltaResult<(Metadata, Protocol)> { let data_batches = self.replay_for_metadata(engine)?; @@ -121,7 +138,8 @@ pub struct LogSegmentBuilder<'a> { checkpoint: Option, start_version: Option, end_version: Option, - no_checkpoint: bool, + no_checkpoint_files: bool, + in_order_commit_files: bool, } impl<'a> LogSegmentBuilder<'a> { pub fn new(fs_client: Arc, log_root: &'a Url) -> Self { @@ -131,7 +149,8 @@ impl<'a> LogSegmentBuilder<'a> { checkpoint: None, start_version: None, end_version: None, - no_checkpoint: false, + no_checkpoint_files: false, + in_order_commit_files: false, } } @@ -148,8 +167,12 @@ impl<'a> LogSegmentBuilder<'a> { let _ = self.end_version.insert(version); self } - pub fn set_omit_checkpoints(mut self) -> Self { - self.no_checkpoint = true; + pub fn with_no_checkpoint_files(mut self) -> Self { + self.no_checkpoint_files = true; + self + } + pub fn with_in_order_commit_files(mut self) -> Self { + self.in_order_commit_files = true; self } pub fn build(self) -> DeltaResult { @@ -159,10 +182,11 @@ impl<'a> LogSegmentBuilder<'a> { checkpoint, start_version, end_version, - no_checkpoint, + no_checkpoint_files, + in_order_commit_files, } = self; let log_url = log_root.join("_delta_log/").unwrap(); - let (mut commit_files, checkpoint_files) = match (checkpoint, end_version) { + let (mut commit_files, mut checkpoint_files) = match (checkpoint, end_version) { (Some(cp), None) => { Self::list_log_files_with_checkpoint(&cp, fs_client.as_ref(), &log_url)? } @@ -172,16 +196,31 @@ impl<'a> LogSegmentBuilder<'a> { _ => Self::list_log_files(fs_client.as_ref(), &log_url)?, }; + if !in_order_commit_files { + // We assume listing returned ordered, we want reverse order + commit_files.reverse(); + } + // remove all files above requested version if let Some(end_version) = end_version { commit_files.retain(|log_path| log_path.version <= end_version); } + // Remove checkpoint files + if no_checkpoint_files { + checkpoint_files.clear(); + } + // only keep commit files above the checkpoint we found if let Some(checkpoint_file) = checkpoint_files.first() { commit_files.retain(|log_path| checkpoint_file.version < log_path.version); } + // only keep commit files above the checkpoint we found + if let Some(start_version) = start_version { + commit_files.retain(|log_path| start_version <= log_path.version); + } + // get the effective version from chosen files let version_eff = commit_files .first() @@ -436,12 +475,17 @@ mod tests { ); let url = Url::parse("memory:///_delta_log/").expect("valid url"); - let (commit_files, checkpoint_files) = + let (mut commit_files, checkpoint_files) = LogSegmentBuilder::list_log_files_with_checkpoint(&checkpoint_metadata, &client, &url) .unwrap(); + + // Make the most recent commit the first in iterator + commit_files.reverse(); + assert_eq!(checkpoint_files.len(), 1); assert_eq!(commit_files.len(), 2); assert_eq!(checkpoint_files[0].version, 5); + println!("commitfiles: {:?}", commit_files); assert_eq!(commit_files[0].version, 7); assert_eq!(commit_files[1].version, 6); } From a2f9810bc835967854983e8fcb9b5ba2c9bc13ae Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 12:07:21 -0800 Subject: [PATCH 21/88] Fix failing test and remove unnecessary code --- kernel/src/log_segment.rs | 21 +-------------------- 1 file changed, 1 insertion(+), 20 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index d81435e28..d0dd00f4e 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -70,23 +70,6 @@ impl LogSegment { Ok(commit_stream.chain(checkpoint_stream)) } - pub(crate) fn replay_commits( - &self, - engine: &dyn Engine, - commit_read_schema: SchemaRef, - meta_predicate: Option, - ) -> DeltaResult< - impl Iterator< - Item = DeltaResult>> + Send>>, - >, - > { - let json_client = engine.get_json_handler(); - let commit_files = self.commit_files.clone(); - let commit_stream = commit_files.into_iter().map(move |file| { - json_client.read_json_files(&[file], commit_read_schema.clone(), meta_predicate.clone()) - }); - Ok(commit_stream) - } // Get the most up-to-date Protocol and Metadata actions pub(crate) fn read_metadata(&self, engine: &dyn Engine) -> DeltaResult<(Metadata, Protocol)> { let data_batches = self.replay_for_metadata(engine)?; @@ -196,6 +179,7 @@ impl<'a> LogSegmentBuilder<'a> { _ => Self::list_log_files(fs_client.as_ref(), &log_url)?, }; + // We assume listing returned ordered. If `in_order_commit_files` is false, we want reverse order. if !in_order_commit_files { // We assume listing returned ordered, we want reverse order commit_files.reverse(); @@ -293,9 +277,6 @@ impl<'a> LogSegmentBuilder<'a> { commit_files ); - // We assume listing returned ordered, we want reverse order - let commit_files = commit_files.into_iter().rev().collect(); - Ok((commit_files, checkpoint_files, max_checkpoint_version)) } From 5f7a6808398785b99832256541a12c5c7f989252 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 12:55:31 -0800 Subject: [PATCH 22/88] remove table changes from this commit --- kernel/src/table_changes/metadata_scanner.rs | 168 ++++++++ kernel/src/table_changes/mod.rs | 86 ++++ kernel/src/table_changes/replay_scanner.rs | 171 ++++++++ kernel/src/table_changes/state.rs | 127 ++++++ .../src/table_changes/table_changes_scan.rs | 371 ++++++++++++++++++ 5 files changed, 923 insertions(+) create mode 100644 kernel/src/table_changes/metadata_scanner.rs create mode 100644 kernel/src/table_changes/mod.rs create mode 100644 kernel/src/table_changes/replay_scanner.rs create mode 100644 kernel/src/table_changes/state.rs create mode 100644 kernel/src/table_changes/table_changes_scan.rs diff --git a/kernel/src/table_changes/metadata_scanner.rs b/kernel/src/table_changes/metadata_scanner.rs new file mode 100644 index 000000000..55c765aa2 --- /dev/null +++ b/kernel/src/table_changes/metadata_scanner.rs @@ -0,0 +1,168 @@ +use std::collections::HashSet; + +use tracing::debug; + +use crate::actions::visitors::{AddVisitor, RemoveVisitor}; +use crate::actions::{get_log_schema, Add, Remove, ADD_NAME, REMOVE_NAME}; +use crate::engine_data::{GetData, TypedGetData}; +use crate::expressions::{column_expr, Expression}; +use crate::scan::data_skipping::DataSkippingFilter; +use crate::scan::log_replay::SCAN_ROW_DATATYPE; +use crate::scan::ScanData; +use crate::{DataVisitor, DeltaResult, EngineData, ExpressionHandler}; + +#[derive(Default)] +pub(crate) struct CdcVisitor { + pub adds: Vec<(Add, usize)>, + pub removes: Vec, + selection_vector: Option>, +} + +const ADD_FIELD_COUNT: usize = 15; + +impl CdcVisitor { + pub(crate) fn new(selection_vector: Option>) -> Self { + CdcVisitor { + selection_vector, + ..Default::default() + } + } +} + +impl DataVisitor for CdcVisitor { + fn visit<'a>(&mut self, row_count: usize, getters: &[&'a dyn GetData<'a>]) -> DeltaResult<()> { + for i in 0..row_count { + // Add will have a path at index 0 if it is valid + if let Some(path) = getters[0].get_opt(i, "add.path")? { + // Keep the file unless the selection vector is present and is false for this row + if !self + .selection_vector + .as_ref() + .is_some_and(|selection| !selection[i]) + { + self.adds.push(( + AddVisitor::visit_add(i, path, &getters[..ADD_FIELD_COUNT])?, + i, + )) + } + } + // Remove will have a path at index 15 if it is valid + // TODO(nick): Should count the fields in Add to ensure we don't get this wrong if more + // are added + // TODO(zach): add a check for selection vector that we never skip a remove + else if let Some(path) = getters[ADD_FIELD_COUNT].get_opt(i, "remove.path")? { + let remove_getters = &getters[ADD_FIELD_COUNT..]; + self.removes + .push(RemoveVisitor::visit_remove(i, path, remove_getters)?); + } + } + Ok(()) + } +} + +pub(crate) struct CdcLogReplayScanner<'a> { + filter: Option<&'a DataSkippingFilter>, + + /// A set of (data file path, dv_unique_id) pairs that have been seen thus + /// far in the log. This is used to filter out files with Remove actions as + /// well as duplicate entries in the log. + seen: HashSet<(String, Option)>, +} + +impl<'a> CdcLogReplayScanner<'a> { + /// Create a new [`LogReplayScanner`] instance + pub(crate) fn new(filter: Option<&'a DataSkippingFilter>) -> Self { + Self { + filter, + seen: Default::default(), + } + } + + fn get_add_transform_expr(&self) -> Expression { + Expression::Struct(vec![ + column_expr!("add.path"), + column_expr!("add.size"), + column_expr!("add.modificationTime"), + column_expr!("add.stats"), + column_expr!("add.deletionVector"), + Expression::Struct(vec![column_expr!("add.partitionValues")]), + ]) + } + + pub(crate) fn process_scan_batch( + &mut self, + expression_handler: &dyn ExpressionHandler, + actions: &dyn EngineData, + ) -> DeltaResult { + // apply data skipping to get back a selection vector for actions that passed skipping + // note: None implies all files passed data skipping. + let filter_vector = self + .filter + .map(|filter| filter.apply(actions)) + .transpose()?; + + // we start our selection vector based on what was filtered. we will add to this vector + // below if a file has been removed + let mut selection_vector = match filter_vector { + Some(ref filter_vector) => filter_vector.clone(), + None => vec![false; actions.length()], + }; + + assert_eq!(selection_vector.len(), actions.length()); + let adds = self.setup_batch_process(filter_vector, actions)?; + + for (add, index) in adds.into_iter() { + // Note: each (add.path + add.dv_unique_id()) pair has a + // unique Add + Remove pair in the log. For example: + // https://github.com/delta-io/delta/blob/master/spark/src/test/resources/delta/table-with-dv-large/_delta_log/00000000000000000001.json + if !self.seen.contains(&(add.path.clone(), add.dv_unique_id())) { + debug!( + "Including file in scan: ({}, {:?})", + add.path, + add.dv_unique_id(), + ); + // Remember file actions from this batch so we can ignore duplicates + // as we process batches from older commit and/or checkpoint files. We + // don't need to track checkpoint batches because they are already the + // oldest actions and can never replace anything. + self.seen.insert((add.path.clone(), add.dv_unique_id())); + selection_vector[index] = true; + } else { + debug!("Filtering out Add due to it being removed {}", add.path); + // we may have a true here because the data-skipping predicate included the file + selection_vector[index] = false; + } + } + + let result = expression_handler + .get_evaluator( + get_log_schema().project(&[ADD_NAME])?, + self.get_add_transform_expr(), + SCAN_ROW_DATATYPE.clone(), + ) + .evaluate(actions)?; + Ok((result, selection_vector)) + } + + // work shared between process_batch and process_scan_batch + fn setup_batch_process( + &mut self, + selection_vector: Option>, + actions: &dyn EngineData, + ) -> DeltaResult> { + let schema_to_use = + // NB: We _must_ pass these in the order `ADD_NAME, REMOVE_NAME` as the visitor assumes + // the Add action comes first. The [`project`] method honors this order, so this works + // as long as we keep this order here. + get_log_schema().project(&[ADD_NAME, REMOVE_NAME])?; + let mut visitor = CdcVisitor::new(selection_vector); + actions.extract(schema_to_use, &mut visitor)?; + + for remove in visitor.removes.into_iter() { + let dv_id = remove.dv_unique_id(); + self.seen.insert((remove.path, dv_id)); + } + + Ok(visitor.adds) + } +} diff --git a/kernel/src/table_changes/mod.rs b/kernel/src/table_changes/mod.rs new file mode 100644 index 000000000..debf9cf61 --- /dev/null +++ b/kernel/src/table_changes/mod.rs @@ -0,0 +1,86 @@ +//! In-memory representation of a change data feed table. + +use std::{collections::HashMap, sync::Arc}; + +use table_changes_scan::TableChangesScanBuilder; +use url::Url; + +use crate::{ + actions::{Metadata, Protocol}, + features::ColumnMappingMode, + log_segment::{LogSegment, LogSegmentBuilder}, + path::AsUrl, + scan::state::DvInfo, + schema::Schema, + snapshot::Snapshot, + DeltaResult, Engine, EngineData, Error, Version, +}; + +mod metadata_scanner; +mod replay_scanner; +mod state; +pub mod table_changes_scan; + +pub type TableChangesScanData = (Box, Vec, Arc>); + +static CDF_ENABLE_FLAG: &str = "delta.enableChangeDataFeed"; + +#[derive(Debug)] +pub struct TableChanges { + pub snapshot: Snapshot, + pub log_segment: LogSegment, + pub schema: Schema, + pub version: Version, + pub metadata: Metadata, + pub protocol: Protocol, + pub(crate) column_mapping_mode: ColumnMappingMode, + pub table_root: Url, +} + +impl TableChanges { + pub fn try_new( + table_root: Url, + engine: &dyn Engine, + start_version: Version, + end_version: Option, + ) -> DeltaResult { + let start_snapshot = + Snapshot::try_new(table_root.as_url().clone(), engine, Some(start_version))?; + let end_snapshot = Snapshot::try_new(table_root.as_url().clone(), engine, end_version)?; + + let start_flag = start_snapshot.metadata().configuration.get(CDF_ENABLE_FLAG); + let end_flag = end_snapshot.metadata().configuration.get(CDF_ENABLE_FLAG); + + // Verify CDF is enabled at the beginning and end of the interval + let is_valid_flag = |flag_res: Option<&String>| flag_res.is_some_and(|val| val == "true"); + if !is_valid_flag(start_flag) || !is_valid_flag(end_flag) { + return Err(Error::TableChangesDisabled(start_version, end_version)); + } + + // Get a log segment for the CDF range + let fs_client = engine.get_file_system_client(); + let mut builder = LogSegmentBuilder::new(fs_client, &table_root); + builder = builder.with_start_version(start_version); + if let Some(end_version) = end_version { + builder = builder.with_end_version(end_version); + } + builder = builder + .with_no_checkpoint_files() + .with_in_order_commit_files(); + let log_segment = builder.build()?; + + Ok(TableChanges { + snapshot: start_snapshot, + log_segment, + schema: end_snapshot.schema().clone(), + column_mapping_mode: end_snapshot.column_mapping_mode, + version: end_snapshot.version(), + protocol: end_snapshot.protocol().clone(), + metadata: end_snapshot.metadata().clone(), + table_root, + }) + } + pub fn into_scan_builder(self) -> TableChangesScanBuilder { + TableChangesScanBuilder::new(self) + } +} diff --git a/kernel/src/table_changes/replay_scanner.rs b/kernel/src/table_changes/replay_scanner.rs new file mode 100644 index 000000000..0de5e6f73 --- /dev/null +++ b/kernel/src/table_changes/replay_scanner.rs @@ -0,0 +1,171 @@ +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; + +use tracing::debug; + +use crate::actions::deletion_vector::DeletionVectorDescriptor; +use crate::actions::visitors::{AddVisitor, RemoveVisitor}; +use crate::actions::{get_log_schema, Add, Remove, ADD_NAME, REMOVE_NAME}; +use crate::engine_data::{GetData, TypedGetData}; +use crate::expressions::{column_expr, Expression}; +use crate::scan::data_skipping::DataSkippingFilter; +use crate::scan::log_replay::SCAN_ROW_DATATYPE; +use crate::scan::state::DvInfo; +use crate::scan::ScanData; +use crate::{DataVisitor, DeltaResult, EngineData, ExpressionHandler}; + +use super::TableChangesScanData; + +#[derive(Default)] +pub(crate) struct AddRemoveCdcVisitor { + pub adds: Vec<(Add, usize)>, + pub removes: Vec<(Remove, usize)>, + selection_vector: Option>, +} + +const ADD_FIELD_COUNT: usize = 15; + +impl AddRemoveCdcVisitor { + pub(crate) fn new(selection_vector: Option>) -> Self { + AddRemoveCdcVisitor { + selection_vector, + ..Default::default() + } + } +} + +impl DataVisitor for AddRemoveCdcVisitor { + fn visit<'a>(&mut self, row_count: usize, getters: &[&'a dyn GetData<'a>]) -> DeltaResult<()> { + for i in 0..row_count { + // Add will have a path at index 0 if it is valid + if let Some(path) = getters[0].get_opt(i, "add.path")? { + // Keep the file unless the selection vector is present and is false for this row + if !self + .selection_vector + .as_ref() + .is_some_and(|selection| !selection[i]) + { + self.adds.push(( + AddVisitor::visit_add(i, path, &getters[..ADD_FIELD_COUNT])?, + i, + )) + } + } + // Remove will have a path at index 15 if it is valid + // TODO(nick): Should count the fields in Add to ensure we don't get this wrong if more + // are added + // TODO(zach): add a check for selection vector that we never skip a remove + else if let Some(path) = getters[ADD_FIELD_COUNT].get_opt(i, "remove.path")? { + let remove_getters = &getters[ADD_FIELD_COUNT..]; + self.removes + .push((RemoveVisitor::visit_remove(i, path, remove_getters)?, i)); + } + } + Ok(()) + } +} + +pub(crate) struct TableChangesLogReplayScanner { + filter: Option, + pub remove_dvs: HashMap, +} + +impl TableChangesLogReplayScanner { + /// Create a new [`LogReplayScanner`] instance + pub(crate) fn new(filter: Option) -> Self { + Self { + filter, + remove_dvs: Default::default(), + } + } + + fn get_add_transform_expr(&self) -> Expression { + Expression::Struct(vec![ + column_expr!("add.path"), + column_expr!("add.size"), + column_expr!("add.modificationTime"), + column_expr!("add.stats"), + column_expr!("add.deletionVector"), + Expression::Struct(vec![column_expr!("add.partitionValues")]), + ]) + } + + pub(crate) fn process_scan_batch( + &mut self, + expression_handler: &dyn ExpressionHandler, + actions: &dyn EngineData, + ) -> DeltaResult { + // apply data skipping to get back a selection vector for actions that passed skipping + // note: None implies all files passed data skipping. + let filter_vector = self + .filter + .as_ref() + .map(|filter| filter.apply(actions)) + .transpose()?; + + // we start our selection vector based on what was filtered. we will add to this vector + // below if a file has been removed + let mut selection_vector = match filter_vector { + Some(ref filter_vector) => filter_vector.clone(), + None => vec![false; actions.length()], + }; + + assert_eq!(selection_vector.len(), actions.length()); + let AddRemoveCdcVisitor { + adds, + removes, + selection_vector: _, + } = self.setup_batch_process(filter_vector, actions)?; + for (add, index) in adds.into_iter() { + // Note: each (add.path + add.dv_unique_id()) pair has a + // unique Add + Remove pair in the log. For example: + // https://github.com/delta-io/delta/blob/master/spark/src/test/resources/delta/table-with-dv-large/_delta_log/00000000000000000001.json + selection_vector[index] = true; + debug!( + "Including file in scan: ({}, {:?})", + add.path, + add.dv_unique_id(), + ); + } + for (remove, index) in removes.into_iter() { + debug!( + "Including file in scan: ({}, {:?})", + remove.path, + remove.dv_unique_id(), + ); + if let Some(dv) = remove.deletion_vector { + let dv_info = DvInfo { + deletion_vector: Some(dv), + }; + self.remove_dvs.insert(remove.path.clone(), dv_info); + } + selection_vector[index] = true; + } + + let result = expression_handler + .get_evaluator( + get_log_schema().project(&[ADD_NAME])?, + self.get_add_transform_expr(), + SCAN_ROW_DATATYPE.clone(), + ) + .evaluate(actions)?; + Ok((result, selection_vector)) + } + + // work shared between process_batch and process_scan_batch + fn setup_batch_process( + &mut self, + selection_vector: Option>, + actions: &dyn EngineData, + ) -> DeltaResult { + let schema_to_use = + // NB: We _must_ pass these in the order `ADD_NAME, REMOVE_NAME` as the visitor assumes + // the Add action comes first. The [`project`] method honors this order, so this works + // as long as we keep this order here. + get_log_schema().project(&[ADD_NAME, REMOVE_NAME])?; + let mut visitor = AddRemoveCdcVisitor::new(selection_vector); + actions.extract(schema_to_use, &mut visitor)?; + + Ok(visitor) + } +} diff --git a/kernel/src/table_changes/state.rs b/kernel/src/table_changes/state.rs new file mode 100644 index 000000000..6bd7f6f73 --- /dev/null +++ b/kernel/src/table_changes/state.rs @@ -0,0 +1,127 @@ +//! This module encapsulates the state of a scan + +use std::collections::HashMap; + +use crate::{ + actions::visitors::visit_deletion_vector_at, + engine_data::{GetData, TypedGetData}, + features::ColumnMappingMode, + scan::{ + log_replay::{self, SCAN_ROW_SCHEMA}, + state::{DvInfo, Stats}, + }, + schema::SchemaRef, + DataVisitor, DeltaResult, EngineData, Error, +}; +use serde::{Deserialize, Serialize}; +use tracing::warn; + +/// State that doesn't change between scans +#[derive(Clone, Debug, Serialize, Deserialize)] +pub(crate) struct GlobalScanState { + pub table_root: String, + pub partition_columns: Vec, + pub logical_schema: SchemaRef, + pub read_schema: SchemaRef, + pub column_mapping_mode: ColumnMappingMode, +} + +pub(crate) type ScanCallback = fn( + context: &mut T, + path: &str, + size: i64, + stats: Option, + dv_info: DvInfo, + partition_values: HashMap, +); + +/// Request that the kernel call a callback on each valid file that needs to be read for the +/// scan. +/// +/// The arguments to the callback are: +/// * `context`: an `&mut context` argument. this can be anything that engine needs to pass through to each call +/// * `path`: a `&str` which is the path to the file +/// * `size`: an `i64` which is the size of the file +/// * `dv_info`: a [`DvInfo`] struct, which allows getting the selection vector for this file +/// * `partition_values`: a `HashMap` which are partition values +/// +/// ## Context +/// A note on the `context`. This can be any value the engine wants. This function takes ownership +/// of the passed arg, but then returns it, so the engine can repeatedly call `visit_scan_files` +/// with the same context. +/// +/// ## Example +/// ```ignore +/// let mut context = [my context]; +/// for res in scan_data { // scan data from scan.get_scan_data() +/// let (data, vector) = res?; +/// context = delta_kernel::scan::state::visit_scan_files( +/// data.as_ref(), +/// vector, +/// context, +/// my_callback, +/// )?; +/// } +/// ``` +pub(crate) fn visit_scan_files( + data: &dyn EngineData, + selection_vector: &[bool], + context: T, + callback: ScanCallback, +) -> DeltaResult { + let mut visitor = ScanFileVisitor { + callback, + selection_vector, + context, + }; + data.extract(log_replay::SCAN_ROW_SCHEMA.clone(), &mut visitor)?; + Ok(visitor.context) +} + +// add some visitor magic for engines +struct ScanFileVisitor<'a, T> { + callback: ScanCallback, + selection_vector: &'a [bool], + context: T, +} + +impl DataVisitor for ScanFileVisitor<'_, T> { + fn visit<'a>(&mut self, row_count: usize, getters: &[&'a dyn GetData<'a>]) -> DeltaResult<()> { + for row_index in 0..row_count { + if !self.selection_vector[row_index] { + // skip skipped rows + continue; + } + // Since path column is required, use it to detect presence of an Add action + if let Some(path) = getters[0].get_opt(row_index, "scanFile.path")? { + let size = getters[1].get(row_index, "scanFile.size")?; + let stats: Option = getters[3].get_opt(row_index, "scanFile.stats")?; + let stats: Option = + stats.and_then(|json| match serde_json::from_str(json.as_str()) { + Ok(stats) => Some(stats), + Err(e) => { + warn!("Invalid stats string in Add file {json}: {}", e); + None + } + }); + + let dv_index = SCAN_ROW_SCHEMA + .index_of("deletionVector") + .ok_or_else(|| Error::missing_column("deletionVector"))?; + let deletion_vector = visit_deletion_vector_at(row_index, &getters[dv_index..])?; + let dv_info = DvInfo { deletion_vector }; + let partition_values = + getters[9].get(row_index, "scanFile.fileConstantValues.partitionValues")?; + (self.callback)( + &mut self.context, + path, + size, + stats, + dv_info, + partition_values, + ) + } + } + Ok(()) + } +} diff --git a/kernel/src/table_changes/table_changes_scan.rs b/kernel/src/table_changes/table_changes_scan.rs new file mode 100644 index 000000000..2f5229002 --- /dev/null +++ b/kernel/src/table_changes/table_changes_scan.rs @@ -0,0 +1,371 @@ +use std::{ + collections::{HashMap, HashSet}, + iter, + sync::Arc, +}; + +use itertools::Itertools; +use tracing::debug; + +use crate::{ + actions::{deletion_vector::split_vector, get_log_schema, Add, Remove, ADD_NAME, REMOVE_NAME}, + expressions, + scan::{ + data_skipping::DataSkippingFilter, + get_state_info, + log_replay::scan_action_iter, + state::{self, DvInfo, GlobalScanState, Stats}, + transform_to_logical_internal, ColumnType, ScanData, ScanResult, + }, + schema::{SchemaRef, StructType}, + table_changes::replay_scanner::AddRemoveCdcVisitor, + DeltaResult, Engine, EngineData, ExpressionRef, FileMeta, +}; + +use super::{replay_scanner::TableChangesLogReplayScanner, TableChanges, TableChangesScanData}; + +/// Builder to scan a snapshot of a table. +pub struct TableChangesScanBuilder { + table_changes: Arc, + schema: Option, + predicate: Option, +} + +impl TableChangesScanBuilder { + /// Create a new [`ScanBuilder`] instance. + pub fn new(table_changes: impl Into>) -> Self { + Self { + table_changes: table_changes.into(), + schema: None, + predicate: None, + } + } + + /// Provide [`Schema`] for columns to select from the [`Snapshot`]. + /// + /// A table with columns `[a, b, c]` could have a scan which reads only the first + /// two columns by using the schema `[a, b]`. + /// + /// [`Schema`]: crate::schema::Schema + /// [`Snapshot`]: crate::snapshot::Snapshot + pub fn with_schema(mut self, schema: SchemaRef) -> Self { + self.schema = Some(schema); + self + } + + /// Optionally provide a [`SchemaRef`] for columns to select from the [`Snapshot`]. See + /// [`ScanBuilder::with_schema`] for details. If `schema_opt` is `None` this is a no-op. + pub fn with_schema_opt(self, schema_opt: Option) -> Self { + match schema_opt { + Some(schema) => self.with_schema(schema), + None => self, + } + } + + /// Optionally provide an expression to filter rows. For example, using the predicate `x < + /// 4` to return a subset of the rows in the scan which satisfy the filter. If `predicate_opt` + /// is `None`, this is a no-op. + /// + /// NOTE: The filtering is best-effort and can produce false positives (rows that should should + /// have been filtered out but were kept). + pub fn with_predicate(mut self, predicate: impl Into>) -> Self { + self.predicate = predicate.into(); + self + } + + /// Build the [`Scan`]. + /// + /// This does not scan the table at this point, but does do some work to ensure that the + /// provided schema make sense, and to prepare some metadata that the scan will need. The + /// [`Scan`] type itself can be used to fetch the files and associated metadata required to + /// perform actual data reads. + pub fn build(self) -> DeltaResult { + // if no schema is provided, use snapshot's entire schema (e.g. SELECT *) + let logical_schema = self + .schema + .unwrap_or_else(|| self.table_changes.schema.clone().into()); + let (all_fields, read_fields, have_partition_cols) = get_state_info( + logical_schema.as_ref(), + &self.table_changes.metadata.partition_columns, + self.table_changes.column_mapping_mode, + )?; + let physical_schema = Arc::new(StructType::new(read_fields)); + Ok(TableChangesScan { + table_changes: self.table_changes, + logical_schema, + physical_schema, + predicate: self.predicate, + all_fields, + have_partition_cols, + }) + } +} +pub struct TableChangesScan { + table_changes: Arc, + logical_schema: SchemaRef, + physical_schema: SchemaRef, + predicate: Option, + all_fields: Vec, + have_partition_cols: bool, +} + +/// Given an iterator of (engine_data, bool) tuples and a predicate, returns an iterator of +/// `(engine_data, selection_vec)`. Each row that is selected in the returned `engine_data` _must_ +/// be processed to complete the scan. Non-selected rows _must_ be ignored. The boolean flag +/// indicates whether the record batch is a log or checkpoint batch. +pub fn table_changes_action_iter( + engine: &dyn Engine, + commit_iter: impl Iterator< + Item = DeltaResult>> + Send>>, + >, + table_schema: &SchemaRef, + predicate: Option, +) -> DeltaResult>> { + let filter = DataSkippingFilter::new(engine, table_schema, predicate); + let expression_handler = engine.get_expression_handler(); + println!("commit iter len: {}", commit_iter.try_len().unwrap()); + let result = commit_iter + .map(move |action_iter| -> DeltaResult<_> { + let action_iter = action_iter?; + let expression_handler = expression_handler.clone(); + let mut log_scanner = TableChangesLogReplayScanner::new(filter.clone()); + + // Find CDC, get commitInfo, and perform metadata scan + let mut batches = vec![]; + for action_res in action_iter { + println!("Action res iter "); + let batch = action_res?; + // TODO: Make this metadata iterator + // log_scanner.process_scan_batch(expression_handler.as_ref(), batch.as_ref())?; + batches.push(batch); + } + + // File metadata output scan + let x: Vec = batches + .into_iter() + .map(|batch| { + println!("Action res iter "); + log_scanner.process_scan_batch(expression_handler.as_ref(), batch.as_ref()) + }) + .try_collect()?; + let remove_dvs = Arc::new(log_scanner.remove_dvs); + let y = x.into_iter().map(move |(a, b)| { + let remove_dvs = remove_dvs.clone(); + (a, b, remove_dvs) + }); + Ok(y) + }) + .flatten_ok(); + Ok(result) + // todo!() + // action_iter + // .map(move |action_res| { + // action_res.and_then(|(batch, is_log_batch)| { + // log_scanner.process_scan_batch( + // expression_handler.as_ref(), + // batch.as_ref(), + // is_log_batch, + // ) + // }) + // }) + // .filter(|action_res| { + // match action_res { + // Ok((_, sel_vec)) => { + // // don't bother returning it if everything is filtered out + // sel_vec.contains(&true) + // } + // Err(_) => true, // just pass through errors + // } + // }) +} + +impl TableChangesScan { + /// Get a shared reference to the [`Schema`] of the scan. + /// + /// [`Schema`]: crate::schema::Schema + pub fn schema(&self) -> &SchemaRef { + &self.logical_schema + } + + /// Get the predicate [`Expression`] of the scan. + pub fn predicate(&self) -> Option { + self.predicate.clone() + } + + /// Get an iterator of [`EngineData`]s that should be included in scan for a query. This handles + /// log-replay, reconciling Add and Remove actions, and applying data skipping (if + /// possible). Each item in the returned iterator is a tuple of: + /// - `Box`: Data in engine format, where each row represents a file to be + /// scanned. The schema for each row can be obtained by calling [`scan_row_schema`]. + /// - `Vec`: A selection vector. If a row is at index `i` and this vector is `false` at + /// index `i`, then that row should *not* be processed (i.e. it is filtered out). If the vector + /// is `true` at index `i` the row *should* be processed. If the selector vector is *shorter* + /// than the number of rows returned, missing elements are considered `true`, i.e. included in + /// the query. NB: If you are using the default engine and plan to call arrow's + /// `filter_record_batch`, you _need_ to extend this vector to the full length of the batch or + /// arrow will drop the extra rows. + pub fn scan_data( + &self, + engine: &dyn Engine, + ) -> DeltaResult>> { + table_changes_action_iter( + engine, + self.replay_for_scan_data(engine)?, + &self.logical_schema, + self.predicate(), + ) + } + + // Factored out to facilitate testing + fn replay_for_scan_data( + &self, + engine: &dyn Engine, + ) -> DeltaResult< + impl Iterator< + Item = DeltaResult>> + Send>>, + >, + > { + let commit_read_schema = get_log_schema().project(&[ADD_NAME, REMOVE_NAME])?; + + // NOTE: We don't pass any meta-predicate because we expect no meaningful row group skipping + // when ~every checkpoint file will contain the adds and removes we are looking for. + self.table_changes + .log_segment + .replay_commits(engine, commit_read_schema, None) + } + + /// Get global state that is valid for the entire scan. This is somewhat expensive so should + /// only be called once per scan. + pub fn global_scan_state(&self) -> GlobalScanState { + GlobalScanState { + table_root: self.table_changes.table_root.to_string(), + partition_columns: self.table_changes.metadata.partition_columns.clone(), + logical_schema: self.logical_schema.clone(), + read_schema: self.physical_schema.clone(), + column_mapping_mode: self.table_changes.column_mapping_mode, + } + } + + /// Perform an "all in one" scan. This will use the provided `engine` to read and + /// process all the data for the query. Each [`ScanResult`] in the resultant iterator encapsulates + /// the raw data and an optional boolean vector built from the deletion vector if it was + /// present. See the documentation for [`ScanResult`] for more details. Generally + /// connectors/engines will want to use [`Scan::scan_data`] so they can have more control over + /// the execution of the scan. + // This calls [`Scan::scan_data`] to get an iterator of `ScanData` actions for the scan, and then uses the + // `engine`'s [`crate::ParquetHandler`] to read the actual table data. + pub fn execute<'a>( + &'a self, + engine: &'a dyn Engine, + ) -> DeltaResult> + 'a> { + struct ScanFileContext { + pub files: Vec, + pub remove_dv: Arc>, + } + struct ScanFile { + path: String, + size: i64, + dv_info: DvInfo, + partition_values: HashMap, + } + fn scan_data_callback( + context: &mut ScanFileContext, + path: &str, + size: i64, + _: Option, + dv_info: DvInfo, + partition_values: HashMap, + ) { + context.files.push(ScanFile { + path: path.to_string(), + size, + dv_info, + partition_values, + }); + } + + debug!( + "Executing scan with logical schema {:#?} and physical schema {:#?}", + self.logical_schema, self.physical_schema + ); + // enum ScanFile { + // Add { add: Add, remove_dv: Option }, + // Remove(Remove), + // } + debug!( + "Executing scan with logical schema {:#?} and physical schema {:#?}", + self.logical_schema, self.physical_schema + ); + + let global_state = Arc::new(self.global_scan_state()); + let scan_data = self.scan_data(engine)?; + let scan_files_iter: Vec<_> = scan_data + .map(|res| -> DeltaResult<_> { + let (data, vec, remove_dv) = res?; + let context = ScanFileContext { + files: vec![], + remove_dv, + }; + let context = + state::visit_scan_files(data.as_ref(), &vec, context, scan_data_callback)?; + Ok(context + .files + .into_iter() + .map(move |x| (x, context.remove_dv.clone()))) + }) + .flatten_ok() + .collect_vec(); + + let result = scan_files_iter + .into_iter() + .map(move |scan_res| -> DeltaResult<_> { + let (scan_file, remove_dvs) = scan_res?; + println!("Remove dvs: {:?}", remove_dvs); + let file_path = self.table_changes.table_root.join(&scan_file.path)?; + let mut selection_vector = scan_file + .dv_info + .get_selection_vector(engine, &self.table_changes.table_root)?; + let meta = FileMeta { + last_modified: 0, + size: scan_file.size as usize, + location: file_path, + }; + let read_result_iter = engine.get_parquet_handler().read_parquet_files( + &[meta], + global_state.read_schema.clone(), + self.predicate(), + )?; + let gs = global_state.clone(); // Arc clone + Ok(read_result_iter.map(move |read_result| -> DeltaResult<_> { + let read_result = read_result?; + // to transform the physical data into the correct logical form + let logical = transform_to_logical_internal( + engine, + read_result, + &gs, + &scan_file.partition_values, + &self.all_fields, + self.have_partition_cols, + ); + let len = logical.as_ref().map_or(0, |res| res.length()); + // need to split the dv_mask. what's left in dv_mask covers this result, and rest + // will cover the following results. we `take()` out of `selection_vector` to avoid + // trying to return a captured variable. We're going to reassign `selection_vector` + // to `rest` in a moment anyway + let mut sv = selection_vector.take(); + let rest = split_vector(sv.as_mut(), len, None); + let result = ScanResult { + raw_data: logical, + raw_mask: sv, + }; + selection_vector = rest; + Ok(result) + })) + }) + // Iterator>>> to Iterator>> + .flatten_ok() + // Iterator>> to Iterator> + .map(|x| x?); + Ok(result) + } +} From 6d8e35fa9aeb8acf025f5d68fe80b411134c565d Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 6 Nov 2024 15:32:48 -0800 Subject: [PATCH 23/88] remove table_changes --- kernel/src/table_changes.rs | 41 -- kernel/src/table_changes/metadata_scanner.rs | 168 -------- kernel/src/table_changes/mod.rs | 86 ---- kernel/src/table_changes/replay_scanner.rs | 171 -------- kernel/src/table_changes/state.rs | 127 ------ .../src/table_changes/table_changes_scan.rs | 371 ------------------ 6 files changed, 964 deletions(-) delete mode 100644 kernel/src/table_changes.rs delete mode 100644 kernel/src/table_changes/metadata_scanner.rs delete mode 100644 kernel/src/table_changes/mod.rs delete mode 100644 kernel/src/table_changes/replay_scanner.rs delete mode 100644 kernel/src/table_changes/state.rs delete mode 100644 kernel/src/table_changes/table_changes_scan.rs diff --git a/kernel/src/table_changes.rs b/kernel/src/table_changes.rs deleted file mode 100644 index 7ac51f584..000000000 --- a/kernel/src/table_changes.rs +++ /dev/null @@ -1,41 +0,0 @@ -//! In-memory representation of a change data feed table. - -use url::Url; - -use crate::{ - log_segment::{LogSegment, LogSegmentBuilder}, - path::AsUrl, - schema::Schema, - snapshot::Snapshot, - DeltaResult, Engine, Version, -}; - -#[derive(Debug)] -pub struct TableChanges { - snapshot: Snapshot, - cdf_range: LogSegment, - schema: Schema, -} - -impl TableChanges { - pub fn try_new( - table_root: Url, - engine: &dyn Engine, - start_version: Version, - end_version: Option, - ) -> DeltaResult { - let fs_client = engine.get_file_system_client(); - let snapshot = Snapshot::try_new(table_root.as_url().clone(), engine, Some(start_version))?; - let mut builder = LogSegmentBuilder::new(fs_client, &table_root); - builder = builder.with_start_version(start_version); - if let Some(end_version) = end_version { - builder = builder.with_start_version(end_version); - } - let log_segment = builder.build()?; - - Ok(TableChanges { - snapshot, - cdf_range: log_segment, - }) - } -} diff --git a/kernel/src/table_changes/metadata_scanner.rs b/kernel/src/table_changes/metadata_scanner.rs deleted file mode 100644 index 55c765aa2..000000000 --- a/kernel/src/table_changes/metadata_scanner.rs +++ /dev/null @@ -1,168 +0,0 @@ -use std::collections::HashSet; - -use tracing::debug; - -use crate::actions::visitors::{AddVisitor, RemoveVisitor}; -use crate::actions::{get_log_schema, Add, Remove, ADD_NAME, REMOVE_NAME}; -use crate::engine_data::{GetData, TypedGetData}; -use crate::expressions::{column_expr, Expression}; -use crate::scan::data_skipping::DataSkippingFilter; -use crate::scan::log_replay::SCAN_ROW_DATATYPE; -use crate::scan::ScanData; -use crate::{DataVisitor, DeltaResult, EngineData, ExpressionHandler}; - -#[derive(Default)] -pub(crate) struct CdcVisitor { - pub adds: Vec<(Add, usize)>, - pub removes: Vec, - selection_vector: Option>, -} - -const ADD_FIELD_COUNT: usize = 15; - -impl CdcVisitor { - pub(crate) fn new(selection_vector: Option>) -> Self { - CdcVisitor { - selection_vector, - ..Default::default() - } - } -} - -impl DataVisitor for CdcVisitor { - fn visit<'a>(&mut self, row_count: usize, getters: &[&'a dyn GetData<'a>]) -> DeltaResult<()> { - for i in 0..row_count { - // Add will have a path at index 0 if it is valid - if let Some(path) = getters[0].get_opt(i, "add.path")? { - // Keep the file unless the selection vector is present and is false for this row - if !self - .selection_vector - .as_ref() - .is_some_and(|selection| !selection[i]) - { - self.adds.push(( - AddVisitor::visit_add(i, path, &getters[..ADD_FIELD_COUNT])?, - i, - )) - } - } - // Remove will have a path at index 15 if it is valid - // TODO(nick): Should count the fields in Add to ensure we don't get this wrong if more - // are added - // TODO(zach): add a check for selection vector that we never skip a remove - else if let Some(path) = getters[ADD_FIELD_COUNT].get_opt(i, "remove.path")? { - let remove_getters = &getters[ADD_FIELD_COUNT..]; - self.removes - .push(RemoveVisitor::visit_remove(i, path, remove_getters)?); - } - } - Ok(()) - } -} - -pub(crate) struct CdcLogReplayScanner<'a> { - filter: Option<&'a DataSkippingFilter>, - - /// A set of (data file path, dv_unique_id) pairs that have been seen thus - /// far in the log. This is used to filter out files with Remove actions as - /// well as duplicate entries in the log. - seen: HashSet<(String, Option)>, -} - -impl<'a> CdcLogReplayScanner<'a> { - /// Create a new [`LogReplayScanner`] instance - pub(crate) fn new(filter: Option<&'a DataSkippingFilter>) -> Self { - Self { - filter, - seen: Default::default(), - } - } - - fn get_add_transform_expr(&self) -> Expression { - Expression::Struct(vec![ - column_expr!("add.path"), - column_expr!("add.size"), - column_expr!("add.modificationTime"), - column_expr!("add.stats"), - column_expr!("add.deletionVector"), - Expression::Struct(vec![column_expr!("add.partitionValues")]), - ]) - } - - pub(crate) fn process_scan_batch( - &mut self, - expression_handler: &dyn ExpressionHandler, - actions: &dyn EngineData, - ) -> DeltaResult { - // apply data skipping to get back a selection vector for actions that passed skipping - // note: None implies all files passed data skipping. - let filter_vector = self - .filter - .map(|filter| filter.apply(actions)) - .transpose()?; - - // we start our selection vector based on what was filtered. we will add to this vector - // below if a file has been removed - let mut selection_vector = match filter_vector { - Some(ref filter_vector) => filter_vector.clone(), - None => vec![false; actions.length()], - }; - - assert_eq!(selection_vector.len(), actions.length()); - let adds = self.setup_batch_process(filter_vector, actions)?; - - for (add, index) in adds.into_iter() { - // Note: each (add.path + add.dv_unique_id()) pair has a - // unique Add + Remove pair in the log. For example: - // https://github.com/delta-io/delta/blob/master/spark/src/test/resources/delta/table-with-dv-large/_delta_log/00000000000000000001.json - if !self.seen.contains(&(add.path.clone(), add.dv_unique_id())) { - debug!( - "Including file in scan: ({}, {:?})", - add.path, - add.dv_unique_id(), - ); - // Remember file actions from this batch so we can ignore duplicates - // as we process batches from older commit and/or checkpoint files. We - // don't need to track checkpoint batches because they are already the - // oldest actions and can never replace anything. - self.seen.insert((add.path.clone(), add.dv_unique_id())); - selection_vector[index] = true; - } else { - debug!("Filtering out Add due to it being removed {}", add.path); - // we may have a true here because the data-skipping predicate included the file - selection_vector[index] = false; - } - } - - let result = expression_handler - .get_evaluator( - get_log_schema().project(&[ADD_NAME])?, - self.get_add_transform_expr(), - SCAN_ROW_DATATYPE.clone(), - ) - .evaluate(actions)?; - Ok((result, selection_vector)) - } - - // work shared between process_batch and process_scan_batch - fn setup_batch_process( - &mut self, - selection_vector: Option>, - actions: &dyn EngineData, - ) -> DeltaResult> { - let schema_to_use = - // NB: We _must_ pass these in the order `ADD_NAME, REMOVE_NAME` as the visitor assumes - // the Add action comes first. The [`project`] method honors this order, so this works - // as long as we keep this order here. - get_log_schema().project(&[ADD_NAME, REMOVE_NAME])?; - let mut visitor = CdcVisitor::new(selection_vector); - actions.extract(schema_to_use, &mut visitor)?; - - for remove in visitor.removes.into_iter() { - let dv_id = remove.dv_unique_id(); - self.seen.insert((remove.path, dv_id)); - } - - Ok(visitor.adds) - } -} diff --git a/kernel/src/table_changes/mod.rs b/kernel/src/table_changes/mod.rs deleted file mode 100644 index debf9cf61..000000000 --- a/kernel/src/table_changes/mod.rs +++ /dev/null @@ -1,86 +0,0 @@ -//! In-memory representation of a change data feed table. - -use std::{collections::HashMap, sync::Arc}; - -use table_changes_scan::TableChangesScanBuilder; -use url::Url; - -use crate::{ - actions::{Metadata, Protocol}, - features::ColumnMappingMode, - log_segment::{LogSegment, LogSegmentBuilder}, - path::AsUrl, - scan::state::DvInfo, - schema::Schema, - snapshot::Snapshot, - DeltaResult, Engine, EngineData, Error, Version, -}; - -mod metadata_scanner; -mod replay_scanner; -mod state; -pub mod table_changes_scan; - -pub type TableChangesScanData = (Box, Vec, Arc>); - -static CDF_ENABLE_FLAG: &str = "delta.enableChangeDataFeed"; - -#[derive(Debug)] -pub struct TableChanges { - pub snapshot: Snapshot, - pub log_segment: LogSegment, - pub schema: Schema, - pub version: Version, - pub metadata: Metadata, - pub protocol: Protocol, - pub(crate) column_mapping_mode: ColumnMappingMode, - pub table_root: Url, -} - -impl TableChanges { - pub fn try_new( - table_root: Url, - engine: &dyn Engine, - start_version: Version, - end_version: Option, - ) -> DeltaResult { - let start_snapshot = - Snapshot::try_new(table_root.as_url().clone(), engine, Some(start_version))?; - let end_snapshot = Snapshot::try_new(table_root.as_url().clone(), engine, end_version)?; - - let start_flag = start_snapshot.metadata().configuration.get(CDF_ENABLE_FLAG); - let end_flag = end_snapshot.metadata().configuration.get(CDF_ENABLE_FLAG); - - // Verify CDF is enabled at the beginning and end of the interval - let is_valid_flag = |flag_res: Option<&String>| flag_res.is_some_and(|val| val == "true"); - if !is_valid_flag(start_flag) || !is_valid_flag(end_flag) { - return Err(Error::TableChangesDisabled(start_version, end_version)); - } - - // Get a log segment for the CDF range - let fs_client = engine.get_file_system_client(); - let mut builder = LogSegmentBuilder::new(fs_client, &table_root); - builder = builder.with_start_version(start_version); - if let Some(end_version) = end_version { - builder = builder.with_end_version(end_version); - } - builder = builder - .with_no_checkpoint_files() - .with_in_order_commit_files(); - let log_segment = builder.build()?; - - Ok(TableChanges { - snapshot: start_snapshot, - log_segment, - schema: end_snapshot.schema().clone(), - column_mapping_mode: end_snapshot.column_mapping_mode, - version: end_snapshot.version(), - protocol: end_snapshot.protocol().clone(), - metadata: end_snapshot.metadata().clone(), - table_root, - }) - } - pub fn into_scan_builder(self) -> TableChangesScanBuilder { - TableChangesScanBuilder::new(self) - } -} diff --git a/kernel/src/table_changes/replay_scanner.rs b/kernel/src/table_changes/replay_scanner.rs deleted file mode 100644 index 0de5e6f73..000000000 --- a/kernel/src/table_changes/replay_scanner.rs +++ /dev/null @@ -1,171 +0,0 @@ -use std::collections::{HashMap, HashSet}; -use std::sync::Arc; - -use tracing::debug; - -use crate::actions::deletion_vector::DeletionVectorDescriptor; -use crate::actions::visitors::{AddVisitor, RemoveVisitor}; -use crate::actions::{get_log_schema, Add, Remove, ADD_NAME, REMOVE_NAME}; -use crate::engine_data::{GetData, TypedGetData}; -use crate::expressions::{column_expr, Expression}; -use crate::scan::data_skipping::DataSkippingFilter; -use crate::scan::log_replay::SCAN_ROW_DATATYPE; -use crate::scan::state::DvInfo; -use crate::scan::ScanData; -use crate::{DataVisitor, DeltaResult, EngineData, ExpressionHandler}; - -use super::TableChangesScanData; - -#[derive(Default)] -pub(crate) struct AddRemoveCdcVisitor { - pub adds: Vec<(Add, usize)>, - pub removes: Vec<(Remove, usize)>, - selection_vector: Option>, -} - -const ADD_FIELD_COUNT: usize = 15; - -impl AddRemoveCdcVisitor { - pub(crate) fn new(selection_vector: Option>) -> Self { - AddRemoveCdcVisitor { - selection_vector, - ..Default::default() - } - } -} - -impl DataVisitor for AddRemoveCdcVisitor { - fn visit<'a>(&mut self, row_count: usize, getters: &[&'a dyn GetData<'a>]) -> DeltaResult<()> { - for i in 0..row_count { - // Add will have a path at index 0 if it is valid - if let Some(path) = getters[0].get_opt(i, "add.path")? { - // Keep the file unless the selection vector is present and is false for this row - if !self - .selection_vector - .as_ref() - .is_some_and(|selection| !selection[i]) - { - self.adds.push(( - AddVisitor::visit_add(i, path, &getters[..ADD_FIELD_COUNT])?, - i, - )) - } - } - // Remove will have a path at index 15 if it is valid - // TODO(nick): Should count the fields in Add to ensure we don't get this wrong if more - // are added - // TODO(zach): add a check for selection vector that we never skip a remove - else if let Some(path) = getters[ADD_FIELD_COUNT].get_opt(i, "remove.path")? { - let remove_getters = &getters[ADD_FIELD_COUNT..]; - self.removes - .push((RemoveVisitor::visit_remove(i, path, remove_getters)?, i)); - } - } - Ok(()) - } -} - -pub(crate) struct TableChangesLogReplayScanner { - filter: Option, - pub remove_dvs: HashMap, -} - -impl TableChangesLogReplayScanner { - /// Create a new [`LogReplayScanner`] instance - pub(crate) fn new(filter: Option) -> Self { - Self { - filter, - remove_dvs: Default::default(), - } - } - - fn get_add_transform_expr(&self) -> Expression { - Expression::Struct(vec![ - column_expr!("add.path"), - column_expr!("add.size"), - column_expr!("add.modificationTime"), - column_expr!("add.stats"), - column_expr!("add.deletionVector"), - Expression::Struct(vec![column_expr!("add.partitionValues")]), - ]) - } - - pub(crate) fn process_scan_batch( - &mut self, - expression_handler: &dyn ExpressionHandler, - actions: &dyn EngineData, - ) -> DeltaResult { - // apply data skipping to get back a selection vector for actions that passed skipping - // note: None implies all files passed data skipping. - let filter_vector = self - .filter - .as_ref() - .map(|filter| filter.apply(actions)) - .transpose()?; - - // we start our selection vector based on what was filtered. we will add to this vector - // below if a file has been removed - let mut selection_vector = match filter_vector { - Some(ref filter_vector) => filter_vector.clone(), - None => vec![false; actions.length()], - }; - - assert_eq!(selection_vector.len(), actions.length()); - let AddRemoveCdcVisitor { - adds, - removes, - selection_vector: _, - } = self.setup_batch_process(filter_vector, actions)?; - for (add, index) in adds.into_iter() { - // Note: each (add.path + add.dv_unique_id()) pair has a - // unique Add + Remove pair in the log. For example: - // https://github.com/delta-io/delta/blob/master/spark/src/test/resources/delta/table-with-dv-large/_delta_log/00000000000000000001.json - selection_vector[index] = true; - debug!( - "Including file in scan: ({}, {:?})", - add.path, - add.dv_unique_id(), - ); - } - for (remove, index) in removes.into_iter() { - debug!( - "Including file in scan: ({}, {:?})", - remove.path, - remove.dv_unique_id(), - ); - if let Some(dv) = remove.deletion_vector { - let dv_info = DvInfo { - deletion_vector: Some(dv), - }; - self.remove_dvs.insert(remove.path.clone(), dv_info); - } - selection_vector[index] = true; - } - - let result = expression_handler - .get_evaluator( - get_log_schema().project(&[ADD_NAME])?, - self.get_add_transform_expr(), - SCAN_ROW_DATATYPE.clone(), - ) - .evaluate(actions)?; - Ok((result, selection_vector)) - } - - // work shared between process_batch and process_scan_batch - fn setup_batch_process( - &mut self, - selection_vector: Option>, - actions: &dyn EngineData, - ) -> DeltaResult { - let schema_to_use = - // NB: We _must_ pass these in the order `ADD_NAME, REMOVE_NAME` as the visitor assumes - // the Add action comes first. The [`project`] method honors this order, so this works - // as long as we keep this order here. - get_log_schema().project(&[ADD_NAME, REMOVE_NAME])?; - let mut visitor = AddRemoveCdcVisitor::new(selection_vector); - actions.extract(schema_to_use, &mut visitor)?; - - Ok(visitor) - } -} diff --git a/kernel/src/table_changes/state.rs b/kernel/src/table_changes/state.rs deleted file mode 100644 index 6bd7f6f73..000000000 --- a/kernel/src/table_changes/state.rs +++ /dev/null @@ -1,127 +0,0 @@ -//! This module encapsulates the state of a scan - -use std::collections::HashMap; - -use crate::{ - actions::visitors::visit_deletion_vector_at, - engine_data::{GetData, TypedGetData}, - features::ColumnMappingMode, - scan::{ - log_replay::{self, SCAN_ROW_SCHEMA}, - state::{DvInfo, Stats}, - }, - schema::SchemaRef, - DataVisitor, DeltaResult, EngineData, Error, -}; -use serde::{Deserialize, Serialize}; -use tracing::warn; - -/// State that doesn't change between scans -#[derive(Clone, Debug, Serialize, Deserialize)] -pub(crate) struct GlobalScanState { - pub table_root: String, - pub partition_columns: Vec, - pub logical_schema: SchemaRef, - pub read_schema: SchemaRef, - pub column_mapping_mode: ColumnMappingMode, -} - -pub(crate) type ScanCallback = fn( - context: &mut T, - path: &str, - size: i64, - stats: Option, - dv_info: DvInfo, - partition_values: HashMap, -); - -/// Request that the kernel call a callback on each valid file that needs to be read for the -/// scan. -/// -/// The arguments to the callback are: -/// * `context`: an `&mut context` argument. this can be anything that engine needs to pass through to each call -/// * `path`: a `&str` which is the path to the file -/// * `size`: an `i64` which is the size of the file -/// * `dv_info`: a [`DvInfo`] struct, which allows getting the selection vector for this file -/// * `partition_values`: a `HashMap` which are partition values -/// -/// ## Context -/// A note on the `context`. This can be any value the engine wants. This function takes ownership -/// of the passed arg, but then returns it, so the engine can repeatedly call `visit_scan_files` -/// with the same context. -/// -/// ## Example -/// ```ignore -/// let mut context = [my context]; -/// for res in scan_data { // scan data from scan.get_scan_data() -/// let (data, vector) = res?; -/// context = delta_kernel::scan::state::visit_scan_files( -/// data.as_ref(), -/// vector, -/// context, -/// my_callback, -/// )?; -/// } -/// ``` -pub(crate) fn visit_scan_files( - data: &dyn EngineData, - selection_vector: &[bool], - context: T, - callback: ScanCallback, -) -> DeltaResult { - let mut visitor = ScanFileVisitor { - callback, - selection_vector, - context, - }; - data.extract(log_replay::SCAN_ROW_SCHEMA.clone(), &mut visitor)?; - Ok(visitor.context) -} - -// add some visitor magic for engines -struct ScanFileVisitor<'a, T> { - callback: ScanCallback, - selection_vector: &'a [bool], - context: T, -} - -impl DataVisitor for ScanFileVisitor<'_, T> { - fn visit<'a>(&mut self, row_count: usize, getters: &[&'a dyn GetData<'a>]) -> DeltaResult<()> { - for row_index in 0..row_count { - if !self.selection_vector[row_index] { - // skip skipped rows - continue; - } - // Since path column is required, use it to detect presence of an Add action - if let Some(path) = getters[0].get_opt(row_index, "scanFile.path")? { - let size = getters[1].get(row_index, "scanFile.size")?; - let stats: Option = getters[3].get_opt(row_index, "scanFile.stats")?; - let stats: Option = - stats.and_then(|json| match serde_json::from_str(json.as_str()) { - Ok(stats) => Some(stats), - Err(e) => { - warn!("Invalid stats string in Add file {json}: {}", e); - None - } - }); - - let dv_index = SCAN_ROW_SCHEMA - .index_of("deletionVector") - .ok_or_else(|| Error::missing_column("deletionVector"))?; - let deletion_vector = visit_deletion_vector_at(row_index, &getters[dv_index..])?; - let dv_info = DvInfo { deletion_vector }; - let partition_values = - getters[9].get(row_index, "scanFile.fileConstantValues.partitionValues")?; - (self.callback)( - &mut self.context, - path, - size, - stats, - dv_info, - partition_values, - ) - } - } - Ok(()) - } -} diff --git a/kernel/src/table_changes/table_changes_scan.rs b/kernel/src/table_changes/table_changes_scan.rs deleted file mode 100644 index 2f5229002..000000000 --- a/kernel/src/table_changes/table_changes_scan.rs +++ /dev/null @@ -1,371 +0,0 @@ -use std::{ - collections::{HashMap, HashSet}, - iter, - sync::Arc, -}; - -use itertools::Itertools; -use tracing::debug; - -use crate::{ - actions::{deletion_vector::split_vector, get_log_schema, Add, Remove, ADD_NAME, REMOVE_NAME}, - expressions, - scan::{ - data_skipping::DataSkippingFilter, - get_state_info, - log_replay::scan_action_iter, - state::{self, DvInfo, GlobalScanState, Stats}, - transform_to_logical_internal, ColumnType, ScanData, ScanResult, - }, - schema::{SchemaRef, StructType}, - table_changes::replay_scanner::AddRemoveCdcVisitor, - DeltaResult, Engine, EngineData, ExpressionRef, FileMeta, -}; - -use super::{replay_scanner::TableChangesLogReplayScanner, TableChanges, TableChangesScanData}; - -/// Builder to scan a snapshot of a table. -pub struct TableChangesScanBuilder { - table_changes: Arc, - schema: Option, - predicate: Option, -} - -impl TableChangesScanBuilder { - /// Create a new [`ScanBuilder`] instance. - pub fn new(table_changes: impl Into>) -> Self { - Self { - table_changes: table_changes.into(), - schema: None, - predicate: None, - } - } - - /// Provide [`Schema`] for columns to select from the [`Snapshot`]. - /// - /// A table with columns `[a, b, c]` could have a scan which reads only the first - /// two columns by using the schema `[a, b]`. - /// - /// [`Schema`]: crate::schema::Schema - /// [`Snapshot`]: crate::snapshot::Snapshot - pub fn with_schema(mut self, schema: SchemaRef) -> Self { - self.schema = Some(schema); - self - } - - /// Optionally provide a [`SchemaRef`] for columns to select from the [`Snapshot`]. See - /// [`ScanBuilder::with_schema`] for details. If `schema_opt` is `None` this is a no-op. - pub fn with_schema_opt(self, schema_opt: Option) -> Self { - match schema_opt { - Some(schema) => self.with_schema(schema), - None => self, - } - } - - /// Optionally provide an expression to filter rows. For example, using the predicate `x < - /// 4` to return a subset of the rows in the scan which satisfy the filter. If `predicate_opt` - /// is `None`, this is a no-op. - /// - /// NOTE: The filtering is best-effort and can produce false positives (rows that should should - /// have been filtered out but were kept). - pub fn with_predicate(mut self, predicate: impl Into>) -> Self { - self.predicate = predicate.into(); - self - } - - /// Build the [`Scan`]. - /// - /// This does not scan the table at this point, but does do some work to ensure that the - /// provided schema make sense, and to prepare some metadata that the scan will need. The - /// [`Scan`] type itself can be used to fetch the files and associated metadata required to - /// perform actual data reads. - pub fn build(self) -> DeltaResult { - // if no schema is provided, use snapshot's entire schema (e.g. SELECT *) - let logical_schema = self - .schema - .unwrap_or_else(|| self.table_changes.schema.clone().into()); - let (all_fields, read_fields, have_partition_cols) = get_state_info( - logical_schema.as_ref(), - &self.table_changes.metadata.partition_columns, - self.table_changes.column_mapping_mode, - )?; - let physical_schema = Arc::new(StructType::new(read_fields)); - Ok(TableChangesScan { - table_changes: self.table_changes, - logical_schema, - physical_schema, - predicate: self.predicate, - all_fields, - have_partition_cols, - }) - } -} -pub struct TableChangesScan { - table_changes: Arc, - logical_schema: SchemaRef, - physical_schema: SchemaRef, - predicate: Option, - all_fields: Vec, - have_partition_cols: bool, -} - -/// Given an iterator of (engine_data, bool) tuples and a predicate, returns an iterator of -/// `(engine_data, selection_vec)`. Each row that is selected in the returned `engine_data` _must_ -/// be processed to complete the scan. Non-selected rows _must_ be ignored. The boolean flag -/// indicates whether the record batch is a log or checkpoint batch. -pub fn table_changes_action_iter( - engine: &dyn Engine, - commit_iter: impl Iterator< - Item = DeltaResult>> + Send>>, - >, - table_schema: &SchemaRef, - predicate: Option, -) -> DeltaResult>> { - let filter = DataSkippingFilter::new(engine, table_schema, predicate); - let expression_handler = engine.get_expression_handler(); - println!("commit iter len: {}", commit_iter.try_len().unwrap()); - let result = commit_iter - .map(move |action_iter| -> DeltaResult<_> { - let action_iter = action_iter?; - let expression_handler = expression_handler.clone(); - let mut log_scanner = TableChangesLogReplayScanner::new(filter.clone()); - - // Find CDC, get commitInfo, and perform metadata scan - let mut batches = vec![]; - for action_res in action_iter { - println!("Action res iter "); - let batch = action_res?; - // TODO: Make this metadata iterator - // log_scanner.process_scan_batch(expression_handler.as_ref(), batch.as_ref())?; - batches.push(batch); - } - - // File metadata output scan - let x: Vec = batches - .into_iter() - .map(|batch| { - println!("Action res iter "); - log_scanner.process_scan_batch(expression_handler.as_ref(), batch.as_ref()) - }) - .try_collect()?; - let remove_dvs = Arc::new(log_scanner.remove_dvs); - let y = x.into_iter().map(move |(a, b)| { - let remove_dvs = remove_dvs.clone(); - (a, b, remove_dvs) - }); - Ok(y) - }) - .flatten_ok(); - Ok(result) - // todo!() - // action_iter - // .map(move |action_res| { - // action_res.and_then(|(batch, is_log_batch)| { - // log_scanner.process_scan_batch( - // expression_handler.as_ref(), - // batch.as_ref(), - // is_log_batch, - // ) - // }) - // }) - // .filter(|action_res| { - // match action_res { - // Ok((_, sel_vec)) => { - // // don't bother returning it if everything is filtered out - // sel_vec.contains(&true) - // } - // Err(_) => true, // just pass through errors - // } - // }) -} - -impl TableChangesScan { - /// Get a shared reference to the [`Schema`] of the scan. - /// - /// [`Schema`]: crate::schema::Schema - pub fn schema(&self) -> &SchemaRef { - &self.logical_schema - } - - /// Get the predicate [`Expression`] of the scan. - pub fn predicate(&self) -> Option { - self.predicate.clone() - } - - /// Get an iterator of [`EngineData`]s that should be included in scan for a query. This handles - /// log-replay, reconciling Add and Remove actions, and applying data skipping (if - /// possible). Each item in the returned iterator is a tuple of: - /// - `Box`: Data in engine format, where each row represents a file to be - /// scanned. The schema for each row can be obtained by calling [`scan_row_schema`]. - /// - `Vec`: A selection vector. If a row is at index `i` and this vector is `false` at - /// index `i`, then that row should *not* be processed (i.e. it is filtered out). If the vector - /// is `true` at index `i` the row *should* be processed. If the selector vector is *shorter* - /// than the number of rows returned, missing elements are considered `true`, i.e. included in - /// the query. NB: If you are using the default engine and plan to call arrow's - /// `filter_record_batch`, you _need_ to extend this vector to the full length of the batch or - /// arrow will drop the extra rows. - pub fn scan_data( - &self, - engine: &dyn Engine, - ) -> DeltaResult>> { - table_changes_action_iter( - engine, - self.replay_for_scan_data(engine)?, - &self.logical_schema, - self.predicate(), - ) - } - - // Factored out to facilitate testing - fn replay_for_scan_data( - &self, - engine: &dyn Engine, - ) -> DeltaResult< - impl Iterator< - Item = DeltaResult>> + Send>>, - >, - > { - let commit_read_schema = get_log_schema().project(&[ADD_NAME, REMOVE_NAME])?; - - // NOTE: We don't pass any meta-predicate because we expect no meaningful row group skipping - // when ~every checkpoint file will contain the adds and removes we are looking for. - self.table_changes - .log_segment - .replay_commits(engine, commit_read_schema, None) - } - - /// Get global state that is valid for the entire scan. This is somewhat expensive so should - /// only be called once per scan. - pub fn global_scan_state(&self) -> GlobalScanState { - GlobalScanState { - table_root: self.table_changes.table_root.to_string(), - partition_columns: self.table_changes.metadata.partition_columns.clone(), - logical_schema: self.logical_schema.clone(), - read_schema: self.physical_schema.clone(), - column_mapping_mode: self.table_changes.column_mapping_mode, - } - } - - /// Perform an "all in one" scan. This will use the provided `engine` to read and - /// process all the data for the query. Each [`ScanResult`] in the resultant iterator encapsulates - /// the raw data and an optional boolean vector built from the deletion vector if it was - /// present. See the documentation for [`ScanResult`] for more details. Generally - /// connectors/engines will want to use [`Scan::scan_data`] so they can have more control over - /// the execution of the scan. - // This calls [`Scan::scan_data`] to get an iterator of `ScanData` actions for the scan, and then uses the - // `engine`'s [`crate::ParquetHandler`] to read the actual table data. - pub fn execute<'a>( - &'a self, - engine: &'a dyn Engine, - ) -> DeltaResult> + 'a> { - struct ScanFileContext { - pub files: Vec, - pub remove_dv: Arc>, - } - struct ScanFile { - path: String, - size: i64, - dv_info: DvInfo, - partition_values: HashMap, - } - fn scan_data_callback( - context: &mut ScanFileContext, - path: &str, - size: i64, - _: Option, - dv_info: DvInfo, - partition_values: HashMap, - ) { - context.files.push(ScanFile { - path: path.to_string(), - size, - dv_info, - partition_values, - }); - } - - debug!( - "Executing scan with logical schema {:#?} and physical schema {:#?}", - self.logical_schema, self.physical_schema - ); - // enum ScanFile { - // Add { add: Add, remove_dv: Option }, - // Remove(Remove), - // } - debug!( - "Executing scan with logical schema {:#?} and physical schema {:#?}", - self.logical_schema, self.physical_schema - ); - - let global_state = Arc::new(self.global_scan_state()); - let scan_data = self.scan_data(engine)?; - let scan_files_iter: Vec<_> = scan_data - .map(|res| -> DeltaResult<_> { - let (data, vec, remove_dv) = res?; - let context = ScanFileContext { - files: vec![], - remove_dv, - }; - let context = - state::visit_scan_files(data.as_ref(), &vec, context, scan_data_callback)?; - Ok(context - .files - .into_iter() - .map(move |x| (x, context.remove_dv.clone()))) - }) - .flatten_ok() - .collect_vec(); - - let result = scan_files_iter - .into_iter() - .map(move |scan_res| -> DeltaResult<_> { - let (scan_file, remove_dvs) = scan_res?; - println!("Remove dvs: {:?}", remove_dvs); - let file_path = self.table_changes.table_root.join(&scan_file.path)?; - let mut selection_vector = scan_file - .dv_info - .get_selection_vector(engine, &self.table_changes.table_root)?; - let meta = FileMeta { - last_modified: 0, - size: scan_file.size as usize, - location: file_path, - }; - let read_result_iter = engine.get_parquet_handler().read_parquet_files( - &[meta], - global_state.read_schema.clone(), - self.predicate(), - )?; - let gs = global_state.clone(); // Arc clone - Ok(read_result_iter.map(move |read_result| -> DeltaResult<_> { - let read_result = read_result?; - // to transform the physical data into the correct logical form - let logical = transform_to_logical_internal( - engine, - read_result, - &gs, - &scan_file.partition_values, - &self.all_fields, - self.have_partition_cols, - ); - let len = logical.as_ref().map_or(0, |res| res.length()); - // need to split the dv_mask. what's left in dv_mask covers this result, and rest - // will cover the following results. we `take()` out of `selection_vector` to avoid - // trying to return a captured variable. We're going to reassign `selection_vector` - // to `rest` in a moment anyway - let mut sv = selection_vector.take(); - let rest = split_vector(sv.as_mut(), len, None); - let result = ScanResult { - raw_data: logical, - raw_mask: sv, - }; - selection_vector = rest; - Ok(result) - })) - }) - // Iterator>>> to Iterator>> - .flatten_ok() - // Iterator>> to Iterator> - .map(|x| x?); - Ok(result) - } -} From c959b1d5ddfb639c3e945e4e17e8375d2b10b141 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 8 Nov 2024 15:10:27 -0800 Subject: [PATCH 24/88] Update log segment to latest version --- kernel/src/log_segment.rs | 38 ++++++++++++++------------------------ 1 file changed, 14 insertions(+), 24 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 60c6f03a8..c49594a56 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -22,7 +22,6 @@ use itertools::Itertools; #[derive(Debug)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) struct LogSegment { - pub(crate) start_version: Version, pub(crate) end_version: Version, pub(crate) log_root: Url, /// Reverse order sorted commit files in the log segment @@ -147,25 +146,20 @@ impl<'a> LogSegmentBuilder<'a> { let _ = self.start_version.insert(version); self } - - #[allow(unused)] pub(crate) fn with_end_version(mut self, version: Version) -> Self { let _ = self.end_version.insert(version); self } - #[allow(unused)] pub(crate) fn with_no_checkpoint_files(mut self) -> Self { self.no_checkpoint_files = true; self } - #[allow(unused)] pub(crate) fn with_in_order_commit_files(mut self) -> Self { self.in_order_commit_files = true; self } - pub(crate) fn build(self) -> DeltaResult { let Self { fs_client, @@ -187,39 +181,31 @@ impl<'a> LogSegmentBuilder<'a> { _ => Self::list_log_files(fs_client.as_ref(), &log_url)?, }; - // We assume listing returned ordered. If `in_order_commit_files` is false, we want reverse order. - if !in_order_commit_files { - // We assume listing returned ordered, we want reverse order - commit_files.reverse(); - } - - // remove all files above requested version - if let Some(end_version) = end_version { - commit_files.retain(|log_path| log_path.version <= end_version); - } - // Remove checkpoint files if no_checkpoint_files { checkpoint_files.clear(); } - // only keep commit files above the checkpoint we found + // Commit file versions must satisfy the following: + // - Must be greater than the start version + // - Must be greater than the most recent checkpoint version if it exists + // - Must be less than or equal to the specified end version. + if let Some(end_version) = end_version { + commit_files.retain(|log_path| log_path.version <= end_version); + } if let Some(checkpoint_file) = checkpoint_files.first() { commit_files.retain(|log_path| checkpoint_file.version < log_path.version); } - - // only keep commit files above the checkpoint we found if let Some(start_version) = start_version { commit_files.retain(|log_path| start_version <= log_path.version); } // get the effective version from chosen files let version_eff = commit_files - .first() + .last() .or(checkpoint_files.first()) .ok_or(Error::MissingVersion)? // TODO: A more descriptive error .version; - if let Some(end_version) = end_version { require!( version_eff == end_version, @@ -227,8 +213,13 @@ impl<'a> LogSegmentBuilder<'a> { ); } + // We assume listing returned ordered. If `in_order_commit_files` is false, we want reverse order. + if !in_order_commit_files { + // We assume listing returned ordered, we want reverse order + commit_files.reverse(); + } + Ok(LogSegment { - start_version: start_version.unwrap_or(0), end_version: version_eff, log_root: log_url, commit_files: commit_files @@ -343,7 +334,6 @@ impl<'a> LogSegmentBuilder<'a> { Ok((commit_files, checkpoint_files)) } } - #[cfg(test)] mod tests { use std::{path::PathBuf, sync::Arc}; From f121f67ca67dc61e2af88b54db78ce3c9da1336c Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 8 Nov 2024 15:28:40 -0800 Subject: [PATCH 25/88] Add doc comments --- kernel/src/log_segment.rs | 39 +++++++++++++++++++++++++-------------- 1 file changed, 25 insertions(+), 14 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index c49594a56..91949bf7e 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -1,10 +1,16 @@ //! Represents a segment of a delta log. [`LogSegment`] wraps a set of checkpoint and commit //! files. +use crate::{ + actions::{get_log_schema, Metadata, Protocol, METADATA_NAME, PROTOCOL_NAME}, + schema::SchemaRef, + DeltaResult, Engine, EngineData, Error, ExpressionRef, FileMeta, +}; use crate::{ path::ParsedLogPath, snapshot::CheckpointMetadata, utils::require, Expression, FileSystemClient, Version, }; +use itertools::Itertools; use std::{ cmp::Ordering, sync::{Arc, LazyLock}, @@ -12,22 +18,15 @@ use std::{ use tracing::warn; use url::Url; -use crate::{ - actions::{get_log_schema, Metadata, Protocol, METADATA_NAME, PROTOCOL_NAME}, - schema::SchemaRef, - DeltaResult, Engine, EngineData, Error, ExpressionRef, FileMeta, -}; -use itertools::Itertools; - #[derive(Debug)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) struct LogSegment { - pub(crate) end_version: Version, - pub(crate) log_root: Url, - /// Reverse order sorted commit files in the log segment - pub(crate) commit_files: Vec, - /// checkpoint files in the log segment. - pub(crate) checkpoint_files: Vec, + pub end_version: Version, + pub log_root: Url, + /// Commit files in the log segment + pub commit_files: Vec, + /// Checkpoint files in the log segment. + pub checkpoint_files: Vec, } impl LogSegment { @@ -135,31 +134,43 @@ impl<'a> LogSegmentBuilder<'a> { in_order_commit_files: false, } } - + /// Optionally provide a checkpoint hint that results from reading the `last_checkpoint` file. pub(crate) fn with_checkpoint(mut self, checkpoint: CheckpointMetadata) -> Self { let _ = self.checkpoint.insert(checkpoint); self } + /// Optionally set the start version of the [`LogSegment`]. This ensures that all commit files + /// are above this version. #[allow(unused)] pub(crate) fn with_start_version(mut self, version: Version) -> Self { let _ = self.start_version.insert(version); self } + /// Optionally set the end version of the [`LogSegment`]. This ensures that all commit files + /// and checkpoints are below the end version. pub(crate) fn with_end_version(mut self, version: Version) -> Self { let _ = self.end_version.insert(version); self } + /// Optionally specify that the [`LogSegment`] will not have any checkpoint files. It will only + /// be made up of commit files. #[allow(unused)] pub(crate) fn with_no_checkpoint_files(mut self) -> Self { self.no_checkpoint_files = true; self } + + /// Optionally specify that the commits in the [`LogSegment`] will be in order. By default, the + /// [`LogSegment`] will reverse the order of commit files, with the latest commit coming first. #[allow(unused)] pub(crate) fn with_in_order_commit_files(mut self) -> Self { self.in_order_commit_files = true; self } + /// Build the [`LogSegment`] + /// + /// This fetches checkpoint and commit files using the `fs_client`. pub(crate) fn build(self) -> DeltaResult { let Self { fs_client, From 6fbecb7f589ccdf13af3e2c757766ac1fa08ac40 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 8 Nov 2024 16:34:43 -0800 Subject: [PATCH 26/88] Fix tests and refactor --- kernel/src/log_segment.rs | 41 ++++++++++++++------------------------- kernel/src/snapshot.rs | 2 +- 2 files changed, 16 insertions(+), 27 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 91949bf7e..05837d98e 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -114,7 +114,7 @@ impl LogSegment { } pub(crate) struct LogSegmentBuilder<'a> { - fs_client: Arc, + fs_client: &'a dyn FileSystemClient, log_root: &'a Url, checkpoint: Option, start_version: Option, @@ -123,7 +123,7 @@ pub(crate) struct LogSegmentBuilder<'a> { in_order_commit_files: bool, } impl<'a> LogSegmentBuilder<'a> { - pub(crate) fn new(fs_client: Arc, log_root: &'a Url) -> Self { + pub(crate) fn new(fs_client: &'a dyn FileSystemClient, log_root: &'a Url) -> Self { LogSegmentBuilder { fs_client, log_root, @@ -183,13 +183,16 @@ impl<'a> LogSegmentBuilder<'a> { } = self; let log_url = log_root.join("_delta_log/").unwrap(); let (mut commit_files, mut checkpoint_files) = match (checkpoint, end_version) { - (Some(cp), None) => { - Self::list_log_files_with_checkpoint(&cp, fs_client.as_ref(), &log_url)? - } + (Some(cp), None) => Self::list_log_files_with_checkpoint(&cp, fs_client, &log_url)?, (Some(cp), Some(version)) if cp.version >= version => { - Self::list_log_files_with_checkpoint(&cp, fs_client.as_ref(), &log_url)? + Self::list_log_files_with_checkpoint(&cp, fs_client, &log_url)? + } + _ => { + let (commit_files, checkpoint_files, _) = + Self::list_log_files_from_version(fs_client, &log_url, None)?; + + (commit_files, checkpoint_files) } - _ => Self::list_log_files(fs_client.as_ref(), &log_url)?, }; // Remove checkpoint files @@ -198,9 +201,9 @@ impl<'a> LogSegmentBuilder<'a> { } // Commit file versions must satisfy the following: - // - Must be greater than the start version - // - Must be greater than the most recent checkpoint version if it exists - // - Must be less than or equal to the specified end version. + // - Be greater than the start version + // - Be greater than the most recent checkpoint version if it exists + // - Be less than or equal to the end version. if let Some(end_version) = end_version { commit_files.retain(|log_path| log_path.version <= end_version); } @@ -329,21 +332,6 @@ impl<'a> LogSegmentBuilder<'a> { } Ok((commit_files, checkpoint_files)) } - - /// List relevant log files. - /// - /// Relevant files are the max checkpoint found and all subsequent commits. - pub(crate) fn list_log_files( - fs_client: &dyn FileSystemClient, - log_root: &Url, - ) -> DeltaResult<(Vec, Vec)> { - let (mut commit_files, checkpoint_files, max_checkpoint_version) = - Self::list_log_files_from_version(fs_client, log_root, None)?; - - commit_files.retain(|f| f.version as i64 > max_checkpoint_version); - - Ok((commit_files, checkpoint_files)) - } } #[cfg(test)] mod tests { @@ -469,11 +457,12 @@ mod tests { let (mut commit_files, checkpoint_files) = LogSegmentBuilder::list_log_files_with_checkpoint(&checkpoint_metadata, &client, &url) .unwrap(); - // Make the most recent commit the first in iterator commit_files.reverse(); assert_eq!(checkpoint_files.len(), 1); + println!("checkpoint: {:?}", checkpoint_files); + println!("commits: {:?}", commit_files); assert_eq!(commit_files.len(), 2); assert_eq!(checkpoint_files[0].version, 5); println!("commitfiles: {:?}", commit_files); diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 852273446..5a2268fdd 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -62,7 +62,7 @@ impl Snapshot { let fs_client = engine.get_file_system_client(); let log_url = table_root.join("_delta_log/").unwrap(); - let mut builder = LogSegmentBuilder::new(fs_client.clone(), &table_root); + let mut builder = LogSegmentBuilder::new(fs_client.as_ref(), &table_root); if let Some(version) = version { builder = builder.with_end_version(version); } From 7338834de77b34cb61b59a68b6ead3432dbcf2b3 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Sat, 9 Nov 2024 14:19:35 -0800 Subject: [PATCH 27/88] small changes --- kernel/src/log_segment.rs | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 05837d98e..f97b58b22 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -195,7 +195,6 @@ impl<'a> LogSegmentBuilder<'a> { } }; - // Remove checkpoint files if no_checkpoint_files { checkpoint_files.clear(); } @@ -204,14 +203,14 @@ impl<'a> LogSegmentBuilder<'a> { // - Be greater than the start version // - Be greater than the most recent checkpoint version if it exists // - Be less than or equal to the end version. - if let Some(end_version) = end_version { - commit_files.retain(|log_path| log_path.version <= end_version); + if let Some(start_version) = start_version { + commit_files.retain(|log_path| start_version <= log_path.version); } if let Some(checkpoint_file) = checkpoint_files.first() { commit_files.retain(|log_path| checkpoint_file.version < log_path.version); } - if let Some(start_version) = start_version { - commit_files.retain(|log_path| start_version <= log_path.version); + if let Some(end_version) = end_version { + commit_files.retain(|log_path| log_path.version <= end_version); } // get the effective version from chosen files @@ -229,7 +228,6 @@ impl<'a> LogSegmentBuilder<'a> { // We assume listing returned ordered. If `in_order_commit_files` is false, we want reverse order. if !in_order_commit_files { - // We assume listing returned ordered, we want reverse order commit_files.reverse(); } From 362900b39dd32008d0efa0c0228cc90620628263 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Sat, 9 Nov 2024 14:23:44 -0800 Subject: [PATCH 28/88] Move out checkpoint retain files --- kernel/src/log_segment.rs | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index f97b58b22..ee5d0a240 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -298,7 +298,7 @@ impl<'a> LogSegmentBuilder<'a> { fs_client: &dyn FileSystemClient, log_root: &Url, ) -> DeltaResult<(Vec, Vec)> { - let (mut commit_files, checkpoint_files, max_checkpoint_version) = + let (commit_files, checkpoint_files, max_checkpoint_version) = Self::list_log_files_from_version( fs_client, log_root, @@ -314,13 +314,10 @@ impl<'a> LogSegmentBuilder<'a> { if max_checkpoint_version != checkpoint_metadata.version as i64 { warn!( - "_last_checkpoint hint is out of date. _last_checkpoint version: {}. Using actual most recent: {}", - checkpoint_metadata.version, - max_checkpoint_version - ); - // we (may) need to drop commits that are before the _actual_ last checkpoint (that - // is, commits between a stale _last_checkpoint and the _actual_ last checkpoint) - commit_files.retain(|parsed_path| parsed_path.version as i64 > max_checkpoint_version); + "_last_checkpoint hint is out of date. _last_checkpoint version: {}. Using actual most recent: {}", + checkpoint_metadata.version, + max_checkpoint_version + ); } else if checkpoint_files.len() != checkpoint_metadata.parts.unwrap_or(1) { return Err(Error::Generic(format!( "_last_checkpoint indicated that checkpoint should have {} parts, but it has {}", @@ -455,8 +452,13 @@ mod tests { let (mut commit_files, checkpoint_files) = LogSegmentBuilder::list_log_files_with_checkpoint(&checkpoint_metadata, &client, &url) .unwrap(); + // Make the most recent commit the first in iterator commit_files.reverse(); + let max_checkpoint_version = checkpoint_files.last().unwrap().version; + // we (may) need to drop commits that are before the _actual_ last checkpoint (that + // is, commits between a stale _last_checkpoint and the _actual_ last checkpoint) + commit_files.retain(|parsed_path| parsed_path.version > max_checkpoint_version); assert_eq!(checkpoint_files.len(), 1); println!("checkpoint: {:?}", checkpoint_files); From 8c5a2186975b005f00247674cc4e5043a64120b2 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 11 Nov 2024 14:05:15 -0800 Subject: [PATCH 29/88] Address nits, upgrade to prasedlogpath --- kernel/src/log_segment.rs | 99 ++++++++++++++++++++------------------- kernel/src/snapshot.rs | 3 +- 2 files changed, 54 insertions(+), 48 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index ee5d0a240..1e54b73d6 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -24,9 +24,9 @@ pub(crate) struct LogSegment { pub end_version: Version, pub log_root: Url, /// Commit files in the log segment - pub commit_files: Vec, + pub commit_files: Vec, /// Checkpoint files in the log segment. - pub checkpoint_files: Vec, + pub checkpoint_files: Vec, } impl LogSegment { @@ -52,7 +52,11 @@ impl LogSegment { let commit_stream = engine .get_json_handler() .read_json_files( - &self.commit_files, + &self + .commit_files + .iter() + .map(|x| x.location.clone()) + .collect::>(), commit_read_schema, meta_predicate.clone(), )? @@ -61,7 +65,11 @@ impl LogSegment { let checkpoint_stream = engine .get_parquet_handler() .read_parquet_files( - &self.checkpoint_files, + &self + .checkpoint_files + .iter() + .map(|x| x.location.clone()) + .collect::>(), checkpoint_read_schema, meta_predicate, )? @@ -115,28 +123,28 @@ impl LogSegment { pub(crate) struct LogSegmentBuilder<'a> { fs_client: &'a dyn FileSystemClient, - log_root: &'a Url, + table_root: &'a Url, checkpoint: Option, start_version: Option, end_version: Option, - no_checkpoint_files: bool, - in_order_commit_files: bool, + omit_checkpoint_files: bool, + reversed_commit_files: bool, } impl<'a> LogSegmentBuilder<'a> { - pub(crate) fn new(fs_client: &'a dyn FileSystemClient, log_root: &'a Url) -> Self { + pub(crate) fn new(fs_client: &'a dyn FileSystemClient, table_root: &'a Url) -> Self { LogSegmentBuilder { fs_client, - log_root, + table_root, checkpoint: None, start_version: None, end_version: None, - no_checkpoint_files: false, - in_order_commit_files: false, + omit_checkpoint_files: false, + reversed_commit_files: false, } } /// Optionally provide a checkpoint hint that results from reading the `last_checkpoint` file. pub(crate) fn with_checkpoint(mut self, checkpoint: CheckpointMetadata) -> Self { - let _ = self.checkpoint.insert(checkpoint); + self.checkpoint = Some(checkpoint); self } @@ -144,28 +152,28 @@ impl<'a> LogSegmentBuilder<'a> { /// are above this version. #[allow(unused)] pub(crate) fn with_start_version(mut self, version: Version) -> Self { - let _ = self.start_version.insert(version); + self.start_version = Some(version); self } /// Optionally set the end version of the [`LogSegment`]. This ensures that all commit files /// and checkpoints are below the end version. pub(crate) fn with_end_version(mut self, version: Version) -> Self { - let _ = self.end_version.insert(version); + self.end_version = Some(version); self } /// Optionally specify that the [`LogSegment`] will not have any checkpoint files. It will only /// be made up of commit files. #[allow(unused)] - pub(crate) fn with_no_checkpoint_files(mut self) -> Self { - self.no_checkpoint_files = true; + pub(crate) fn omit_checkpoint_files(mut self) -> Self { + self.omit_checkpoint_files = true; self } /// Optionally specify that the commits in the [`LogSegment`] will be in order. By default, the /// [`LogSegment`] will reverse the order of commit files, with the latest commit coming first. #[allow(unused)] - pub(crate) fn with_in_order_commit_files(mut self) -> Self { - self.in_order_commit_files = true; + pub(crate) fn with_reversed_commit_files(mut self) -> Self { + self.reversed_commit_files = true; self } /// Build the [`LogSegment`] @@ -174,28 +182,33 @@ impl<'a> LogSegmentBuilder<'a> { pub(crate) fn build(self) -> DeltaResult { let Self { fs_client, - log_root, + table_root, checkpoint, start_version, end_version, - no_checkpoint_files, - in_order_commit_files, + omit_checkpoint_files, + reversed_commit_files, } = self; - let log_url = log_root.join("_delta_log/").unwrap(); + let log_root = table_root.join("_delta_log/").unwrap(); let (mut commit_files, mut checkpoint_files) = match (checkpoint, end_version) { - (Some(cp), None) => Self::list_log_files_with_checkpoint(&cp, fs_client, &log_url)?, + (Some(cp), None) => { + println!("List log files with checkpoint"); + let x = Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)?; + println!("x: {:?}", x); + x + } (Some(cp), Some(version)) if cp.version >= version => { - Self::list_log_files_with_checkpoint(&cp, fs_client, &log_url)? + Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)? } _ => { let (commit_files, checkpoint_files, _) = - Self::list_log_files_from_version(fs_client, &log_url, None)?; + Self::list_log_files_from_version(fs_client, &log_root, None)?; (commit_files, checkpoint_files) } }; - if no_checkpoint_files { + if omit_checkpoint_files { checkpoint_files.clear(); } @@ -227,21 +240,15 @@ impl<'a> LogSegmentBuilder<'a> { } // We assume listing returned ordered. If `in_order_commit_files` is false, we want reverse order. - if !in_order_commit_files { + if reversed_commit_files { commit_files.reverse(); } Ok(LogSegment { end_version: version_eff, - log_root: log_url, - commit_files: commit_files - .into_iter() - .map(|log_path| log_path.location) - .collect(), - checkpoint_files: checkpoint_files - .into_iter() - .map(|log_path| log_path.location) - .collect(), + log_root, + commit_files, + checkpoint_files, }) } pub(crate) fn list_log_files_from_version( @@ -448,17 +455,15 @@ mod tests { Arc::new(TokioBackgroundExecutor::new()), ); - let url = Url::parse("memory:///_delta_log/").expect("valid url"); - let (mut commit_files, checkpoint_files) = - LogSegmentBuilder::list_log_files_with_checkpoint(&checkpoint_metadata, &client, &url) - .unwrap(); - - // Make the most recent commit the first in iterator - commit_files.reverse(); - let max_checkpoint_version = checkpoint_files.last().unwrap().version; - // we (may) need to drop commits that are before the _actual_ last checkpoint (that - // is, commits between a stale _last_checkpoint and the _actual_ last checkpoint) - commit_files.retain(|parsed_path| parsed_path.version > max_checkpoint_version); + let table_root = Url::parse("memory:///").expect("valid url"); + + let log_segment = LogSegmentBuilder::new(&client, &table_root) + .with_reversed_commit_files() + .with_checkpoint(checkpoint_metadata) + .build() + .unwrap(); + let (commit_files, checkpoint_files) = + (log_segment.commit_files, log_segment.checkpoint_files); assert_eq!(checkpoint_files.len(), 1); println!("checkpoint: {:?}", checkpoint_files); diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 5a2268fdd..59ea983a9 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -62,7 +62,8 @@ impl Snapshot { let fs_client = engine.get_file_system_client(); let log_url = table_root.join("_delta_log/").unwrap(); - let mut builder = LogSegmentBuilder::new(fs_client.as_ref(), &table_root); + let mut builder = + LogSegmentBuilder::new(fs_client.as_ref(), &table_root).with_reversed_commit_files(); if let Some(version) = version { builder = builder.with_end_version(version); } From 5dcff78d42f853be7e40998658bfedb9eec299ff Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 11 Nov 2024 18:43:45 -0800 Subject: [PATCH 30/88] Address ommit --- kernel/src/log_segment.rs | 23 ++++++----------------- 1 file changed, 6 insertions(+), 17 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 101a2f978..c0738b0e5 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -121,13 +121,13 @@ impl LogSegment { } } +/// Builder for [`LogSegment`] from from `start_version` to `end_version` inclusive pub(crate) struct LogSegmentBuilder<'a> { fs_client: &'a dyn FileSystemClient, table_root: &'a Url, checkpoint: Option, start_version: Option, end_version: Option, - omit_checkpoint_files: bool, reversed_commit_files: bool, } impl<'a> LogSegmentBuilder<'a> { @@ -138,7 +138,6 @@ impl<'a> LogSegmentBuilder<'a> { checkpoint: None, start_version: None, end_version: None, - omit_checkpoint_files: false, reversed_commit_files: false, } } @@ -161,14 +160,6 @@ impl<'a> LogSegmentBuilder<'a> { self.end_version = Some(version); self } - /// Optionally specify that the [`LogSegment`] will not have any checkpoint files. It will only - /// be made up of commit files. - #[allow(unused)] - pub(crate) fn omit_checkpoint_files(mut self) -> Self { - self.omit_checkpoint_files = true; - self - } - /// Optionally specify that the commits in the [`LogSegment`] will be in order. By default, the /// [`LogSegment`] will reverse the order of commit files, with the latest commit coming first. #[allow(unused)] @@ -186,11 +177,10 @@ impl<'a> LogSegmentBuilder<'a> { checkpoint, start_version, end_version, - omit_checkpoint_files, reversed_commit_files, } = self; let log_root = table_root.join("_delta_log/").unwrap(); - let (mut commit_files, mut checkpoint_files) = match (checkpoint, end_version) { + let (mut commit_files, checkpoint_files) = match (checkpoint, end_version) { (Some(cp), None) => Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)?, (Some(cp), Some(version)) if cp.version >= version => { Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)? @@ -203,10 +193,6 @@ impl<'a> LogSegmentBuilder<'a> { } }; - if omit_checkpoint_files { - checkpoint_files.clear(); - } - // Commit file versions must satisfy the following: // - Be greater than the start version // - Be greater than the most recent checkpoint version if it exists @@ -215,7 +201,10 @@ impl<'a> LogSegmentBuilder<'a> { commit_files.retain(|log_path| start_version <= log_path.version); } if let Some(checkpoint_file) = checkpoint_files.first() { - commit_files.retain(|log_path| checkpoint_file.version < log_path.version); + commit_files.retain(|log_path| { + checkpoint_file.version < log_path.version + || start_version.is_some_and(|start_version| start_version >= log_path.version) + }); } if let Some(end_version) = end_version { commit_files.retain(|log_path| log_path.version <= end_version); From ba079e616d4e238534905dc29cf3c397aceeb2c0 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 11 Nov 2024 19:10:07 -0800 Subject: [PATCH 31/88] Fix omit --- kernel/src/log_segment.rs | 30 +++++++++++++----------------- 1 file changed, 13 insertions(+), 17 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index c0738b0e5..dc5b78367 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -1,20 +1,18 @@ //! Represents a segment of a delta log. [`LogSegment`] wraps a set of checkpoint and commit //! files. +use crate::actions::{get_log_schema, Metadata, Protocol, METADATA_NAME, PROTOCOL_NAME}; +use crate::path::ParsedLogPath; +use crate::schema::SchemaRef; +use crate::snapshot::CheckpointMetadata; +use crate::utils::require; use crate::{ - actions::{get_log_schema, Metadata, Protocol, METADATA_NAME, PROTOCOL_NAME}, - schema::SchemaRef, - DeltaResult, Engine, EngineData, Error, ExpressionRef, FileMeta, -}; -use crate::{ - path::ParsedLogPath, snapshot::CheckpointMetadata, utils::require, Expression, - FileSystemClient, Version, + DeltaResult, Engine, EngineData, Error, Expression, ExpressionRef, FileMeta, FileSystemClient, + Version, }; use itertools::Itertools; -use std::{ - cmp::Ordering, - sync::{Arc, LazyLock}, -}; +use std::cmp::Ordering; +use std::sync::{Arc, LazyLock}; use tracing::warn; use url::Url; @@ -148,7 +146,7 @@ impl<'a> LogSegmentBuilder<'a> { } /// Optionally set the start version of the [`LogSegment`]. This ensures that all commit files - /// are above this version. + /// are above this version. Checkpoint files will be omitted if this is specified. #[allow(unused)] pub(crate) fn with_start_version(mut self, version: Version) -> Self { self.start_version = Some(version); @@ -180,7 +178,7 @@ impl<'a> LogSegmentBuilder<'a> { reversed_commit_files, } = self; let log_root = table_root.join("_delta_log/").unwrap(); - let (mut commit_files, checkpoint_files) = match (checkpoint, end_version) { + let (mut commit_files, mut checkpoint_files) = match (checkpoint, end_version) { (Some(cp), None) => Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)?, (Some(cp), Some(version)) if cp.version >= version => { Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)? @@ -198,13 +196,11 @@ impl<'a> LogSegmentBuilder<'a> { // - Be greater than the most recent checkpoint version if it exists // - Be less than or equal to the end version. if let Some(start_version) = start_version { + checkpoint_files.clear(); commit_files.retain(|log_path| start_version <= log_path.version); } if let Some(checkpoint_file) = checkpoint_files.first() { - commit_files.retain(|log_path| { - checkpoint_file.version < log_path.version - || start_version.is_some_and(|start_version| start_version >= log_path.version) - }); + commit_files.retain(|log_path| checkpoint_file.version < log_path.version); } if let Some(end_version) = end_version { commit_files.retain(|log_path| log_path.version <= end_version); From ccaed090e5b7255625bb08689d0113ce92ba98c5 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 11 Nov 2024 19:12:13 -0800 Subject: [PATCH 32/88] Address nit --- kernel/src/log_segment.rs | 30 ++++++++++++------------------ 1 file changed, 12 insertions(+), 18 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index dc5b78367..13e5eb610 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -47,30 +47,24 @@ impl LogSegment { checkpoint_read_schema: SchemaRef, meta_predicate: Option, ) -> DeltaResult, bool)>> + Send> { + let commit_files: Vec<_> = self + .commit_files + .iter() + .map(|f| f.location.clone()) + .collect(); let commit_stream = engine .get_json_handler() - .read_json_files( - &self - .commit_files - .iter() - .map(|f| f.location.clone()) - .collect::>(), - commit_read_schema, - meta_predicate.clone(), - )? + .read_json_files(&commit_files, commit_read_schema, meta_predicate.clone())? .map_ok(|batch| (batch, true)); + let checkpoint_files: Vec<_> = self + .checkpoint_files + .iter() + .map(|f| f.location.clone()) + .collect(); let checkpoint_stream = engine .get_parquet_handler() - .read_parquet_files( - &self - .checkpoint_files - .iter() - .map(|f| f.location.clone()) - .collect::>(), - checkpoint_read_schema, - meta_predicate, - )? + .read_parquet_files(&checkpoint_files, checkpoint_read_schema, meta_predicate)? .map_ok(|batch| (batch, false)); Ok(commit_stream.chain(checkpoint_stream)) From 62be3d78f148a1fbf34be85907cadda8b252452b Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 11 Nov 2024 19:14:00 -0800 Subject: [PATCH 33/88] Apply suggestions from code review Co-authored-by: Zach Schuermann --- kernel/src/log_segment.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 13e5eb610..5b550aac3 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -139,15 +139,15 @@ impl<'a> LogSegmentBuilder<'a> { self } - /// Optionally set the start version of the [`LogSegment`]. This ensures that all commit files + /// Optionally set the start version (inclusive) of the [`LogSegment`]. This ensures that all commit files /// are above this version. Checkpoint files will be omitted if this is specified. #[allow(unused)] pub(crate) fn with_start_version(mut self, version: Version) -> Self { self.start_version = Some(version); self } - /// Optionally set the end version of the [`LogSegment`]. This ensures that all commit files - /// and checkpoints are below the end version. + /// Optionally set the end version (inclusive) of the [`LogSegment`]. This ensures that all commit files + /// and checkpoints are at or below the end version. pub(crate) fn with_end_version(mut self, version: Version) -> Self { self.end_version = Some(version); self @@ -191,7 +191,7 @@ impl<'a> LogSegmentBuilder<'a> { // - Be less than or equal to the end version. if let Some(start_version) = start_version { checkpoint_files.clear(); - commit_files.retain(|log_path| start_version <= log_path.version); + commit_files.retain(|log_path| log_path.version >= start_version); } if let Some(checkpoint_file) = checkpoint_files.first() { commit_files.retain(|log_path| checkpoint_file.version < log_path.version); From d22d661d8c7575ff71aa6f0734581691152d5ab2 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 11 Nov 2024 19:15:13 -0800 Subject: [PATCH 34/88] address nit --- kernel/src/log_segment.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 13e5eb610..d3d20a470 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -7,8 +7,7 @@ use crate::schema::SchemaRef; use crate::snapshot::CheckpointMetadata; use crate::utils::require; use crate::{ - DeltaResult, Engine, EngineData, Error, Expression, ExpressionRef, FileMeta, FileSystemClient, - Version, + DeltaResult, Engine, EngineData, Error, Expression, ExpressionRef, FileSystemClient, Version, }; use itertools::Itertools; use std::cmp::Ordering; @@ -140,7 +139,7 @@ impl<'a> LogSegmentBuilder<'a> { } /// Optionally set the start version of the [`LogSegment`]. This ensures that all commit files - /// are above this version. Checkpoint files will be omitted if this is specified. + /// are at or above this version. Checkpoint files will be omitted if `start_version` is specified. #[allow(unused)] pub(crate) fn with_start_version(mut self, version: Version) -> Self { self.start_version = Some(version); From 4b79a333a8dacc15d346c6c7306f3420a959631c Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Mon, 11 Nov 2024 19:17:33 -0800 Subject: [PATCH 35/88] fix comment --- kernel/src/log_segment.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 60a514adb..3160fc297 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -139,7 +139,7 @@ impl<'a> LogSegmentBuilder<'a> { } /// Optionally set the start version of the [`LogSegment`]. This ensures that all commit files - /// are at or above this version. Checkpoint files will be omitted if `start_version` is specified. + /// are at this version or above it. Checkpoint files will be omitted if `start_version` is specified. #[allow(unused)] pub(crate) fn with_start_version(mut self, version: Version) -> Self { self.start_version = Some(version); From 8aec522901a1e19083c61196203e90adf7d750a4 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Tue, 12 Nov 2024 11:35:54 -0800 Subject: [PATCH 36/88] Add docs and bring back omit --- kernel/src/log_segment.rs | 32 +++++++++++++++++++++++++++++--- 1 file changed, 29 insertions(+), 3 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 3160fc297..69a5c5f82 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -15,6 +15,18 @@ use std::sync::{Arc, LazyLock}; use tracing::warn; use url::Url; +/// A [`LogSegment`] represents a contiguous section of the log and is made up of checkpoint files +/// and commit files. It is built with [`LogSegmentBuilder`], and guarantees the following: +/// 1. Commit file versions will be less than or equal to `end_version` if specified. +/// 2. Commit file versions will be greater than or equal to `start_version` if specified. +/// 3. Commit file versions will be greater than the most recent checkpoint's version. +/// +/// [`LogSegment`] is used in both [`Snapshot`] and in [`TableChanges`] to hold commit files and +/// checkpoint files. +/// - For a Snapshot at version `n`: Its LogSegment is made up of zero or one checkpoint, and all +/// commits between the checkpoint and the end version `n`. +/// - For a TableChanges between versions `a` and `b`: Its LogSegment is made up of zero +/// checkpoints and all commits between versions `a` and `b` #[derive(Debug)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) struct LogSegment { @@ -120,6 +132,7 @@ pub(crate) struct LogSegmentBuilder<'a> { start_version: Option, end_version: Option, reversed_commit_files: bool, + omit_checkpoint_files: bool, } impl<'a> LogSegmentBuilder<'a> { pub(crate) fn new(fs_client: &'a dyn FileSystemClient, table_root: &'a Url) -> Self { @@ -130,6 +143,7 @@ impl<'a> LogSegmentBuilder<'a> { start_version: None, end_version: None, reversed_commit_files: false, + omit_checkpoint_files: false, } } /// Optionally provide a checkpoint hint that results from reading the `last_checkpoint` file. @@ -139,7 +153,7 @@ impl<'a> LogSegmentBuilder<'a> { } /// Optionally set the start version of the [`LogSegment`]. This ensures that all commit files - /// are at this version or above it. Checkpoint files will be omitted if `start_version` is specified. + /// are at this version or above it. #[allow(unused)] pub(crate) fn with_start_version(mut self, version: Version) -> Self { self.start_version = Some(version); @@ -151,6 +165,14 @@ impl<'a> LogSegmentBuilder<'a> { self.end_version = Some(version); self } + + /// Optionally specify that the [`LogSegment`] will not have any checkpoint files. It will only + /// be made up of commit files. + #[allow(unused)] + pub(crate) fn omit_checkpoint_files(mut self) -> Self { + self.omit_checkpoint_files = true; + self + } /// Optionally specify that the commits in the [`LogSegment`] will be in order. By default, the /// [`LogSegment`] will reverse the order of commit files, with the latest commit coming first. #[allow(unused)] @@ -169,6 +191,7 @@ impl<'a> LogSegmentBuilder<'a> { start_version, end_version, reversed_commit_files, + omit_checkpoint_files, } = self; let log_root = table_root.join("_delta_log/").unwrap(); let (mut commit_files, mut checkpoint_files) = match (checkpoint, end_version) { @@ -184,12 +207,15 @@ impl<'a> LogSegmentBuilder<'a> { } }; + if omit_checkpoint_files { + checkpoint_files.clear(); + } + // Commit file versions must satisfy the following: - // - Be greater than the start version + // - Be greater than or equal to the start version // - Be greater than the most recent checkpoint version if it exists // - Be less than or equal to the end version. if let Some(start_version) = start_version { - checkpoint_files.clear(); commit_files.retain(|log_path| log_path.version >= start_version); } if let Some(checkpoint_file) = checkpoint_files.first() { From d88e31e4822b5ac2f7647c84a14c146b00f876e4 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Tue, 12 Nov 2024 17:07:06 -0800 Subject: [PATCH 37/88] Apply suggestions from code review Co-authored-by: Zach Schuermann --- kernel/src/log_segment.rs | 22 ++++++++++------------ kernel/src/snapshot.rs | 1 - 2 files changed, 10 insertions(+), 13 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 69a5c5f82..bfc34923b 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -17,9 +17,11 @@ use url::Url; /// A [`LogSegment`] represents a contiguous section of the log and is made up of checkpoint files /// and commit files. It is built with [`LogSegmentBuilder`], and guarantees the following: -/// 1. Commit file versions will be less than or equal to `end_version` if specified. -/// 2. Commit file versions will be greater than or equal to `start_version` if specified. -/// 3. Commit file versions will be greater than the most recent checkpoint's version. +/// 1. Commit/checkpoint file versions will be less than or equal to `end_version` if specified. +/// 2. Commit/checkpoint file versions will be greater than or equal to `start_version` if specified. +/// 3. If checkpoint(s) is/are present in the range, only commits with versions greater than the most +/// recent checkpoint version are retained. Checkpoints can be omitted (and this rule skipped) +/// whenever the `LogSegment` is created. See [`LogSegmentBuilder::with_omit_checkpoint_files`]. /// /// [`LogSegment`] is used in both [`Snapshot`] and in [`TableChanges`] to hold commit files and /// checkpoint files. @@ -128,7 +130,7 @@ impl LogSegment { pub(crate) struct LogSegmentBuilder<'a> { fs_client: &'a dyn FileSystemClient, table_root: &'a Url, - checkpoint: Option, + start_checkpoint: Option, start_version: Option, end_version: Option, reversed_commit_files: bool, @@ -146,8 +148,8 @@ impl<'a> LogSegmentBuilder<'a> { omit_checkpoint_files: false, } } - /// Optionally provide a checkpoint hint that results from reading the `last_checkpoint` file. - pub(crate) fn with_checkpoint(mut self, checkpoint: CheckpointMetadata) -> Self { + /// Optionally provide checkpoint metadata to start the log segment from (e.g. from reading the `last_checkpoint` file). + pub(crate) fn with_start_checkpoint(mut self, checkpoint: CheckpointMetadata) -> Self { self.checkpoint = Some(checkpoint); self } @@ -267,13 +269,12 @@ impl<'a> LogSegmentBuilder<'a> { for meta_res in fs_client.list_from(&start_from)? { let meta = meta_res?; - let parsed_path = ParsedLogPath::try_from(meta)?; // TODO this filters out .crc files etc which start with "." - how do we want to use these kind of files? - if let Some(parsed_path) = parsed_path { + if let Some(parsed_path) = ParsedLogPath::try_from(meta)? { if parsed_path.is_commit() { commit_files.push(parsed_path); } else if parsed_path.is_checkpoint() { - let path_version = parsed_path.version as i64; + let path_version = parsed_path.version.try_into()?; match path_version.cmp(&max_checkpoint_version) { Ordering::Greater => { max_checkpoint_version = path_version; @@ -465,11 +466,8 @@ mod tests { (log_segment.commit_files, log_segment.checkpoint_files); assert_eq!(checkpoint_files.len(), 1); - println!("checkpoint: {:?}", checkpoint_files); - println!("commits: {:?}", commit_files); assert_eq!(commit_files.len(), 2); assert_eq!(checkpoint_files[0].version, 5); - println!("commitfiles: {:?}", commit_files); assert_eq!(commit_files[0].version, 7); assert_eq!(commit_files[1].version, 6); } diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 59ea983a9..4b07f62e3 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -207,7 +207,6 @@ mod tests { use crate::engine::default::executor::tokio::TokioBackgroundExecutor; use crate::engine::default::filesystem::ObjectStoreFileSystemClient; use crate::engine::sync::SyncEngine; - use crate::path::ParsedLogPath; use crate::schema::StructType; #[test] From 4370737c6c122b18c44621101a9ab6a227cd7dd4 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 10:11:33 -0800 Subject: [PATCH 38/88] Address pr comments --- kernel/src/log_segment.rs | 147 ++++++++++++++++++++------------------ kernel/src/snapshot.rs | 6 +- 2 files changed, 81 insertions(+), 72 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index bfc34923b..4090e51ad 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -19,7 +19,7 @@ use url::Url; /// and commit files. It is built with [`LogSegmentBuilder`], and guarantees the following: /// 1. Commit/checkpoint file versions will be less than or equal to `end_version` if specified. /// 2. Commit/checkpoint file versions will be greater than or equal to `start_version` if specified. -/// 3. If checkpoint(s) is/are present in the range, only commits with versions greater than the most +/// 3. If checkpoint(s) is/are present in the range, only commits with versions greater than the most /// recent checkpoint version are retained. Checkpoints can be omitted (and this rule skipped) /// whenever the `LogSegment` is created. See [`LogSegmentBuilder::with_omit_checkpoint_files`]. /// @@ -133,7 +133,7 @@ pub(crate) struct LogSegmentBuilder<'a> { start_checkpoint: Option, start_version: Option, end_version: Option, - reversed_commit_files: bool, + commit_files_sorted_ascending: bool, omit_checkpoint_files: bool, } impl<'a> LogSegmentBuilder<'a> { @@ -141,21 +141,27 @@ impl<'a> LogSegmentBuilder<'a> { LogSegmentBuilder { fs_client, table_root, - checkpoint: None, + start_checkpoint: None, start_version: None, end_version: None, - reversed_commit_files: false, + commit_files_sorted_ascending: false, omit_checkpoint_files: false, } } /// Optionally provide checkpoint metadata to start the log segment from (e.g. from reading the `last_checkpoint` file). - pub(crate) fn with_start_checkpoint(mut self, checkpoint: CheckpointMetadata) -> Self { - self.checkpoint = Some(checkpoint); + /// + /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] + /// with both will result in an error. + pub(crate) fn with_start_checkpoint(mut self, start_checkpoint: CheckpointMetadata) -> Self { + self.start_checkpoint = Some(start_checkpoint); self } /// Optionally set the start version of the [`LogSegment`]. This ensures that all commit files /// are at this version or above it. + /// + /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] + /// with both will result in an error. #[allow(unused)] pub(crate) fn with_start_version(mut self, version: Version) -> Self { self.start_version = Some(version); @@ -178,39 +184,40 @@ impl<'a> LogSegmentBuilder<'a> { /// Optionally specify that the commits in the [`LogSegment`] will be in order. By default, the /// [`LogSegment`] will reverse the order of commit files, with the latest commit coming first. #[allow(unused)] - pub(crate) fn with_reversed_commit_files(mut self) -> Self { - self.reversed_commit_files = true; + pub(crate) fn with_commit_files_sorted_ascending(mut self) -> Self { + self.commit_files_sorted_ascending = true; self } /// Build the [`LogSegment`] /// /// This fetches checkpoint and commit files using the `fs_client`. pub(crate) fn build(self) -> DeltaResult { + if self.start_version.is_some() && self.start_checkpoint.is_some() { + return Err(Error::generic("Failed to build LogSegment: Cannot specify both start_version and start_checkpoint")); + } let Self { fs_client, table_root, - checkpoint, + start_checkpoint, start_version, end_version, - reversed_commit_files, + commit_files_sorted_ascending, omit_checkpoint_files, } = self; let log_root = table_root.join("_delta_log/").unwrap(); - let (mut commit_files, mut checkpoint_files) = match (checkpoint, end_version) { - (Some(cp), None) => Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)?, - (Some(cp), Some(version)) if cp.version >= version => { - Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)? - } - _ => { - let (commit_files, checkpoint_files, _) = - Self::list_log_files_from_version(fs_client, &log_root, None)?; - - (commit_files, checkpoint_files) - } - }; + let (mut sorted_commit_files, mut sorted_checkpoint_files) = + match (start_checkpoint, end_version) { + (Some(cp), None) => { + Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)? + } + (Some(cp), Some(version)) if cp.version <= version => { + Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)? + } + _ => Self::list_log_files_from_version(fs_client, &log_root, None)?, + }; if omit_checkpoint_files { - checkpoint_files.clear(); + sorted_checkpoint_files.clear(); } // Commit file versions must satisfy the following: @@ -218,19 +225,19 @@ impl<'a> LogSegmentBuilder<'a> { // - Be greater than the most recent checkpoint version if it exists // - Be less than or equal to the end version. if let Some(start_version) = start_version { - commit_files.retain(|log_path| log_path.version >= start_version); + sorted_commit_files.retain(|log_path| log_path.version >= start_version); } - if let Some(checkpoint_file) = checkpoint_files.first() { - commit_files.retain(|log_path| checkpoint_file.version < log_path.version); + if let Some(checkpoint_file) = sorted_checkpoint_files.first() { + sorted_commit_files.retain(|log_path| checkpoint_file.version < log_path.version); } if let Some(end_version) = end_version { - commit_files.retain(|log_path| log_path.version <= end_version); + sorted_commit_files.retain(|log_path| log_path.version <= end_version); } // get the effective version from chosen files - let version_eff = commit_files + let version_eff = sorted_commit_files .last() - .or(checkpoint_files.first()) + .or(sorted_checkpoint_files.first()) .ok_or(Error::MissingVersion)? // TODO: A more descriptive error .version; if let Some(end_version) = end_version { @@ -240,28 +247,29 @@ impl<'a> LogSegmentBuilder<'a> { ); } - // We assume listing returned ordered. If `in_order_commit_files` is false, we want reverse order. - if reversed_commit_files { - commit_files.reverse(); + // We assume commit files are sorted in ascending order. If `commit_files_sorted_ascending` + // is false, reverse to make it descending. + if !commit_files_sorted_ascending { + sorted_commit_files.reverse(); } Ok(LogSegment { end_version: version_eff, log_root, - commit_files, - checkpoint_files, + commit_files: sorted_commit_files, + checkpoint_files: sorted_checkpoint_files, }) } pub(crate) fn list_log_files_from_version( fs_client: &dyn FileSystemClient, log_root: &Url, version: Option, - ) -> DeltaResult<(Vec, Vec, i64)> { + ) -> DeltaResult<(Vec, Vec)> { let begin_version = version.unwrap_or(0); let version_prefix = format!("{:020}", begin_version); let start_from = log_root.join(&version_prefix)?; - let mut max_checkpoint_version = version.map_or(-1, |x| x as i64); + let mut max_checkpoint_version = version; let mut checkpoint_files = vec![]; // We expect 10 commit files per checkpoint, so start with that size. We could adjust this based // on config at some point @@ -270,20 +278,27 @@ impl<'a> LogSegmentBuilder<'a> { for meta_res in fs_client.list_from(&start_from)? { let meta = meta_res?; // TODO this filters out .crc files etc which start with "." - how do we want to use these kind of files? - if let Some(parsed_path) = ParsedLogPath::try_from(meta)? { - if parsed_path.is_commit() { - commit_files.push(parsed_path); - } else if parsed_path.is_checkpoint() { - let path_version = parsed_path.version.try_into()?; - match path_version.cmp(&max_checkpoint_version) { + let Some(parsed_path) = ParsedLogPath::try_from(meta)? else { + continue; + }; + if parsed_path.is_commit() { + commit_files.push(parsed_path); + } else if parsed_path.is_checkpoint() { + let path_version = parsed_path.version; + match max_checkpoint_version { + None => { + checkpoint_files.push(parsed_path); + max_checkpoint_version = Some(path_version); + } + Some(checkpoint_version) => match path_version.cmp(&checkpoint_version) { Ordering::Greater => { - max_checkpoint_version = path_version; + max_checkpoint_version = Some(path_version); checkpoint_files.clear(); checkpoint_files.push(parsed_path); } Ordering::Equal => checkpoint_files.push(parsed_path), Ordering::Less => {} - } + }, } } } @@ -296,7 +311,7 @@ impl<'a> LogSegmentBuilder<'a> { commit_files ); - Ok((commit_files, checkpoint_files, max_checkpoint_version)) + Ok((commit_files, checkpoint_files)) } /// List all log files after a given checkpoint. @@ -305,25 +320,24 @@ impl<'a> LogSegmentBuilder<'a> { fs_client: &dyn FileSystemClient, log_root: &Url, ) -> DeltaResult<(Vec, Vec)> { - let (commit_files, checkpoint_files, max_checkpoint_version) = - Self::list_log_files_from_version( - fs_client, - log_root, - Some(checkpoint_metadata.version), - )?; - - if checkpoint_files.is_empty() { + let (commit_files, checkpoint_files) = Self::list_log_files_from_version( + fs_client, + log_root, + Some(checkpoint_metadata.version), + )?; + + let Some(latest_checkpoint) = checkpoint_files.last() else { // TODO: We could potentially recover here return Err(Error::generic( "Had a _last_checkpoint hint but didn't find any checkpoints", )); - } + }; - if max_checkpoint_version != checkpoint_metadata.version as i64 { + if latest_checkpoint.version != checkpoint_metadata.version { warn!( "_last_checkpoint hint is out of date. _last_checkpoint version: {}. Using actual most recent: {}", checkpoint_metadata.version, - max_checkpoint_version + latest_checkpoint.version ); } else if checkpoint_files.len() != checkpoint_metadata.parts.unwrap_or(1) { return Err(Error::Generic(format!( @@ -343,17 +357,12 @@ mod tests { use object_store::{memory::InMemory, path::Path, ObjectStore}; use url::Url; - use crate::{ - engine::{ - default::{ - executor::tokio::TokioBackgroundExecutor, filesystem::ObjectStoreFileSystemClient, - }, - sync::SyncEngine, - }, - log_segment::LogSegmentBuilder, - snapshot::CheckpointMetadata, - Table, - }; + use crate::engine::default::executor::tokio::TokioBackgroundExecutor; + use crate::engine::default::filesystem::ObjectStoreFileSystemClient; + use crate::engine::sync::SyncEngine; + use crate::log_segment::LogSegmentBuilder; + use crate::snapshot::CheckpointMetadata; + use crate::Table; // NOTE: In addition to testing the meta-predicate for metadata replay, this test also verifies // that the parquet reader properly infers nullcount = rowcount for missing columns. The two @@ -458,8 +467,8 @@ mod tests { let table_root = Url::parse("memory:///").expect("valid url"); let log_segment = LogSegmentBuilder::new(&client, &table_root) - .with_reversed_commit_files() - .with_checkpoint(checkpoint_metadata) + .with_commit_files_sorted_ascending() + .with_start_checkpoint(checkpoint_metadata) .build() .unwrap(); let (commit_files, checkpoint_files) = diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 4b07f62e3..09a6a2183 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -62,13 +62,12 @@ impl Snapshot { let fs_client = engine.get_file_system_client(); let log_url = table_root.join("_delta_log/").unwrap(); - let mut builder = - LogSegmentBuilder::new(fs_client.as_ref(), &table_root).with_reversed_commit_files(); + let mut builder = LogSegmentBuilder::new(fs_client.as_ref(), &table_root); if let Some(version) = version { builder = builder.with_end_version(version); } if let Some(checkpoint) = read_last_checkpoint(fs_client.as_ref(), &log_url)? { - builder = builder.with_checkpoint(checkpoint); + builder = builder.with_start_checkpoint(checkpoint); } let log_segment = builder.build()?; @@ -207,6 +206,7 @@ mod tests { use crate::engine::default::executor::tokio::TokioBackgroundExecutor; use crate::engine::default::filesystem::ObjectStoreFileSystemClient; use crate::engine::sync::SyncEngine; + use crate::path::ParsedLogPath; use crate::schema::StructType; #[test] From 6fccf5808e6576aacadf2b4683245ce3666e7907 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 10:12:55 -0800 Subject: [PATCH 39/88] Change checkpoint parts --- kernel/src/log_segment.rs | 62 +++++++++++++++++++-------------------- 1 file changed, 31 insertions(+), 31 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 4090e51ad..325f120e1 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -21,7 +21,7 @@ use url::Url; /// 2. Commit/checkpoint file versions will be greater than or equal to `start_version` if specified. /// 3. If checkpoint(s) is/are present in the range, only commits with versions greater than the most /// recent checkpoint version are retained. Checkpoints can be omitted (and this rule skipped) -/// whenever the `LogSegment` is created. See [`LogSegmentBuilder::with_omit_checkpoint_files`]. +/// whenever the `LogSegment` is created. See [`LogSegmentBuilder::with_omit_checkpoint_parts`]. /// /// [`LogSegment`] is used in both [`Snapshot`] and in [`TableChanges`] to hold commit files and /// checkpoint files. @@ -37,7 +37,7 @@ pub(crate) struct LogSegment { /// Commit files in the log segment pub commit_files: Vec, /// Checkpoint files in the log segment. - pub checkpoint_files: Vec, + pub checkpoint_parts: Vec, } impl LogSegment { @@ -70,14 +70,14 @@ impl LogSegment { .read_json_files(&commit_files, commit_read_schema, meta_predicate.clone())? .map_ok(|batch| (batch, true)); - let checkpoint_files: Vec<_> = self - .checkpoint_files + let checkpoint_parts: Vec<_> = self + .checkpoint_parts .iter() .map(|f| f.location.clone()) .collect(); let checkpoint_stream = engine .get_parquet_handler() - .read_parquet_files(&checkpoint_files, checkpoint_read_schema, meta_predicate)? + .read_parquet_files(&checkpoint_parts, checkpoint_read_schema, meta_predicate)? .map_ok(|batch| (batch, false)); Ok(commit_stream.chain(checkpoint_stream)) @@ -134,7 +134,7 @@ pub(crate) struct LogSegmentBuilder<'a> { start_version: Option, end_version: Option, commit_files_sorted_ascending: bool, - omit_checkpoint_files: bool, + omit_checkpoint_parts: bool, } impl<'a> LogSegmentBuilder<'a> { pub(crate) fn new(fs_client: &'a dyn FileSystemClient, table_root: &'a Url) -> Self { @@ -145,7 +145,7 @@ impl<'a> LogSegmentBuilder<'a> { start_version: None, end_version: None, commit_files_sorted_ascending: false, - omit_checkpoint_files: false, + omit_checkpoint_parts: false, } } /// Optionally provide checkpoint metadata to start the log segment from (e.g. from reading the `last_checkpoint` file). @@ -177,8 +177,8 @@ impl<'a> LogSegmentBuilder<'a> { /// Optionally specify that the [`LogSegment`] will not have any checkpoint files. It will only /// be made up of commit files. #[allow(unused)] - pub(crate) fn omit_checkpoint_files(mut self) -> Self { - self.omit_checkpoint_files = true; + pub(crate) fn omit_checkpoint_parts(mut self) -> Self { + self.omit_checkpoint_parts = true; self } /// Optionally specify that the commits in the [`LogSegment`] will be in order. By default, the @@ -202,10 +202,10 @@ impl<'a> LogSegmentBuilder<'a> { start_version, end_version, commit_files_sorted_ascending, - omit_checkpoint_files, + omit_checkpoint_parts, } = self; let log_root = table_root.join("_delta_log/").unwrap(); - let (mut sorted_commit_files, mut sorted_checkpoint_files) = + let (mut sorted_commit_files, mut sorted_checkpoint_parts) = match (start_checkpoint, end_version) { (Some(cp), None) => { Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)? @@ -216,8 +216,8 @@ impl<'a> LogSegmentBuilder<'a> { _ => Self::list_log_files_from_version(fs_client, &log_root, None)?, }; - if omit_checkpoint_files { - sorted_checkpoint_files.clear(); + if omit_checkpoint_parts { + sorted_checkpoint_parts.clear(); } // Commit file versions must satisfy the following: @@ -227,7 +227,7 @@ impl<'a> LogSegmentBuilder<'a> { if let Some(start_version) = start_version { sorted_commit_files.retain(|log_path| log_path.version >= start_version); } - if let Some(checkpoint_file) = sorted_checkpoint_files.first() { + if let Some(checkpoint_file) = sorted_checkpoint_parts.first() { sorted_commit_files.retain(|log_path| checkpoint_file.version < log_path.version); } if let Some(end_version) = end_version { @@ -237,7 +237,7 @@ impl<'a> LogSegmentBuilder<'a> { // get the effective version from chosen files let version_eff = sorted_commit_files .last() - .or(sorted_checkpoint_files.first()) + .or(sorted_checkpoint_parts.first()) .ok_or(Error::MissingVersion)? // TODO: A more descriptive error .version; if let Some(end_version) = end_version { @@ -257,7 +257,7 @@ impl<'a> LogSegmentBuilder<'a> { end_version: version_eff, log_root, commit_files: sorted_commit_files, - checkpoint_files: sorted_checkpoint_files, + checkpoint_parts: sorted_checkpoint_parts, }) } pub(crate) fn list_log_files_from_version( @@ -270,7 +270,7 @@ impl<'a> LogSegmentBuilder<'a> { let start_from = log_root.join(&version_prefix)?; let mut max_checkpoint_version = version; - let mut checkpoint_files = vec![]; + let mut checkpoint_parts = vec![]; // We expect 10 commit files per checkpoint, so start with that size. We could adjust this based // on config at some point let mut commit_files = Vec::with_capacity(10); @@ -287,16 +287,16 @@ impl<'a> LogSegmentBuilder<'a> { let path_version = parsed_path.version; match max_checkpoint_version { None => { - checkpoint_files.push(parsed_path); + checkpoint_parts.push(parsed_path); max_checkpoint_version = Some(path_version); } Some(checkpoint_version) => match path_version.cmp(&checkpoint_version) { Ordering::Greater => { max_checkpoint_version = Some(path_version); - checkpoint_files.clear(); - checkpoint_files.push(parsed_path); + checkpoint_parts.clear(); + checkpoint_parts.push(parsed_path); } - Ordering::Equal => checkpoint_files.push(parsed_path), + Ordering::Equal => checkpoint_parts.push(parsed_path), Ordering::Less => {} }, } @@ -311,7 +311,7 @@ impl<'a> LogSegmentBuilder<'a> { commit_files ); - Ok((commit_files, checkpoint_files)) + Ok((commit_files, checkpoint_parts)) } /// List all log files after a given checkpoint. @@ -320,13 +320,13 @@ impl<'a> LogSegmentBuilder<'a> { fs_client: &dyn FileSystemClient, log_root: &Url, ) -> DeltaResult<(Vec, Vec)> { - let (commit_files, checkpoint_files) = Self::list_log_files_from_version( + let (commit_files, checkpoint_parts) = Self::list_log_files_from_version( fs_client, log_root, Some(checkpoint_metadata.version), )?; - let Some(latest_checkpoint) = checkpoint_files.last() else { + let Some(latest_checkpoint) = checkpoint_parts.last() else { // TODO: We could potentially recover here return Err(Error::generic( "Had a _last_checkpoint hint but didn't find any checkpoints", @@ -339,14 +339,14 @@ impl<'a> LogSegmentBuilder<'a> { checkpoint_metadata.version, latest_checkpoint.version ); - } else if checkpoint_files.len() != checkpoint_metadata.parts.unwrap_or(1) { + } else if checkpoint_parts.len() != checkpoint_metadata.parts.unwrap_or(1) { return Err(Error::Generic(format!( "_last_checkpoint indicated that checkpoint should have {} parts, but it has {}", checkpoint_metadata.parts.unwrap_or(1), - checkpoint_files.len() + checkpoint_parts.len() ))); } - Ok((commit_files, checkpoint_files)) + Ok((commit_files, checkpoint_parts)) } } #[cfg(test)] @@ -471,12 +471,12 @@ mod tests { .with_start_checkpoint(checkpoint_metadata) .build() .unwrap(); - let (commit_files, checkpoint_files) = - (log_segment.commit_files, log_segment.checkpoint_files); + let (commit_files, checkpoint_parts) = + (log_segment.commit_files, log_segment.checkpoint_parts); - assert_eq!(checkpoint_files.len(), 1); + assert_eq!(checkpoint_parts.len(), 1); assert_eq!(commit_files.len(), 2); - assert_eq!(checkpoint_files[0].version, 5); + assert_eq!(checkpoint_parts[0].version, 5); assert_eq!(commit_files[0].version, 7); assert_eq!(commit_files[1].version, 6); } From b83c74bffa6b952fd4c24a03de84a2bfe564c4a4 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 10:14:15 -0800 Subject: [PATCH 40/88] Fix test --- kernel/src/log_segment.rs | 1 - kernel/src/snapshot.rs | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 325f120e1..52e2adc58 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -467,7 +467,6 @@ mod tests { let table_root = Url::parse("memory:///").expect("valid url"); let log_segment = LogSegmentBuilder::new(&client, &table_root) - .with_commit_files_sorted_ascending() .with_start_checkpoint(checkpoint_metadata) .build() .unwrap(); diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 09a6a2183..4bd8f3986 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -325,9 +325,9 @@ mod tests { let engine = SyncEngine::new(); let snapshot = Snapshot::try_new(location, &engine, None).unwrap(); - assert_eq!(snapshot.log_segment.checkpoint_files.len(), 1); + assert_eq!(snapshot.log_segment.checkpoint_parts.len(), 1); assert_eq!( - ParsedLogPath::try_from(snapshot.log_segment.checkpoint_files[0].location.clone()) + ParsedLogPath::try_from(snapshot.log_segment.checkpoint_parts[0].location.clone()) .unwrap() .unwrap() .version, From 87b32478ab943c95767f8e81dc7ceeaa5e336edf Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 10:27:00 -0800 Subject: [PATCH 41/88] address more comments --- kernel/src/log_segment.rs | 36 +++++++++++++++++++----------------- 1 file changed, 19 insertions(+), 17 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 52e2adc58..02c7729b4 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -18,7 +18,7 @@ use url::Url; /// A [`LogSegment`] represents a contiguous section of the log and is made up of checkpoint files /// and commit files. It is built with [`LogSegmentBuilder`], and guarantees the following: /// 1. Commit/checkpoint file versions will be less than or equal to `end_version` if specified. -/// 2. Commit/checkpoint file versions will be greater than or equal to `start_version` if specified. +/// 2. Commit file versions will be greater than or equal to `start_version` if specified. /// 3. If checkpoint(s) is/are present in the range, only commits with versions greater than the most /// recent checkpoint version are retained. Checkpoints can be omitted (and this rule skipped) /// whenever the `LogSegment` is created. See [`LogSegmentBuilder::with_omit_checkpoint_parts`]. @@ -133,6 +133,9 @@ pub(crate) struct LogSegmentBuilder<'a> { start_checkpoint: Option, start_version: Option, end_version: Option, + /// When `commit_files_sorted_ascending` is set to `true`, the commit files are sorted in + /// ascending order. Otherwise if it is set to `false`, the commit files are sorted in + /// descending order. This is set to `false` by default. commit_files_sorted_ascending: bool, omit_checkpoint_parts: bool, } @@ -181,8 +184,8 @@ impl<'a> LogSegmentBuilder<'a> { self.omit_checkpoint_parts = true; self } - /// Optionally specify that the commits in the [`LogSegment`] will be in order. By default, the - /// [`LogSegment`] will reverse the order of commit files, with the latest commit coming first. + /// Optionally specify that the commits in the [`LogSegment`] will be sorted by version in ascending + /// order. By default, commits are sorted in descending order of versions. #[allow(unused)] pub(crate) fn with_commit_files_sorted_ascending(mut self) -> Self { self.commit_files_sorted_ascending = true; @@ -205,19 +208,17 @@ impl<'a> LogSegmentBuilder<'a> { omit_checkpoint_parts, } = self; let log_root = table_root.join("_delta_log/").unwrap(); - let (mut sorted_commit_files, mut sorted_checkpoint_parts) = - match (start_checkpoint, end_version) { - (Some(cp), None) => { - Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)? - } - (Some(cp), Some(version)) if cp.version <= version => { - Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)? - } - _ => Self::list_log_files_from_version(fs_client, &log_root, None)?, - }; + let (mut sorted_commit_files, mut checkpoint_parts) = match (start_checkpoint, end_version) + { + (Some(cp), None) => Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)?, + (Some(cp), Some(version)) if cp.version <= version => { + Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)? + } + _ => Self::list_log_files_from_version(fs_client, &log_root, None)?, + }; if omit_checkpoint_parts { - sorted_checkpoint_parts.clear(); + checkpoint_parts.clear(); } // Commit file versions must satisfy the following: @@ -227,7 +228,7 @@ impl<'a> LogSegmentBuilder<'a> { if let Some(start_version) = start_version { sorted_commit_files.retain(|log_path| log_path.version >= start_version); } - if let Some(checkpoint_file) = sorted_checkpoint_parts.first() { + if let Some(checkpoint_file) = checkpoint_parts.first() { sorted_commit_files.retain(|log_path| checkpoint_file.version < log_path.version); } if let Some(end_version) = end_version { @@ -237,7 +238,7 @@ impl<'a> LogSegmentBuilder<'a> { // get the effective version from chosen files let version_eff = sorted_commit_files .last() - .or(sorted_checkpoint_parts.first()) + .or(checkpoint_parts.first()) .ok_or(Error::MissingVersion)? // TODO: A more descriptive error .version; if let Some(end_version) = end_version { @@ -257,9 +258,10 @@ impl<'a> LogSegmentBuilder<'a> { end_version: version_eff, log_root, commit_files: sorted_commit_files, - checkpoint_parts: sorted_checkpoint_parts, + checkpoint_parts, }) } + pub(crate) fn list_log_files_from_version( fs_client: &dyn FileSystemClient, log_root: &Url, From 83436d2439271e2231ed600fa28c838cc4ece512 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 11:56:06 -0800 Subject: [PATCH 42/88] Added tests for log segment builder --- kernel/src/log_segment.rs | 255 ++++++++++++++++++++++++++++++++------ 1 file changed, 216 insertions(+), 39 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 02c7729b4..7d5fde033 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -180,7 +180,7 @@ impl<'a> LogSegmentBuilder<'a> { /// Optionally specify that the [`LogSegment`] will not have any checkpoint files. It will only /// be made up of commit files. #[allow(unused)] - pub(crate) fn omit_checkpoint_parts(mut self) -> Self { + pub(crate) fn with_omit_checkpoint_parts(mut self) -> Self { self.omit_checkpoint_parts = true; self } @@ -234,6 +234,14 @@ impl<'a> LogSegmentBuilder<'a> { if let Some(end_version) = end_version { sorted_commit_files.retain(|log_path| log_path.version <= end_version); } + let ordered_commits = sorted_commit_files + .windows(2) + .all(|cfs| cfs[0].version + 1 == cfs[1].version); + if !ordered_commits { + return Err(Error::generic( + "Expected filesystem client to return ordered contiguous commits", + )); + } // get the effective version from chosen files let version_eff = sorted_commit_files @@ -364,7 +372,7 @@ mod tests { use crate::engine::sync::SyncEngine; use crate::log_segment::LogSegmentBuilder; use crate::snapshot::CheckpointMetadata; - use crate::Table; + use crate::{FileSystemClient, Table}; // NOTE: In addition to testing the meta-predicate for metadata replay, this test also verifies // that the parquet reader properly infers nullcount = rowcount for missing columns. The two @@ -409,54 +417,42 @@ mod tests { assert_eq!(data.len(), 4); } - #[test] - fn test_read_log_with_out_of_date_last_checkpoint() { + fn get_path(index: usize, suffix: &str) -> Path { + let path = format!("_delta_log/{index:020}.{suffix}"); + Path::from(path.as_str()) + } + + // Utility method to build a log using a list of logzpaths and an optional checkpoint hint. The + // CheckpointMetadata is written to `_delta_log/_last_checkpoint`. + fn build_log_with_paths_and_checkpoint( + paths: &[Path], + checkpoint_metadata: Option<&CheckpointMetadata>, + ) -> (Box, Url) { let store = Arc::new(InMemory::new()); - fn get_path(index: usize, suffix: &str) -> Path { - let path = format!("_delta_log/{index:020}.{suffix}"); - Path::from(path.as_str()) - } let data = bytes::Bytes::from("kernel-data"); - let checkpoint_metadata = CheckpointMetadata { - version: 3, - size: 10, - parts: None, - size_in_bytes: None, - num_of_add_files: None, - checkpoint_schema: None, - checksum: None, - }; - // add log files to store tokio::runtime::Runtime::new() .expect("create tokio runtime") .block_on(async { - for path in [ - get_path(0, "json"), - get_path(1, "checkpoint.parquet"), - get_path(2, "json"), - get_path(3, "checkpoint.parquet"), - get_path(4, "json"), - get_path(5, "checkpoint.parquet"), - get_path(6, "json"), - get_path(7, "json"), - ] { + for path in paths { store - .put(&path, data.clone().into()) + .put(path, data.clone().into()) .await .expect("put log file in store"); } - let checkpoint_str = - serde_json::to_string(&checkpoint_metadata).expect("Serialize checkpoint"); - store - .put( - &Path::from("_delta_log/_last_checkpoint"), - checkpoint_str.into(), - ) - .await - .expect("Write _last_checkpoint"); + if let Some(checkpoint_metadata) = checkpoint_metadata { + let checkpoint_str = + serde_json::to_string(checkpoint_metadata).expect("Serialize checkpoint"); + store + .put( + &Path::from("_delta_log/_last_checkpoint"), + checkpoint_str.into(), + ) + .await + .expect("Write _last_checkpoint"); + } }); let client = ObjectStoreFileSystemClient::new( @@ -467,8 +463,78 @@ mod tests { ); let table_root = Url::parse("memory:///").expect("valid url"); + (Box::new(client), table_root) + } + + #[test] + fn test_read_log_with_out_of_date_last_checkpoint() { + let checkpoint_metadata = CheckpointMetadata { + version: 3, + size: 10, + parts: None, + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + get_path(0, "json"), + get_path(1, "checkpoint.parquet"), + get_path(2, "json"), + get_path(3, "checkpoint.parquet"), + get_path(4, "json"), + get_path(5, "checkpoint.parquet"), + get_path(6, "json"), + get_path(7, "json"), + ], + Some(&checkpoint_metadata), + ); + + let log_segment = LogSegmentBuilder::new(client.as_ref(), &table_root) + .with_start_checkpoint(checkpoint_metadata) + .build() + .unwrap(); + let (commit_files, checkpoint_parts) = + (log_segment.commit_files, log_segment.checkpoint_parts); + + assert_eq!(checkpoint_parts.len(), 1); + assert_eq!(commit_files.len(), 2); + assert_eq!(checkpoint_parts[0].version, 5); + assert_eq!(commit_files[0].version, 7); + assert_eq!(commit_files[1].version, 6); + } + #[test] + fn test_read_log_with_correct_last_checkpoint() { + let checkpoint_metadata = CheckpointMetadata { + version: 5, + size: 10, + parts: None, + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + get_path(0, "json"), + get_path(1, "checkpoint.parquet"), + get_path(1, "json"), + get_path(2, "json"), + get_path(3, "checkpoint.parquet"), + get_path(3, "json"), + get_path(4, "json"), + get_path(5, "checkpoint.parquet"), + get_path(5, "json"), + get_path(6, "json"), + get_path(7, "json"), + ], + Some(&checkpoint_metadata), + ); - let log_segment = LogSegmentBuilder::new(&client, &table_root) + let log_segment = LogSegmentBuilder::new(client.as_ref(), &table_root) .with_start_checkpoint(checkpoint_metadata) .build() .unwrap(); @@ -481,4 +547,115 @@ mod tests { assert_eq!(commit_files[0].version, 7); assert_eq!(commit_files[1].version, 6); } + + #[test] + fn test_builder_omit_checkpoints() { + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + get_path(0, "json"), + get_path(1, "json"), + get_path(1, "checkpoint.parquet"), + get_path(2, "json"), + get_path(3, "json"), + get_path(3, "checkpoint.parquet"), + get_path(4, "json"), + get_path(5, "json"), + get_path(5, "checkpoint.parquet"), + get_path(6, "json"), + get_path(7, "json"), + ], + None, + ); + let log_segment = LogSegmentBuilder::new(client.as_ref(), &table_root) + .with_omit_checkpoint_parts() + .build() + .unwrap(); + let (commit_files, checkpoint_parts) = + (log_segment.commit_files, log_segment.checkpoint_parts); + + // Checkpoints should be omitted + assert_eq!(checkpoint_parts.len(), 0); + + // All commit files should still be there + let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); + let expected_versions = (0..=7).rev().collect_vec(); + assert_eq!(versions, expected_versions); + } + #[test] + fn test_commit_ordering() { + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + get_path(0, "json"), + get_path(1, "json"), + get_path(1, "checkpoint.parquet"), + get_path(2, "json"), + get_path(3, "json"), + get_path(3, "checkpoint.parquet"), + get_path(4, "json"), + get_path(5, "json"), + get_path(5, "checkpoint.parquet"), + get_path(6, "json"), + get_path(7, "json"), + ], + None, + ); + let log_segment = LogSegmentBuilder::new(client.as_ref(), &table_root) + .with_end_version(5) + .with_start_version(2) + .with_omit_checkpoint_parts() + .with_commit_files_sorted_ascending() + .build() + .unwrap(); + let (commit_files, checkpoint_parts) = + (log_segment.commit_files, log_segment.checkpoint_parts); + + // Checkpoints should be omitted + assert_eq!(checkpoint_parts.len(), 0); + + // Commits between 2 and 5 (inclusive) should be returned + let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); + let expected_versions = (2..=5).collect_vec(); + assert_eq!(versions, expected_versions); + } + + #[test] + fn test_non_contiguous_log() { + let (client, table_root) = + build_log_with_paths_and_checkpoint(&[get_path(0, "json"), get_path(2, "json")], None); + let log_segment_res = LogSegmentBuilder::new(client.as_ref(), &table_root).build(); + assert!(log_segment_res.is_err()); + } + + #[test] + fn test_start_version_and_checkpoint() { + let checkpoint_metadata = CheckpointMetadata { + version: 3, + size: 10, + parts: None, + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + get_path(0, "json"), + get_path(1, "checkpoint.parquet"), + get_path(2, "json"), + get_path(3, "checkpoint.parquet"), + get_path(4, "json"), + get_path(5, "checkpoint.parquet"), + get_path(6, "json"), + get_path(7, "json"), + ], + Some(&checkpoint_metadata), + ); + + let log_segment_res = LogSegmentBuilder::new(client.as_ref(), &table_root) + .with_start_checkpoint(checkpoint_metadata) + .with_start_version(5) + .build(); + assert!(log_segment_res.is_err()); + } } From a7c246152c7e2c5239e33862571bb7ff17f45077 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 13:15:27 -0800 Subject: [PATCH 43/88] Add tests --- kernel/src/log_segment.rs | 25 ++++++++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 7d5fde033..5e8a6222d 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -582,7 +582,7 @@ mod tests { assert_eq!(versions, expected_versions); } #[test] - fn test_commit_ordering() { + fn test_log_segment_commit_versions() { let (client, table_root) = build_log_with_paths_and_checkpoint( &[ get_path(0, "json"), @@ -599,6 +599,10 @@ mod tests { ], None, ); + + // -------------------------------------------------------------------------------- + // | Specify start version and end version | + // -------------------------------------------------------------------------------- let log_segment = LogSegmentBuilder::new(client.as_ref(), &table_root) .with_end_version(5) .with_start_version(2) @@ -616,6 +620,25 @@ mod tests { let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); let expected_versions = (2..=5).collect_vec(); assert_eq!(versions, expected_versions); + + // -------------------------------------------------------------------------------- + // | Specify no start or end version | + // -------------------------------------------------------------------------------- + let log_segment = LogSegmentBuilder::new(client.as_ref(), &table_root) + .with_omit_checkpoint_parts() + .with_commit_files_sorted_ascending() + .build() + .unwrap(); + let (commit_files, checkpoint_parts) = + (log_segment.commit_files, log_segment.checkpoint_parts); + + // Checkpoints should be omitted + assert_eq!(checkpoint_parts.len(), 0); + + // Commits between 2 and 7 (inclusive) should be returned + let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); + let expected_versions = (0..=7).collect_vec(); + assert_eq!(versions, expected_versions); } #[test] From f7f05f22d1c7020c62c88c8801d2d594ccf82993 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 14:27:38 -0800 Subject: [PATCH 44/88] Address comments --- kernel/src/log_segment.rs | 266 +++++++++++++++++++++----------------- kernel/src/snapshot.rs | 12 +- 2 files changed, 150 insertions(+), 128 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 5e8a6222d..c6d4b0024 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -127,40 +127,50 @@ impl LogSegment { } /// Builder for [`LogSegment`] from from `start_version` to `end_version` inclusive -pub(crate) struct LogSegmentBuilder<'a> { - fs_client: &'a dyn FileSystemClient, - table_root: &'a Url, +pub(crate) struct LogSegmentBuilder { start_checkpoint: Option, start_version: Option, end_version: Option, - /// When `commit_files_sorted_ascending` is set to `true`, the commit files are sorted in + /// When `sort_commit_files_ascending` is set to `true`, the commit files are sorted in /// ascending order. Otherwise if it is set to `false`, the commit files are sorted in /// descending order. This is set to `false` by default. - commit_files_sorted_ascending: bool, + sort_commit_files_ascending: bool, omit_checkpoint_parts: bool, } -impl<'a> LogSegmentBuilder<'a> { - pub(crate) fn new(fs_client: &'a dyn FileSystemClient, table_root: &'a Url) -> Self { +impl LogSegmentBuilder { + pub(crate) fn new() -> Self { LogSegmentBuilder { - fs_client, - table_root, start_checkpoint: None, start_version: None, end_version: None, - commit_files_sorted_ascending: false, + sort_commit_files_ascending: false, omit_checkpoint_parts: false, } } - /// Optionally provide checkpoint metadata to start the log segment from (e.g. from reading the `last_checkpoint` file). + /// Provide checkpoint metadata to start the log segment from (e.g. from reading the `last_checkpoint` file). /// /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] /// with both will result in an error. + #[allow(unused)] pub(crate) fn with_start_checkpoint(mut self, start_checkpoint: CheckpointMetadata) -> Self { self.start_checkpoint = Some(start_checkpoint); self } - /// Optionally set the start version of the [`LogSegment`]. This ensures that all commit files + /// Provide checkpoint metadata to start the log segment. See [`LogSegmentBuilder::with_start_checkpoint`] + /// for details. If `start_checkpoint` is `None`, this is a no-op. + /// + /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] + /// with both will result in an error. + pub(crate) fn with_start_checkpoint_opt( + mut self, + start_checkpoint: Option, + ) -> Self { + self.start_checkpoint = start_checkpoint; + self + } + + /// Provide a `start_version` (inclusive) of the [`LogSegment`] that ensures that all commit files /// are at this version or above it. /// /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] @@ -170,51 +180,69 @@ impl<'a> LogSegmentBuilder<'a> { self.start_version = Some(version); self } - /// Optionally set the end version (inclusive) of the [`LogSegment`]. This ensures that all commit files - /// and checkpoints are at or below the end version. + /// Optionally provide the `start_version` of the [`LogSegment`]. See [`LogSegmentBuilder::with_start_version`] + /// for details. If `start_version` is `None`, this is a no-op. + #[allow(unused)] + pub(crate) fn with_start_version_opt(mut self, version: Option) -> Self { + self.start_version = version; + self + } + + /// Provide an `end_version` (inclusive) of the [`LogSegment`]. This ensures that all commit and + /// checkpoint files are at or below the end version. + #[allow(unused)] pub(crate) fn with_end_version(mut self, version: Version) -> Self { self.end_version = Some(version); self } - /// Optionally specify that the [`LogSegment`] will not have any checkpoint files. It will only - /// be made up of commit files. + /// Optionally set the end version (inclusive) of the [`LogSegment`]. See [`LogSegmentBuilder::with_end_version`] + /// for details. If `end_version` is `None`, this is a no-op. + pub(crate) fn with_end_version_opt(mut self, version: Option) -> Self { + self.end_version = version; + self + } + + /// Specify that the [`LogSegment`] will not have any checkpoint files. It will only be made + /// up of commit files. #[allow(unused)] pub(crate) fn with_omit_checkpoint_parts(mut self) -> Self { self.omit_checkpoint_parts = true; self } - /// Optionally specify that the commits in the [`LogSegment`] will be sorted by version in ascending - /// order. By default, commits are sorted in descending order of versions. + /// Specify that the commits in the [`LogSegment`] will be sorted by version in ascending + /// order. By default, commits are sorted by version in descending order. #[allow(unused)] - pub(crate) fn with_commit_files_sorted_ascending(mut self) -> Self { - self.commit_files_sorted_ascending = true; + pub(crate) fn with_sort_commit_files_ascending(mut self) -> Self { + self.sort_commit_files_ascending = true; self } /// Build the [`LogSegment`] /// /// This fetches checkpoint and commit files using the `fs_client`. - pub(crate) fn build(self) -> DeltaResult { + pub(crate) fn build( + self, + fs_client: &dyn FileSystemClient, + table_root: &Url, + ) -> DeltaResult { if self.start_version.is_some() && self.start_checkpoint.is_some() { return Err(Error::generic("Failed to build LogSegment: Cannot specify both start_version and start_checkpoint")); } let Self { - fs_client, - table_root, start_checkpoint, start_version, end_version, - commit_files_sorted_ascending, + sort_commit_files_ascending, omit_checkpoint_parts, } = self; let log_root = table_root.join("_delta_log/").unwrap(); let (mut sorted_commit_files, mut checkpoint_parts) = match (start_checkpoint, end_version) { - (Some(cp), None) => Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)?, + (Some(cp), None) => list_log_files_with_checkpoint(&cp, fs_client, &log_root)?, (Some(cp), Some(version)) if cp.version <= version => { - Self::list_log_files_with_checkpoint(&cp, fs_client, &log_root)? + list_log_files_with_checkpoint(&cp, fs_client, &log_root)? } - _ => Self::list_log_files_from_version(fs_client, &log_root, None)?, + _ => list_log_files_from_version(fs_client, &log_root, None)?, }; if omit_checkpoint_parts { @@ -256,9 +284,9 @@ impl<'a> LogSegmentBuilder<'a> { ); } - // We assume commit files are sorted in ascending order. If `commit_files_sorted_ascending` + // We assume commit files are sorted in ascending order. If `sort_commit_files_ascending` // is false, reverse to make it descending. - if !commit_files_sorted_ascending { + if !sort_commit_files_ascending { sorted_commit_files.reverse(); } @@ -269,96 +297,94 @@ impl<'a> LogSegmentBuilder<'a> { checkpoint_parts, }) } +} - pub(crate) fn list_log_files_from_version( - fs_client: &dyn FileSystemClient, - log_root: &Url, - version: Option, - ) -> DeltaResult<(Vec, Vec)> { - let begin_version = version.unwrap_or(0); - let version_prefix = format!("{:020}", begin_version); - let start_from = log_root.join(&version_prefix)?; - - let mut max_checkpoint_version = version; - let mut checkpoint_parts = vec![]; - // We expect 10 commit files per checkpoint, so start with that size. We could adjust this based - // on config at some point - let mut commit_files = Vec::with_capacity(10); - - for meta_res in fs_client.list_from(&start_from)? { - let meta = meta_res?; - // TODO this filters out .crc files etc which start with "." - how do we want to use these kind of files? - let Some(parsed_path) = ParsedLogPath::try_from(meta)? else { - continue; - }; - if parsed_path.is_commit() { - commit_files.push(parsed_path); - } else if parsed_path.is_checkpoint() { - let path_version = parsed_path.version; - match max_checkpoint_version { - None => { - checkpoint_parts.push(parsed_path); +fn list_log_files_from_version( + fs_client: &dyn FileSystemClient, + log_root: &Url, + version: Option, +) -> DeltaResult<(Vec, Vec)> { + let begin_version = version.unwrap_or(0); + let version_prefix = format!("{:020}", begin_version); + let start_from = log_root.join(&version_prefix)?; + + let mut max_checkpoint_version = version; + let mut checkpoint_parts = vec![]; + // We expect 10 commit files per checkpoint, so start with that size. We could adjust this based + // on config at some point + let mut commit_files = Vec::with_capacity(10); + + for meta_res in fs_client.list_from(&start_from)? { + let meta = meta_res?; + // TODO this filters out .crc files etc which start with "." - how do we want to use these kind of files? + let Some(parsed_path) = ParsedLogPath::try_from(meta)? else { + continue; + }; + if parsed_path.is_commit() { + commit_files.push(parsed_path); + } else if parsed_path.is_checkpoint() { + let path_version = parsed_path.version; + match max_checkpoint_version { + None => { + checkpoint_parts.push(parsed_path); + max_checkpoint_version = Some(path_version); + } + Some(checkpoint_version) => match path_version.cmp(&checkpoint_version) { + Ordering::Greater => { max_checkpoint_version = Some(path_version); + checkpoint_parts.clear(); + checkpoint_parts.push(parsed_path); } - Some(checkpoint_version) => match path_version.cmp(&checkpoint_version) { - Ordering::Greater => { - max_checkpoint_version = Some(path_version); - checkpoint_parts.clear(); - checkpoint_parts.push(parsed_path); - } - Ordering::Equal => checkpoint_parts.push(parsed_path), - Ordering::Less => {} - }, - } + Ordering::Equal => checkpoint_parts.push(parsed_path), + Ordering::Less => {} + }, } } - - debug_assert!( - commit_files - .windows(2) - .all(|cfs| cfs[0].version <= cfs[1].version), - "fs_client.list_from() didn't return a sorted listing! {:?}", - commit_files - ); - - Ok((commit_files, checkpoint_parts)) } - /// List all log files after a given checkpoint. - pub(crate) fn list_log_files_with_checkpoint( - checkpoint_metadata: &CheckpointMetadata, - fs_client: &dyn FileSystemClient, - log_root: &Url, - ) -> DeltaResult<(Vec, Vec)> { - let (commit_files, checkpoint_parts) = Self::list_log_files_from_version( - fs_client, - log_root, - Some(checkpoint_metadata.version), - )?; + debug_assert!( + commit_files + .windows(2) + .all(|cfs| cfs[0].version <= cfs[1].version), + "fs_client.list_from() didn't return a sorted listing! {:?}", + commit_files + ); - let Some(latest_checkpoint) = checkpoint_parts.last() else { - // TODO: We could potentially recover here - return Err(Error::generic( - "Had a _last_checkpoint hint but didn't find any checkpoints", - )); - }; + Ok((commit_files, checkpoint_parts)) +} - if latest_checkpoint.version != checkpoint_metadata.version { - warn!( +/// List all log files after a given checkpoint. +fn list_log_files_with_checkpoint( + checkpoint_metadata: &CheckpointMetadata, + fs_client: &dyn FileSystemClient, + log_root: &Url, +) -> DeltaResult<(Vec, Vec)> { + let (commit_files, checkpoint_parts) = + list_log_files_from_version(fs_client, log_root, Some(checkpoint_metadata.version))?; + + let Some(latest_checkpoint) = checkpoint_parts.last() else { + // TODO: We could potentially recover here + return Err(Error::generic( + "Had a _last_checkpoint hint but didn't find any checkpoints", + )); + }; + + if latest_checkpoint.version != checkpoint_metadata.version { + warn!( "_last_checkpoint hint is out of date. _last_checkpoint version: {}. Using actual most recent: {}", checkpoint_metadata.version, latest_checkpoint.version ); - } else if checkpoint_parts.len() != checkpoint_metadata.parts.unwrap_or(1) { - return Err(Error::Generic(format!( - "_last_checkpoint indicated that checkpoint should have {} parts, but it has {}", - checkpoint_metadata.parts.unwrap_or(1), - checkpoint_parts.len() - ))); - } - Ok((commit_files, checkpoint_parts)) + } else if checkpoint_parts.len() != checkpoint_metadata.parts.unwrap_or(1) { + return Err(Error::Generic(format!( + "_last_checkpoint indicated that checkpoint should have {} parts, but it has {}", + checkpoint_metadata.parts.unwrap_or(1), + checkpoint_parts.len() + ))); } + Ok((commit_files, checkpoint_parts)) } + #[cfg(test)] mod tests { use std::{path::PathBuf, sync::Arc}; @@ -492,9 +518,9 @@ mod tests { Some(&checkpoint_metadata), ); - let log_segment = LogSegmentBuilder::new(client.as_ref(), &table_root) + let log_segment = LogSegmentBuilder::new() .with_start_checkpoint(checkpoint_metadata) - .build() + .build(client.as_ref(), &table_root) .unwrap(); let (commit_files, checkpoint_parts) = (log_segment.commit_files, log_segment.checkpoint_parts); @@ -534,9 +560,9 @@ mod tests { Some(&checkpoint_metadata), ); - let log_segment = LogSegmentBuilder::new(client.as_ref(), &table_root) + let log_segment = LogSegmentBuilder::new() .with_start_checkpoint(checkpoint_metadata) - .build() + .build(client.as_ref(), &table_root) .unwrap(); let (commit_files, checkpoint_parts) = (log_segment.commit_files, log_segment.checkpoint_parts); @@ -566,9 +592,9 @@ mod tests { ], None, ); - let log_segment = LogSegmentBuilder::new(client.as_ref(), &table_root) + let log_segment = LogSegmentBuilder::new() .with_omit_checkpoint_parts() - .build() + .build(client.as_ref(), &table_root) .unwrap(); let (commit_files, checkpoint_parts) = (log_segment.commit_files, log_segment.checkpoint_parts); @@ -603,12 +629,12 @@ mod tests { // -------------------------------------------------------------------------------- // | Specify start version and end version | // -------------------------------------------------------------------------------- - let log_segment = LogSegmentBuilder::new(client.as_ref(), &table_root) + let log_segment = LogSegmentBuilder::new() .with_end_version(5) .with_start_version(2) .with_omit_checkpoint_parts() - .with_commit_files_sorted_ascending() - .build() + .with_sort_commit_files_ascending() + .build(client.as_ref(), &table_root) .unwrap(); let (commit_files, checkpoint_parts) = (log_segment.commit_files, log_segment.checkpoint_parts); @@ -624,10 +650,10 @@ mod tests { // -------------------------------------------------------------------------------- // | Specify no start or end version | // -------------------------------------------------------------------------------- - let log_segment = LogSegmentBuilder::new(client.as_ref(), &table_root) + let log_segment = LogSegmentBuilder::new() .with_omit_checkpoint_parts() - .with_commit_files_sorted_ascending() - .build() + .with_sort_commit_files_ascending() + .build(client.as_ref(), &table_root) .unwrap(); let (commit_files, checkpoint_parts) = (log_segment.commit_files, log_segment.checkpoint_parts); @@ -645,12 +671,12 @@ mod tests { fn test_non_contiguous_log() { let (client, table_root) = build_log_with_paths_and_checkpoint(&[get_path(0, "json"), get_path(2, "json")], None); - let log_segment_res = LogSegmentBuilder::new(client.as_ref(), &table_root).build(); + let log_segment_res = LogSegmentBuilder::new().build(client.as_ref(), &table_root); assert!(log_segment_res.is_err()); } #[test] - fn test_start_version_and_checkpoint() { + fn test_build_with_start_version_and_checkpoint_fails() { let checkpoint_metadata = CheckpointMetadata { version: 3, size: 10, @@ -675,10 +701,10 @@ mod tests { Some(&checkpoint_metadata), ); - let log_segment_res = LogSegmentBuilder::new(client.as_ref(), &table_root) + let log_segment_res = LogSegmentBuilder::new() .with_start_checkpoint(checkpoint_metadata) .with_start_version(5) - .build(); + .build(client.as_ref(), &table_root); assert!(log_segment_res.is_err()); } } diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 4bd8f3986..d6375d51f 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -62,14 +62,10 @@ impl Snapshot { let fs_client = engine.get_file_system_client(); let log_url = table_root.join("_delta_log/").unwrap(); - let mut builder = LogSegmentBuilder::new(fs_client.as_ref(), &table_root); - if let Some(version) = version { - builder = builder.with_end_version(version); - } - if let Some(checkpoint) = read_last_checkpoint(fs_client.as_ref(), &log_url)? { - builder = builder.with_start_checkpoint(checkpoint); - } - let log_segment = builder.build()?; + let log_segment = LogSegmentBuilder::new() + .with_end_version_opt(version) + .with_start_checkpoint_opt(read_last_checkpoint(fs_client.as_ref(), &log_url)?) + .build(fs_client.as_ref(), &table_root)?; Self::try_new_from_log_segment(table_root, log_segment, engine) } From 11032a5785a1ee1e85c62fddda8739aa2d4c2812 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 14:32:35 -0800 Subject: [PATCH 45/88] small nits --- kernel/src/log_segment.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index c6d4b0024..f9d247917 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -669,6 +669,7 @@ mod tests { #[test] fn test_non_contiguous_log() { + // Commit with version 1 is missing let (client, table_root) = build_log_with_paths_and_checkpoint(&[get_path(0, "json"), get_path(2, "json")], None); let log_segment_res = LogSegmentBuilder::new().build(client.as_ref(), &table_root); From 84027cbff14d44fba36636762edf332509ee69f3 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 14:36:36 -0800 Subject: [PATCH 46/88] change checkpoint parts to checkpoint files --- kernel/src/log_segment.rs | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index f9d247917..ba8328903 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -21,9 +21,9 @@ use url::Url; /// 2. Commit file versions will be greater than or equal to `start_version` if specified. /// 3. If checkpoint(s) is/are present in the range, only commits with versions greater than the most /// recent checkpoint version are retained. Checkpoints can be omitted (and this rule skipped) -/// whenever the `LogSegment` is created. See [`LogSegmentBuilder::with_omit_checkpoint_parts`]. +/// whenever the `LogSegment` is created. See [`LogSegmentBuilder::with_omit_checkpoint_files`]. /// -/// [`LogSegment`] is used in both [`Snapshot`] and in [`TableChanges`] to hold commit files and +/// [`LogSegment`] is used in both [`Snapshot`] and in `TableChanges` to hold commit files and /// checkpoint files. /// - For a Snapshot at version `n`: Its LogSegment is made up of zero or one checkpoint, and all /// commits between the checkpoint and the end version `n`. @@ -135,7 +135,7 @@ pub(crate) struct LogSegmentBuilder { /// ascending order. Otherwise if it is set to `false`, the commit files are sorted in /// descending order. This is set to `false` by default. sort_commit_files_ascending: bool, - omit_checkpoint_parts: bool, + omit_checkpoint_files: bool, } impl LogSegmentBuilder { pub(crate) fn new() -> Self { @@ -144,7 +144,7 @@ impl LogSegmentBuilder { start_version: None, end_version: None, sort_commit_files_ascending: false, - omit_checkpoint_parts: false, + omit_checkpoint_files: false, } } /// Provide checkpoint metadata to start the log segment from (e.g. from reading the `last_checkpoint` file). @@ -206,8 +206,8 @@ impl LogSegmentBuilder { /// Specify that the [`LogSegment`] will not have any checkpoint files. It will only be made /// up of commit files. #[allow(unused)] - pub(crate) fn with_omit_checkpoint_parts(mut self) -> Self { - self.omit_checkpoint_parts = true; + pub(crate) fn with_omit_checkpoint_files(mut self) -> Self { + self.omit_checkpoint_files = true; self } /// Specify that the commits in the [`LogSegment`] will be sorted by version in ascending @@ -233,7 +233,7 @@ impl LogSegmentBuilder { start_version, end_version, sort_commit_files_ascending, - omit_checkpoint_parts, + omit_checkpoint_files, } = self; let log_root = table_root.join("_delta_log/").unwrap(); let (mut sorted_commit_files, mut checkpoint_parts) = match (start_checkpoint, end_version) @@ -245,7 +245,7 @@ impl LogSegmentBuilder { _ => list_log_files_from_version(fs_client, &log_root, None)?, }; - if omit_checkpoint_parts { + if omit_checkpoint_files { checkpoint_parts.clear(); } @@ -593,7 +593,7 @@ mod tests { None, ); let log_segment = LogSegmentBuilder::new() - .with_omit_checkpoint_parts() + .with_omit_checkpoint_files() .build(client.as_ref(), &table_root) .unwrap(); let (commit_files, checkpoint_parts) = @@ -632,7 +632,7 @@ mod tests { let log_segment = LogSegmentBuilder::new() .with_end_version(5) .with_start_version(2) - .with_omit_checkpoint_parts() + .with_omit_checkpoint_files() .with_sort_commit_files_ascending() .build(client.as_ref(), &table_root) .unwrap(); @@ -651,7 +651,7 @@ mod tests { // | Specify no start or end version | // -------------------------------------------------------------------------------- let log_segment = LogSegmentBuilder::new() - .with_omit_checkpoint_parts() + .with_omit_checkpoint_files() .with_sort_commit_files_ascending() .build(client.as_ref(), &table_root) .unwrap(); From 4b7ff1782322a66d72d0c9653fa2d768420632d0 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 14:37:58 -0800 Subject: [PATCH 47/88] Make visibility the same between builder and logsegment --- kernel/src/log_segment.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index ba8328903..e427455d6 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -127,6 +127,7 @@ impl LogSegment { } /// Builder for [`LogSegment`] from from `start_version` to `end_version` inclusive +#[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) struct LogSegmentBuilder { start_checkpoint: Option, start_version: Option, From 4c9c96bef1024ac39746a5a193a0bfa2038a4471 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 15:09:35 -0800 Subject: [PATCH 48/88] fix comment --- kernel/src/log_segment.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index e427455d6..46e42d16c 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -148,7 +148,7 @@ impl LogSegmentBuilder { omit_checkpoint_files: false, } } - /// Provide checkpoint metadata to start the log segment from (e.g. from reading the `last_checkpoint` file). + /// Provide a checkpoint metadata to start the log segment from (e.g. from reading the `last_checkpoint` file). /// /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] /// with both will result in an error. @@ -158,7 +158,7 @@ impl LogSegmentBuilder { self } - /// Provide checkpoint metadata to start the log segment. See [`LogSegmentBuilder::with_start_checkpoint`] + /// Optionally provide a checkpoint metadata to start the log segment. See [`LogSegmentBuilder::with_start_checkpoint`] /// for details. If `start_checkpoint` is `None`, this is a no-op. /// /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] @@ -181,7 +181,7 @@ impl LogSegmentBuilder { self.start_version = Some(version); self } - /// Optionally provide the `start_version` of the [`LogSegment`]. See [`LogSegmentBuilder::with_start_version`] + /// Optionally provide a `start_version` of the [`LogSegment`]. See [`LogSegmentBuilder::with_start_version`] /// for details. If `start_version` is `None`, this is a no-op. #[allow(unused)] pub(crate) fn with_start_version_opt(mut self, version: Option) -> Self { @@ -197,7 +197,7 @@ impl LogSegmentBuilder { self } - /// Optionally set the end version (inclusive) of the [`LogSegment`]. See [`LogSegmentBuilder::with_end_version`] + /// Optionally provide an `end_version` (inclusive) of the [`LogSegment`]. See [`LogSegmentBuilder::with_end_version`] /// for details. If `end_version` is `None`, this is a no-op. pub(crate) fn with_end_version_opt(mut self, version: Option) -> Self { self.end_version = version; From a53f38a63731c14316e441e3c8de479119fded0c Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 15:20:11 -0800 Subject: [PATCH 49/88] Update comments --- kernel/src/log_segment.rs | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 46e42d16c..41fed9423 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -157,7 +157,6 @@ impl LogSegmentBuilder { self.start_checkpoint = Some(start_checkpoint); self } - /// Optionally provide a checkpoint metadata to start the log segment. See [`LogSegmentBuilder::with_start_checkpoint`] /// for details. If `start_checkpoint` is `None`, this is a no-op. /// @@ -170,7 +169,6 @@ impl LogSegmentBuilder { self.start_checkpoint = start_checkpoint; self } - /// Provide a `start_version` (inclusive) of the [`LogSegment`] that ensures that all commit files /// are at this version or above it. /// @@ -188,7 +186,6 @@ impl LogSegmentBuilder { self.start_version = version; self } - /// Provide an `end_version` (inclusive) of the [`LogSegment`]. This ensures that all commit and /// checkpoint files are at or below the end version. #[allow(unused)] @@ -196,14 +193,12 @@ impl LogSegmentBuilder { self.end_version = Some(version); self } - /// Optionally provide an `end_version` (inclusive) of the [`LogSegment`]. See [`LogSegmentBuilder::with_end_version`] /// for details. If `end_version` is `None`, this is a no-op. pub(crate) fn with_end_version_opt(mut self, version: Option) -> Self { self.end_version = version; self } - /// Specify that the [`LogSegment`] will not have any checkpoint files. It will only be made /// up of commit files. #[allow(unused)] @@ -243,7 +238,7 @@ impl LogSegmentBuilder { (Some(cp), Some(version)) if cp.version <= version => { list_log_files_with_checkpoint(&cp, fs_client, &log_root)? } - _ => list_log_files_from_version(fs_client, &log_root, None)?, + _ => list_log_files_with_version(fs_client, &log_root, None)?, }; if omit_checkpoint_files { @@ -300,7 +295,12 @@ impl LogSegmentBuilder { } } -fn list_log_files_from_version( +/// List all commit and checkpoint files with versions above the provided `version`. If successful, this returns +/// a tuple `(sorted_commit_files_paths, checkpoint_parts): (Vec, Vec)`. +/// The commit files are guaranteed to be sorted in ascending order by version. The elements of +/// `checkpoint_parts` are all the parts of the same checkpoint. Checkpoint parts share the same +/// version. +fn list_log_files_with_version( fs_client: &dyn FileSystemClient, log_root: &Url, version: Option, @@ -354,14 +354,15 @@ fn list_log_files_from_version( Ok((commit_files, checkpoint_parts)) } -/// List all log files after a given checkpoint. +/// List all commit and checkpoint files after the provided checkpoint. +/// See [`list_log_files_with_version`] for details on the return type. fn list_log_files_with_checkpoint( checkpoint_metadata: &CheckpointMetadata, fs_client: &dyn FileSystemClient, log_root: &Url, ) -> DeltaResult<(Vec, Vec)> { let (commit_files, checkpoint_parts) = - list_log_files_from_version(fs_client, log_root, Some(checkpoint_metadata.version))?; + list_log_files_with_version(fs_client, log_root, Some(checkpoint_metadata.version))?; let Some(latest_checkpoint) = checkpoint_parts.last() else { // TODO: We could potentially recover here From b0b6514de55091a35e40a7e9a2ef2102c48c654f Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 15:21:12 -0800 Subject: [PATCH 50/88] error message --- kernel/src/log_segment.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 41fed9423..124799015 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -262,9 +262,7 @@ impl LogSegmentBuilder { .windows(2) .all(|cfs| cfs[0].version + 1 == cfs[1].version); if !ordered_commits { - return Err(Error::generic( - "Expected filesystem client to return ordered contiguous commits", - )); + return Err(Error::generic("Expected ordered contiguous commit files")); } // get the effective version from chosen files From 2630a9e0a66f906aad9c941c3347db7c9087342b Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 15:32:24 -0800 Subject: [PATCH 51/88] Use delta_path_for_version instead of get_path --- kernel/src/log_segment.rs | 113 +++++++++++++++++++------------------- 1 file changed, 57 insertions(+), 56 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 124799015..6afaff4d8 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -399,6 +399,7 @@ mod tests { use crate::log_segment::LogSegmentBuilder; use crate::snapshot::CheckpointMetadata; use crate::{FileSystemClient, Table}; + use test_utils::delta_path_for_version; // NOTE: In addition to testing the meta-predicate for metadata replay, this test also verifies // that the parquet reader properly infers nullcount = rowcount for missing columns. The two @@ -443,11 +444,6 @@ mod tests { assert_eq!(data.len(), 4); } - fn get_path(index: usize, suffix: &str) -> Path { - let path = format!("_delta_log/{index:020}.{suffix}"); - Path::from(path.as_str()) - } - // Utility method to build a log using a list of logzpaths and an optional checkpoint hint. The // CheckpointMetadata is written to `_delta_log/_last_checkpoint`. fn build_log_with_paths_and_checkpoint( @@ -506,14 +502,14 @@ mod tests { let (client, table_root) = build_log_with_paths_and_checkpoint( &[ - get_path(0, "json"), - get_path(1, "checkpoint.parquet"), - get_path(2, "json"), - get_path(3, "checkpoint.parquet"), - get_path(4, "json"), - get_path(5, "checkpoint.parquet"), - get_path(6, "json"), - get_path(7, "json"), + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(4, "json"), + delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), ], Some(&checkpoint_metadata), ); @@ -545,17 +541,17 @@ mod tests { let (client, table_root) = build_log_with_paths_and_checkpoint( &[ - get_path(0, "json"), - get_path(1, "checkpoint.parquet"), - get_path(1, "json"), - get_path(2, "json"), - get_path(3, "checkpoint.parquet"), - get_path(3, "json"), - get_path(4, "json"), - get_path(5, "checkpoint.parquet"), - get_path(5, "json"), - get_path(6, "json"), - get_path(7, "json"), + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(1, "json"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(3, "json"), + delta_path_for_version(4, "json"), + delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_version(5, "json"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), ], Some(&checkpoint_metadata), ); @@ -578,17 +574,17 @@ mod tests { fn test_builder_omit_checkpoints() { let (client, table_root) = build_log_with_paths_and_checkpoint( &[ - get_path(0, "json"), - get_path(1, "json"), - get_path(1, "checkpoint.parquet"), - get_path(2, "json"), - get_path(3, "json"), - get_path(3, "checkpoint.parquet"), - get_path(4, "json"), - get_path(5, "json"), - get_path(5, "checkpoint.parquet"), - get_path(6, "json"), - get_path(7, "json"), + delta_path_for_version(0, "json"), + delta_path_for_version(1, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(4, "json"), + delta_path_for_version(5, "json"), + delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), ], None, ); @@ -611,17 +607,17 @@ mod tests { fn test_log_segment_commit_versions() { let (client, table_root) = build_log_with_paths_and_checkpoint( &[ - get_path(0, "json"), - get_path(1, "json"), - get_path(1, "checkpoint.parquet"), - get_path(2, "json"), - get_path(3, "json"), - get_path(3, "checkpoint.parquet"), - get_path(4, "json"), - get_path(5, "json"), - get_path(5, "checkpoint.parquet"), - get_path(6, "json"), - get_path(7, "json"), + delta_path_for_version(0, "json"), + delta_path_for_version(1, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(4, "json"), + delta_path_for_version(5, "json"), + delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), ], None, ); @@ -670,8 +666,13 @@ mod tests { #[test] fn test_non_contiguous_log() { // Commit with version 1 is missing - let (client, table_root) = - build_log_with_paths_and_checkpoint(&[get_path(0, "json"), get_path(2, "json")], None); + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(2, "json"), + ], + None, + ); let log_segment_res = LogSegmentBuilder::new().build(client.as_ref(), &table_root); assert!(log_segment_res.is_err()); } @@ -690,14 +691,14 @@ mod tests { let (client, table_root) = build_log_with_paths_and_checkpoint( &[ - get_path(0, "json"), - get_path(1, "checkpoint.parquet"), - get_path(2, "json"), - get_path(3, "checkpoint.parquet"), - get_path(4, "json"), - get_path(5, "checkpoint.parquet"), - get_path(6, "json"), - get_path(7, "json"), + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(4, "json"), + delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), ], Some(&checkpoint_metadata), ); From db5dddbebf6276f0f638813ebdec708825a0d495 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 15:40:14 -0800 Subject: [PATCH 52/88] Add reference import --- kernel/src/log_segment.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 6afaff4d8..047aadae5 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -29,6 +29,8 @@ use url::Url; /// commits between the checkpoint and the end version `n`. /// - For a TableChanges between versions `a` and `b`: Its LogSegment is made up of zero /// checkpoints and all commits between versions `a` and `b` +/// +/// [`Snapshot`]: crate::snapshot::Snapshot #[derive(Debug)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) struct LogSegment { From a1531ee2b6e05b2de94a3df30da5706f7550a281 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 16:13:06 -0800 Subject: [PATCH 53/88] Apply suggestions from code review Co-authored-by: Zach Schuermann --- kernel/src/log_segment.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 047aadae5..72274066a 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -446,7 +446,7 @@ mod tests { assert_eq!(data.len(), 4); } - // Utility method to build a log using a list of logzpaths and an optional checkpoint hint. The + // Utility method to build a log using a list of log paths and an optional checkpoint hint. The // CheckpointMetadata is written to `_delta_log/_last_checkpoint`. fn build_log_with_paths_and_checkpoint( paths: &[Path], From 33c4039f13cd50efd93f47d62b2b62aff965e007 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 16:27:13 -0800 Subject: [PATCH 54/88] Update kernel/src/log_segment.rs Co-authored-by: Zach Schuermann --- kernel/src/log_segment.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 72274066a..39e007afb 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -234,12 +234,15 @@ impl LogSegmentBuilder { omit_checkpoint_files, } = self; let log_root = table_root.join("_delta_log/").unwrap(); - let (mut sorted_commit_files, mut checkpoint_parts) = match (start_checkpoint, end_version) + let (mut sorted_commit_files, mut checkpoint_parts) = match (start_checkpoint, start_version, end_version) { - (Some(cp), None) => list_log_files_with_checkpoint(&cp, fs_client, &log_root)?, - (Some(cp), Some(version)) if cp.version <= version => { + (Some(cp), None, None) => list_log_files_with_checkpoint(&cp, fs_client, &log_root)?, + (Some(cp), None, Some(end_version)) if cp.version <= end_version => { list_log_files_with_checkpoint(&cp, fs_client, &log_root)? } + (None, Some(start_version), _) => { + list_log_files_with_version(fs_client, &log_root, Some(start_version))?, + } _ => list_log_files_with_version(fs_client, &log_root, None)?, }; From 1f077b011a824843323024cf844e7e6b9c2d6dd8 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 16:30:36 -0800 Subject: [PATCH 55/88] More nits --- kernel/src/log_segment.rs | 56 ++++++++++++++++++++++++++++++--------- 1 file changed, 44 insertions(+), 12 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 39e007afb..7920bbc66 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -129,7 +129,6 @@ impl LogSegment { } /// Builder for [`LogSegment`] from from `start_version` to `end_version` inclusive -#[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) struct LogSegmentBuilder { start_checkpoint: Option, start_version: Option, @@ -226,6 +225,11 @@ impl LogSegmentBuilder { if self.start_version.is_some() && self.start_checkpoint.is_some() { return Err(Error::generic("Failed to build LogSegment: Cannot specify both start_version and start_checkpoint")); } + if let (Some(start_version), Some(end_version)) = (self.start_version, self.end_version) { + if start_version > end_version { + return Err(Error::generic("Failed to build LogSegment: `start_version` cannot be greater than end_version")); + } + } let Self { start_checkpoint, start_version, @@ -234,17 +238,19 @@ impl LogSegmentBuilder { omit_checkpoint_files, } = self; let log_root = table_root.join("_delta_log/").unwrap(); - let (mut sorted_commit_files, mut checkpoint_parts) = match (start_checkpoint, start_version, end_version) - { - (Some(cp), None, None) => list_log_files_with_checkpoint(&cp, fs_client, &log_root)?, - (Some(cp), None, Some(end_version)) if cp.version <= end_version => { - list_log_files_with_checkpoint(&cp, fs_client, &log_root)? - } - (None, Some(start_version), _) => { - list_log_files_with_version(fs_client, &log_root, Some(start_version))?, - } - _ => list_log_files_with_version(fs_client, &log_root, None)?, - }; + let (mut sorted_commit_files, mut checkpoint_parts) = + match (start_checkpoint, start_version, end_version) { + (Some(cp), None, None) => { + list_log_files_with_checkpoint(&cp, fs_client, &log_root)? + } + (Some(cp), None, Some(end_version)) if cp.version <= end_version => { + list_log_files_with_checkpoint(&cp, fs_client, &log_root)? + } + (None, Some(start_version), _) => { + list_log_files_with_version(fs_client, &log_root, Some(start_version))? + } + _ => list_log_files_with_version(fs_client, &log_root, None)?, + }; if omit_checkpoint_files { checkpoint_parts.clear(); @@ -263,6 +269,9 @@ impl LogSegmentBuilder { if let Some(end_version) = end_version { sorted_commit_files.retain(|log_path| log_path.version <= end_version); } + + // After (possibly) omitting checkpoint files and filtering commits, we should have commits that are + // contiguous. In other words, there must be no gap between commit versions. let ordered_commits = sorted_commit_files .windows(2) .all(|cfs| cfs[0].version + 1 == cfs[1].version); @@ -682,6 +691,29 @@ mod tests { assert!(log_segment_res.is_err()); } + #[test] + fn test_larger_start_version_is_fail() { + // Commit with version 1 is missing + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "json"), + ], + None, + ); + let log_segment_res = LogSegmentBuilder::new() + .with_start_version(1) + .with_end_version(0) + .build(client.as_ref(), &table_root); + assert!(log_segment_res.is_err()); + + let log_segment_res = LogSegmentBuilder::new() + .with_start_version(0) + .with_end_version(0) + .build(client.as_ref(), &table_root); + assert!(log_segment_res.is_ok()); + } + #[test] fn test_build_with_start_version_and_checkpoint_fails() { let checkpoint_metadata = CheckpointMetadata { From 9d0a24fa3c8548eb6edae731559e58ba3ca65f96 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 16:33:25 -0800 Subject: [PATCH 56/88] Return dev visibility --- kernel/src/log_segment.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 7920bbc66..9d23ff5cb 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -129,6 +129,7 @@ impl LogSegment { } /// Builder for [`LogSegment`] from from `start_version` to `end_version` inclusive +#[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) struct LogSegmentBuilder { start_checkpoint: Option, start_version: Option, From f8457d9c83107efd256c127c84442ef8bd56a6c2 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 16:45:11 -0800 Subject: [PATCH 57/88] make builder methods match builder visibility --- kernel/src/log_segment.rs | 27 +++++++++++++++++---------- 1 file changed, 17 insertions(+), 10 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 9d23ff5cb..204ff5346 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -140,8 +140,9 @@ pub(crate) struct LogSegmentBuilder { sort_commit_files_ascending: bool, omit_checkpoint_files: bool, } + impl LogSegmentBuilder { - pub(crate) fn new() -> Self { + pub fn new() -> Self { LogSegmentBuilder { start_checkpoint: None, start_version: None, @@ -155,7 +156,7 @@ impl LogSegmentBuilder { /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] /// with both will result in an error. #[allow(unused)] - pub(crate) fn with_start_checkpoint(mut self, start_checkpoint: CheckpointMetadata) -> Self { + pub fn with_start_checkpoint(mut self, start_checkpoint: CheckpointMetadata) -> Self { self.start_checkpoint = Some(start_checkpoint); self } @@ -164,7 +165,7 @@ impl LogSegmentBuilder { /// /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] /// with both will result in an error. - pub(crate) fn with_start_checkpoint_opt( + pub fn with_start_checkpoint_opt( mut self, start_checkpoint: Option, ) -> Self { @@ -177,48 +178,48 @@ impl LogSegmentBuilder { /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] /// with both will result in an error. #[allow(unused)] - pub(crate) fn with_start_version(mut self, version: Version) -> Self { + pub fn with_start_version(mut self, version: Version) -> Self { self.start_version = Some(version); self } /// Optionally provide a `start_version` of the [`LogSegment`]. See [`LogSegmentBuilder::with_start_version`] /// for details. If `start_version` is `None`, this is a no-op. #[allow(unused)] - pub(crate) fn with_start_version_opt(mut self, version: Option) -> Self { + pub fn with_start_version_opt(mut self, version: Option) -> Self { self.start_version = version; self } /// Provide an `end_version` (inclusive) of the [`LogSegment`]. This ensures that all commit and /// checkpoint files are at or below the end version. #[allow(unused)] - pub(crate) fn with_end_version(mut self, version: Version) -> Self { + pub fn with_end_version(mut self, version: Version) -> Self { self.end_version = Some(version); self } /// Optionally provide an `end_version` (inclusive) of the [`LogSegment`]. See [`LogSegmentBuilder::with_end_version`] /// for details. If `end_version` is `None`, this is a no-op. - pub(crate) fn with_end_version_opt(mut self, version: Option) -> Self { + pub fn with_end_version_opt(mut self, version: Option) -> Self { self.end_version = version; self } /// Specify that the [`LogSegment`] will not have any checkpoint files. It will only be made /// up of commit files. #[allow(unused)] - pub(crate) fn with_omit_checkpoint_files(mut self) -> Self { + pub fn with_omit_checkpoint_files(mut self) -> Self { self.omit_checkpoint_files = true; self } /// Specify that the commits in the [`LogSegment`] will be sorted by version in ascending /// order. By default, commits are sorted by version in descending order. #[allow(unused)] - pub(crate) fn with_sort_commit_files_ascending(mut self) -> Self { + pub fn with_sort_commit_files_ascending(mut self) -> Self { self.sort_commit_files_ascending = true; self } /// Build the [`LogSegment`] /// /// This fetches checkpoint and commit files using the `fs_client`. - pub(crate) fn build( + pub fn build( self, fs_client: &dyn FileSystemClient, table_root: &Url, @@ -308,6 +309,12 @@ impl LogSegmentBuilder { } } +impl Default for LogSegmentBuilder { + fn default() -> Self { + Self::new() + } +} + /// List all commit and checkpoint files with versions above the provided `version`. If successful, this returns /// a tuple `(sorted_commit_files_paths, checkpoint_parts): (Vec, Vec)`. /// The commit files are guaranteed to be sorted in ascending order by version. The elements of From 9c44a693684b575f456c99eea03120e918e707c3 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Wed, 13 Nov 2024 17:02:03 -0800 Subject: [PATCH 58/88] Fix doc issue --- kernel/src/log_segment.rs | 33 +++++++++++++-------------------- 1 file changed, 13 insertions(+), 20 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 204ff5346..f40219d7b 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -16,12 +16,12 @@ use tracing::warn; use url::Url; /// A [`LogSegment`] represents a contiguous section of the log and is made up of checkpoint files -/// and commit files. It is built with [`LogSegmentBuilder`], and guarantees the following: +/// and commit files. It is built with `LogSegmentBuilder`, and guarantees the following: /// 1. Commit/checkpoint file versions will be less than or equal to `end_version` if specified. /// 2. Commit file versions will be greater than or equal to `start_version` if specified. /// 3. If checkpoint(s) is/are present in the range, only commits with versions greater than the most /// recent checkpoint version are retained. Checkpoints can be omitted (and this rule skipped) -/// whenever the `LogSegment` is created. See [`LogSegmentBuilder::with_omit_checkpoint_files`]. +/// whenever the `LogSegment` is created. See `LogSegmentBuilder::with_omit_checkpoint_files`. /// /// [`LogSegment`] is used in both [`Snapshot`] and in `TableChanges` to hold commit files and /// checkpoint files. @@ -129,7 +129,7 @@ impl LogSegment { } /// Builder for [`LogSegment`] from from `start_version` to `end_version` inclusive -#[cfg_attr(feature = "developer-visibility", visibility::make(pub))] +/// pub(crate) struct LogSegmentBuilder { start_checkpoint: Option, start_version: Option, @@ -140,9 +140,8 @@ pub(crate) struct LogSegmentBuilder { sort_commit_files_ascending: bool, omit_checkpoint_files: bool, } - impl LogSegmentBuilder { - pub fn new() -> Self { + pub(crate) fn new() -> Self { LogSegmentBuilder { start_checkpoint: None, start_version: None, @@ -156,7 +155,7 @@ impl LogSegmentBuilder { /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] /// with both will result in an error. #[allow(unused)] - pub fn with_start_checkpoint(mut self, start_checkpoint: CheckpointMetadata) -> Self { + pub(crate) fn with_start_checkpoint(mut self, start_checkpoint: CheckpointMetadata) -> Self { self.start_checkpoint = Some(start_checkpoint); self } @@ -165,7 +164,7 @@ impl LogSegmentBuilder { /// /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] /// with both will result in an error. - pub fn with_start_checkpoint_opt( + pub(crate) fn with_start_checkpoint_opt( mut self, start_checkpoint: Option, ) -> Self { @@ -178,48 +177,48 @@ impl LogSegmentBuilder { /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] /// with both will result in an error. #[allow(unused)] - pub fn with_start_version(mut self, version: Version) -> Self { + pub(crate) fn with_start_version(mut self, version: Version) -> Self { self.start_version = Some(version); self } /// Optionally provide a `start_version` of the [`LogSegment`]. See [`LogSegmentBuilder::with_start_version`] /// for details. If `start_version` is `None`, this is a no-op. #[allow(unused)] - pub fn with_start_version_opt(mut self, version: Option) -> Self { + pub(crate) fn with_start_version_opt(mut self, version: Option) -> Self { self.start_version = version; self } /// Provide an `end_version` (inclusive) of the [`LogSegment`]. This ensures that all commit and /// checkpoint files are at or below the end version. #[allow(unused)] - pub fn with_end_version(mut self, version: Version) -> Self { + pub(crate) fn with_end_version(mut self, version: Version) -> Self { self.end_version = Some(version); self } /// Optionally provide an `end_version` (inclusive) of the [`LogSegment`]. See [`LogSegmentBuilder::with_end_version`] /// for details. If `end_version` is `None`, this is a no-op. - pub fn with_end_version_opt(mut self, version: Option) -> Self { + pub(crate) fn with_end_version_opt(mut self, version: Option) -> Self { self.end_version = version; self } /// Specify that the [`LogSegment`] will not have any checkpoint files. It will only be made /// up of commit files. #[allow(unused)] - pub fn with_omit_checkpoint_files(mut self) -> Self { + pub(crate) fn with_omit_checkpoint_files(mut self) -> Self { self.omit_checkpoint_files = true; self } /// Specify that the commits in the [`LogSegment`] will be sorted by version in ascending /// order. By default, commits are sorted by version in descending order. #[allow(unused)] - pub fn with_sort_commit_files_ascending(mut self) -> Self { + pub(crate) fn with_sort_commit_files_ascending(mut self) -> Self { self.sort_commit_files_ascending = true; self } /// Build the [`LogSegment`] /// /// This fetches checkpoint and commit files using the `fs_client`. - pub fn build( + pub(crate) fn build( self, fs_client: &dyn FileSystemClient, table_root: &Url, @@ -309,12 +308,6 @@ impl LogSegmentBuilder { } } -impl Default for LogSegmentBuilder { - fn default() -> Self { - Self::new() - } -} - /// List all commit and checkpoint files with versions above the provided `version`. If successful, this returns /// a tuple `(sorted_commit_files_paths, checkpoint_parts): (Vec, Vec)`. /// The commit files are guaranteed to be sorted in ascending order by version. The elements of From 3a214deb300fa684675d00d58ba2877ce1bb7821 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 09:57:13 -0800 Subject: [PATCH 59/88] Address nits --- kernel/src/log_segment.rs | 68 ++++++++++++--------------------------- kernel/src/snapshot.rs | 4 +-- 2 files changed, 22 insertions(+), 50 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index f40219d7b..a0473ec53 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -21,14 +21,14 @@ use url::Url; /// 2. Commit file versions will be greater than or equal to `start_version` if specified. /// 3. If checkpoint(s) is/are present in the range, only commits with versions greater than the most /// recent checkpoint version are retained. Checkpoints can be omitted (and this rule skipped) -/// whenever the `LogSegment` is created. See `LogSegmentBuilder::with_omit_checkpoint_files`. +/// whenever the `LogSegment` is created. See `LogSegmentBuilder::without_checkpoint_files`. /// /// [`LogSegment`] is used in both [`Snapshot`] and in `TableChanges` to hold commit files and /// checkpoint files. /// - For a Snapshot at version `n`: Its LogSegment is made up of zero or one checkpoint, and all /// commits between the checkpoint and the end version `n`. /// - For a TableChanges between versions `a` and `b`: Its LogSegment is made up of zero -/// checkpoints and all commits between versions `a` and `b` +/// checkpoints and all commits between versions `a` and `b` (inclusive) /// /// [`Snapshot`]: crate::snapshot::Snapshot #[derive(Debug)] @@ -129,7 +129,7 @@ impl LogSegment { } /// Builder for [`LogSegment`] from from `start_version` to `end_version` inclusive -/// +#[derive(Default)] pub(crate) struct LogSegmentBuilder { start_checkpoint: Option, start_version: Option, @@ -138,37 +138,22 @@ pub(crate) struct LogSegmentBuilder { /// ascending order. Otherwise if it is set to `false`, the commit files are sorted in /// descending order. This is set to `false` by default. sort_commit_files_ascending: bool, - omit_checkpoint_files: bool, + without_checkpoint_files: bool, } impl LogSegmentBuilder { pub(crate) fn new() -> Self { - LogSegmentBuilder { - start_checkpoint: None, - start_version: None, - end_version: None, - sort_commit_files_ascending: false, - omit_checkpoint_files: false, - } + Self::default() } /// Provide a checkpoint metadata to start the log segment from (e.g. from reading the `last_checkpoint` file). /// /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] /// with both will result in an error. #[allow(unused)] - pub(crate) fn with_start_checkpoint(mut self, start_checkpoint: CheckpointMetadata) -> Self { - self.start_checkpoint = Some(start_checkpoint); - self - } - /// Optionally provide a checkpoint metadata to start the log segment. See [`LogSegmentBuilder::with_start_checkpoint`] - /// for details. If `start_checkpoint` is `None`, this is a no-op. - /// - /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] - /// with both will result in an error. - pub(crate) fn with_start_checkpoint_opt( + pub(crate) fn with_start_checkpoint( mut self, - start_checkpoint: Option, + start_checkpoint: impl Into>, ) -> Self { - self.start_checkpoint = start_checkpoint; + self.start_checkpoint = start_checkpoint.into(); self } /// Provide a `start_version` (inclusive) of the [`LogSegment`] that ensures that all commit files @@ -177,35 +162,22 @@ impl LogSegmentBuilder { /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] /// with both will result in an error. #[allow(unused)] - pub(crate) fn with_start_version(mut self, version: Version) -> Self { - self.start_version = Some(version); - self - } - /// Optionally provide a `start_version` of the [`LogSegment`]. See [`LogSegmentBuilder::with_start_version`] - /// for details. If `start_version` is `None`, this is a no-op. - #[allow(unused)] - pub(crate) fn with_start_version_opt(mut self, version: Option) -> Self { - self.start_version = version; + pub(crate) fn with_start_version(mut self, version: impl Into>) -> Self { + self.start_version = version.into(); self } /// Provide an `end_version` (inclusive) of the [`LogSegment`]. This ensures that all commit and /// checkpoint files are at or below the end version. #[allow(unused)] - pub(crate) fn with_end_version(mut self, version: Version) -> Self { - self.end_version = Some(version); - self - } - /// Optionally provide an `end_version` (inclusive) of the [`LogSegment`]. See [`LogSegmentBuilder::with_end_version`] - /// for details. If `end_version` is `None`, this is a no-op. - pub(crate) fn with_end_version_opt(mut self, version: Option) -> Self { - self.end_version = version; + pub(crate) fn with_end_version(mut self, version: impl Into>) -> Self { + self.end_version = version.into(); self } /// Specify that the [`LogSegment`] will not have any checkpoint files. It will only be made /// up of commit files. #[allow(unused)] - pub(crate) fn with_omit_checkpoint_files(mut self) -> Self { - self.omit_checkpoint_files = true; + pub(crate) fn without_checkpoint_files(mut self) -> Self { + self.without_checkpoint_files = true; self } /// Specify that the commits in the [`LogSegment`] will be sorted by version in ascending @@ -236,7 +208,7 @@ impl LogSegmentBuilder { start_version, end_version, sort_commit_files_ascending, - omit_checkpoint_files, + without_checkpoint_files, } = self; let log_root = table_root.join("_delta_log/").unwrap(); let (mut sorted_commit_files, mut checkpoint_parts) = @@ -253,7 +225,7 @@ impl LogSegmentBuilder { _ => list_log_files_with_version(fs_client, &log_root, None)?, }; - if omit_checkpoint_files { + if without_checkpoint_files { checkpoint_parts.clear(); } @@ -586,7 +558,7 @@ mod tests { } #[test] - fn test_builder_omit_checkpoints() { + fn test_builder_witouth_checkpoints() { let (client, table_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), @@ -604,7 +576,7 @@ mod tests { None, ); let log_segment = LogSegmentBuilder::new() - .with_omit_checkpoint_files() + .without_checkpoint_files() .build(client.as_ref(), &table_root) .unwrap(); let (commit_files, checkpoint_parts) = @@ -643,7 +615,7 @@ mod tests { let log_segment = LogSegmentBuilder::new() .with_end_version(5) .with_start_version(2) - .with_omit_checkpoint_files() + .without_checkpoint_files() .with_sort_commit_files_ascending() .build(client.as_ref(), &table_root) .unwrap(); @@ -662,7 +634,7 @@ mod tests { // | Specify no start or end version | // -------------------------------------------------------------------------------- let log_segment = LogSegmentBuilder::new() - .with_omit_checkpoint_files() + .without_checkpoint_files() .with_sort_commit_files_ascending() .build(client.as_ref(), &table_root) .unwrap(); diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index d6375d51f..2305c2421 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -63,8 +63,8 @@ impl Snapshot { let log_url = table_root.join("_delta_log/").unwrap(); let log_segment = LogSegmentBuilder::new() - .with_end_version_opt(version) - .with_start_checkpoint_opt(read_last_checkpoint(fs_client.as_ref(), &log_url)?) + .with_end_version(version) + .with_start_checkpoint(read_last_checkpoint(fs_client.as_ref(), &log_url)?) .build(fs_client.as_ref(), &table_root)?; Self::try_new_from_log_segment(table_root, log_segment, engine) From d5adae173c04e8f952811b5ae089d31ba09db92b Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 09:57:37 -0800 Subject: [PATCH 60/88] Update kernel/src/log_segment.rs Co-authored-by: Zach Schuermann --- kernel/src/log_segment.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index a0473ec53..82bcc7bb7 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -36,7 +36,7 @@ use url::Url; pub(crate) struct LogSegment { pub end_version: Version, pub log_root: Url, - /// Commit files in the log segment + /// Sorted commit files in the log segment (see builder for sorting) pub commit_files: Vec, /// Checkpoint files in the log segment. pub checkpoint_parts: Vec, From 6a5484cd677549c6f371a5f3f945d1cd30327d57 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 11:15:54 -0800 Subject: [PATCH 61/88] add test, fix bug --- kernel/src/log_segment.rs | 76 ++++++++++++++++++++++++++++++++------- 1 file changed, 64 insertions(+), 12 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index a0473ec53..3ee4446d2 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -214,15 +214,18 @@ impl LogSegmentBuilder { let (mut sorted_commit_files, mut checkpoint_parts) = match (start_checkpoint, start_version, end_version) { (Some(cp), None, None) => { - list_log_files_with_checkpoint(&cp, fs_client, &log_root)? + list_log_files_with_checkpoint(&cp, fs_client, &log_root, end_version)? } (Some(cp), None, Some(end_version)) if cp.version <= end_version => { - list_log_files_with_checkpoint(&cp, fs_client, &log_root)? + list_log_files_with_checkpoint(&cp, fs_client, &log_root, Some(end_version))? } - (None, Some(start_version), _) => { - list_log_files_with_version(fs_client, &log_root, Some(start_version))? - } - _ => list_log_files_with_version(fs_client, &log_root, None)?, + (None, Some(start_version), _) => list_log_files_with_version( + fs_client, + &log_root, + Some(start_version), + end_version, + )?, + _ => list_log_files_with_version(fs_client, &log_root, None, end_version)?, }; if without_checkpoint_files { @@ -236,12 +239,12 @@ impl LogSegmentBuilder { if let Some(start_version) = start_version { sorted_commit_files.retain(|log_path| log_path.version >= start_version); } - if let Some(checkpoint_file) = checkpoint_parts.first() { - sorted_commit_files.retain(|log_path| checkpoint_file.version < log_path.version); - } if let Some(end_version) = end_version { sorted_commit_files.retain(|log_path| log_path.version <= end_version); } + if let Some(checkpoint_file) = checkpoint_parts.first() { + sorted_commit_files.retain(|log_path| checkpoint_file.version < log_path.version); + } // After (possibly) omitting checkpoint files and filtering commits, we should have commits that are // contiguous. In other words, there must be no gap between commit versions. @@ -261,7 +264,10 @@ impl LogSegmentBuilder { if let Some(end_version) = end_version { require!( version_eff == end_version, - Error::MissingVersion // TODO more descriptive error + Error::generic(format!( + "version effective not the same as end_version {}, {}", + version_eff, end_version + )) // TODO more descriptive error ); } @@ -289,6 +295,7 @@ fn list_log_files_with_version( fs_client: &dyn FileSystemClient, log_root: &Url, version: Option, + end_version: Option, ) -> DeltaResult<(Vec, Vec)> { let begin_version = version.unwrap_or(0); let version_prefix = format!("{:020}", begin_version); @@ -310,6 +317,9 @@ fn list_log_files_with_version( commit_files.push(parsed_path); } else if parsed_path.is_checkpoint() { let path_version = parsed_path.version; + if end_version.is_some_and(|end_version| path_version > end_version) { + continue; + } match max_checkpoint_version { None => { checkpoint_parts.push(parsed_path); @@ -345,9 +355,14 @@ fn list_log_files_with_checkpoint( checkpoint_metadata: &CheckpointMetadata, fs_client: &dyn FileSystemClient, log_root: &Url, + end_version: Option, ) -> DeltaResult<(Vec, Vec)> { - let (commit_files, checkpoint_parts) = - list_log_files_with_version(fs_client, log_root, Some(checkpoint_metadata.version))?; + let (commit_files, checkpoint_parts) = list_log_files_with_version( + fs_client, + log_root, + Some(checkpoint_metadata.version), + end_version, + )?; let Some(latest_checkpoint) = checkpoint_parts.last() else { // TODO: We could potentially recover here @@ -719,4 +734,41 @@ mod tests { .build(client.as_ref(), &table_root); assert!(log_segment_res.is_err()); } + + #[test] + fn test_build_with_start_checkpoint_and_end_version() { + let checkpoint_metadata = CheckpointMetadata { + version: 3, + size: 10, + parts: None, + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(4, "json"), + delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + Some(&checkpoint_metadata), + ); + + let log_segment = LogSegmentBuilder::new() + .with_start_checkpoint(checkpoint_metadata) + .with_end_version(4) + .build(client.as_ref(), &table_root) + .unwrap(); + assert_eq!(log_segment.checkpoint_parts.len(), 1); + assert_eq!(log_segment.checkpoint_parts[0].version, 3); + assert_eq!(log_segment.commit_files.len(), 1); + assert_eq!(log_segment.commit_files[0].version, 4); + } } From 669ae84ee615d3364cb14863fbf42fb6ce5847c7 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 15:44:33 -0800 Subject: [PATCH 62/88] New log segment implementation --- kernel/src/log_segment.rs | 540 +++++++++++++++++++------------------- kernel/src/snapshot.rs | 9 +- 2 files changed, 267 insertions(+), 282 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 5450877ba..48ecb2adb 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -11,38 +11,154 @@ use crate::{ }; use itertools::Itertools; use std::cmp::Ordering; +use std::convert::identity; use std::sync::{Arc, LazyLock}; use tracing::warn; use url::Url; /// A [`LogSegment`] represents a contiguous section of the log and is made up of checkpoint files -/// and commit files. It is built with `LogSegmentBuilder`, and guarantees the following: -/// 1. Commit/checkpoint file versions will be less than or equal to `end_version` if specified. -/// 2. Commit file versions will be greater than or equal to `start_version` if specified. +/// and commit files. It is built with either [`LogSegment::for_table_changes`], or +/// [`LogSegment::for_snapshot`], and guarantees the following: +/// 1. Commit/checkpoint file versions will be less than or equal to `end_version`. +/// 2. Commit file versions will not have any gaps between them. /// 3. If checkpoint(s) is/are present in the range, only commits with versions greater than the most -/// recent checkpoint version are retained. Checkpoints can be omitted (and this rule skipped) -/// whenever the `LogSegment` is created. See `LogSegmentBuilder::without_checkpoint_files`. +/// recent checkpoint version are retained. /// /// [`LogSegment`] is used in both [`Snapshot`] and in `TableChanges` to hold commit files and /// checkpoint files. -/// - For a Snapshot at version `n`: Its LogSegment is made up of zero or one checkpoint, and all -/// commits between the checkpoint and the end version `n`. -/// - For a TableChanges between versions `a` and `b`: Its LogSegment is made up of zero -/// checkpoints and all commits between versions `a` and `b` (inclusive) -/// +/// - For a Snapshot at version `n`: This is created using [`LogSegment::for_snapshot`]. Its LogSegment is +/// made up of zero or one checkpoint, and all commits between the checkpoint up to and including the +/// end version `n`. /// [`Snapshot`]: crate::snapshot::Snapshot #[derive(Debug)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) struct LogSegment { pub end_version: Version, pub log_root: Url, - /// Sorted commit files in the log segment (see builder for sorting) + /// Sorted commit files in the log segment pub commit_files: Vec, /// Checkpoint files in the log segment. pub checkpoint_parts: Vec, } impl LogSegment { + fn try_new( + sorted_commit_files: Vec, + checkpoint_parts: Vec, + log_root: Url, + expected_end_version: Option, + ) -> DeltaResult { + // We should require that commits that are contiguous. In other words, there must be no gap between commit versions. + // There must also be no gap between a checkpoint and the first commit version. + require!( + sorted_commit_files + .windows(2) + .all(|cfs| cfs[0].version + 1 == cfs[1].version), + Error::generic(format!( + "Expected ordered contiguous commit files {:?}", + sorted_commit_files + )) + ); + if let (Some(checkpoint_file), Some(commit_file)) = + (checkpoint_parts.first(), sorted_commit_files.first()) + { + require!(checkpoint_file.version + 1 == commit_file.version, + Error::generic(format!("Expected commit file version {} to be next version to checkpoint file version {}", commit_file.version, checkpoint_file.version ))) + } + + // Get the effective version from chosen files + let version_eff = sorted_commit_files + .last() + .or(checkpoint_parts.first()) + .ok_or(Error::MissingVersion)? // TODO: A more descriptive error + .version; + if let Some(end_version) = expected_end_version { + require!( + version_eff == end_version, + Error::generic(format!( + "version effective not the same as end_version {}, {}", + version_eff, end_version + )) + ); + } + + Ok(LogSegment { + end_version: version_eff, + log_root, + commit_files: sorted_commit_files, + checkpoint_parts, + }) + } + + pub(crate) fn for_snapshot( + fs_client: &dyn FileSystemClient, + table_root: &Url, + checkpoint_hint: impl Into>, + time_travel_version: impl Into>, + ) -> DeltaResult { + let time_travel_version = time_travel_version.into(); + let log_root = table_root.join("_delta_log/").unwrap(); + + let (mut sorted_commit_files, checkpoint_parts) = + match (checkpoint_hint.into(), time_travel_version) { + (Some(cp), None) => { + list_log_files_with_checkpoint(&cp, fs_client, &log_root, None)? + } + (Some(cp), Some(end_version)) if cp.version <= end_version => { + list_log_files_with_checkpoint(&cp, fs_client, &log_root, Some(end_version))? + } + _ => list_log_files_with_version(fs_client, &log_root, None, time_travel_version)?, + }; + + // Commit file versions must be greater than the most recent checkpoint version if it exists + if let Some(checkpoint_file) = checkpoint_parts.first() { + sorted_commit_files.retain(|log_path| checkpoint_file.version < log_path.version); + } + + LogSegment::try_new( + sorted_commit_files, + checkpoint_parts, + log_root, + time_travel_version, + ) + } + + /// Constructs a [`LogSegment`] to be used for `TableChanges`. For a TableChanges between versions + /// `start_version` and `end_version`: Its LogSegment is made up of zero checkpoints and all commits + /// between versions `start_version` and `end_version` (inclusive). If no `end_version` is specified + /// it will be the most recent version by default. + #[allow(unused)] + pub(crate) fn for_table_changes( + fs_client: &dyn FileSystemClient, + table_root: &Url, + start_version: Version, + end_version: impl Into>, + ) -> DeltaResult { + let log_root = table_root.join("_delta_log/").unwrap(); + + let end_version = end_version.into(); + if let (start_version, Some(end_version)) = (start_version, end_version) { + if start_version > end_version { + return Err(Error::generic("Failed to build LogSegment: `start_version` cannot be greater than end_version")); + } + } + + let sorted_commit_files: Vec<_> = + get_parsed_log_files_iter(fs_client, &log_root, start_version, end_version)? + .filter_ok(|x| x.is_commit()) + .try_collect()?; + + require!( + sorted_commit_files + .first() + .is_some_and(|first_commit| first_commit.version == start_version), + Error::generic(format!( + "Expected the first commit to have version {}", + start_version + )) + ); + LogSegment::try_new(sorted_commit_files, vec![], log_root, end_version) + } /// Read a stream of log data from this log segment. /// /// The log files will be read from most recent to oldest. @@ -62,9 +178,12 @@ impl LogSegment { checkpoint_read_schema: SchemaRef, meta_predicate: Option, ) -> DeltaResult, bool)>> + Send> { + // `replay` expects commit files to be sorted in descending order, so we reverse the sorted + // commit files let commit_files: Vec<_> = self .commit_files .iter() + .rev() .map(|f| f.location.clone()) .collect(); let commit_stream = engine @@ -128,198 +247,62 @@ impl LogSegment { } } -/// Builder for [`LogSegment`] from from `start_version` to `end_version` inclusive -#[derive(Default)] -pub(crate) struct LogSegmentBuilder { - start_checkpoint: Option, - start_version: Option, - end_version: Option, - /// When `sort_commit_files_ascending` is set to `true`, the commit files are sorted in - /// ascending order. Otherwise if it is set to `false`, the commit files are sorted in - /// descending order. This is set to `false` by default. - sort_commit_files_ascending: bool, - without_checkpoint_files: bool, -} -impl LogSegmentBuilder { - pub(crate) fn new() -> Self { - Self::default() - } - /// Provide a checkpoint metadata to start the log segment from (e.g. from reading the `last_checkpoint` file). - /// - /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] - /// with both will result in an error. - #[allow(unused)] - pub(crate) fn with_start_checkpoint( - mut self, - start_checkpoint: impl Into>, - ) -> Self { - self.start_checkpoint = start_checkpoint.into(); - self - } - /// Provide a `start_version` (inclusive) of the [`LogSegment`] that ensures that all commit files - /// are at this version or above it. - /// - /// Note: Either `start_version` or `start_checkpoint` may be specified. Attempting to build a [`LogSegment`] - /// with both will result in an error. - #[allow(unused)] - pub(crate) fn with_start_version(mut self, version: impl Into>) -> Self { - self.start_version = version.into(); - self - } - /// Provide an `end_version` (inclusive) of the [`LogSegment`]. This ensures that all commit and - /// checkpoint files are at or below the end version. - #[allow(unused)] - pub(crate) fn with_end_version(mut self, version: impl Into>) -> Self { - self.end_version = version.into(); - self - } - /// Specify that the [`LogSegment`] will not have any checkpoint files. It will only be made - /// up of commit files. - #[allow(unused)] - pub(crate) fn without_checkpoint_files(mut self) -> Self { - self.without_checkpoint_files = true; - self - } - /// Specify that the commits in the [`LogSegment`] will be sorted by version in ascending - /// order. By default, commits are sorted by version in descending order. - #[allow(unused)] - pub(crate) fn with_sort_commit_files_ascending(mut self) -> Self { - self.sort_commit_files_ascending = true; - self - } - /// Build the [`LogSegment`] - /// - /// This fetches checkpoint and commit files using the `fs_client`. - pub(crate) fn build( - self, - fs_client: &dyn FileSystemClient, - table_root: &Url, - ) -> DeltaResult { - if self.start_version.is_some() && self.start_checkpoint.is_some() { - return Err(Error::generic("Failed to build LogSegment: Cannot specify both start_version and start_checkpoint")); - } - if let (Some(start_version), Some(end_version)) = (self.start_version, self.end_version) { - if start_version > end_version { - return Err(Error::generic("Failed to build LogSegment: `start_version` cannot be greater than end_version")); - } - } - let Self { - start_checkpoint, - start_version, - end_version, - sort_commit_files_ascending, - without_checkpoint_files, - } = self; - let log_root = table_root.join("_delta_log/").unwrap(); - let (mut sorted_commit_files, mut checkpoint_parts) = - match (start_checkpoint, start_version, end_version) { - (Some(cp), None, None) => { - list_log_files_with_checkpoint(&cp, fs_client, &log_root, end_version)? - } - (Some(cp), None, Some(end_version)) if cp.version <= end_version => { - list_log_files_with_checkpoint(&cp, fs_client, &log_root, Some(end_version))? - } - (None, Some(start_version), _) => list_log_files_with_version( - fs_client, - &log_root, - Some(start_version), - end_version, - )?, - _ => list_log_files_with_version(fs_client, &log_root, None, end_version)?, - }; - - if without_checkpoint_files { - checkpoint_parts.clear(); - } - - // Commit file versions must satisfy the following: - // - Be greater than or equal to the start version - // - Be greater than the most recent checkpoint version if it exists - // - Be less than or equal to the end version. - if let Some(start_version) = start_version { - sorted_commit_files.retain(|log_path| log_path.version >= start_version); - } - if let Some(end_version) = end_version { - sorted_commit_files.retain(|log_path| log_path.version <= end_version); - } - if let Some(checkpoint_file) = checkpoint_parts.first() { - sorted_commit_files.retain(|log_path| checkpoint_file.version < log_path.version); - } - - // After (possibly) omitting checkpoint files and filtering commits, we should have commits that are - // contiguous. In other words, there must be no gap between commit versions. - let ordered_commits = sorted_commit_files - .windows(2) - .all(|cfs| cfs[0].version + 1 == cfs[1].version); - if !ordered_commits { - return Err(Error::generic("Expected ordered contiguous commit files")); - } +/// Returns a fallible iterator of [`ParsedLogPath`] that are between the provided `start_version` (inclusive) +/// and `end_version` (inclusive). If `start_version` is not specified, the files will begin from +/// version number 0. If `end_version` is not specified, files up to the most recent version will be +/// included. +/// +/// Note: this calls [`FileSystemClient::list_from`] to get the list of log files. +fn get_parsed_log_files_iter( + fs_client: &dyn FileSystemClient, + log_root: &Url, + start_version: impl Into>, + end_version: impl Into>, +) -> DeltaResult>> { + let start_version = start_version.into().unwrap_or(0); + let end_version = end_version.into(); - // get the effective version from chosen files - let version_eff = sorted_commit_files - .last() - .or(checkpoint_parts.first()) - .ok_or(Error::MissingVersion)? // TODO: A more descriptive error - .version; - if let Some(end_version) = end_version { - require!( - version_eff == end_version, - Error::generic(format!( - "version effective not the same as end_version {}, {}", - version_eff, end_version - )) // TODO more descriptive error - ); - } + let version_prefix = format!("{:020}", start_version); + let start_from = log_root.join(&version_prefix)?; - // We assume commit files are sorted in ascending order. If `sort_commit_files_ascending` - // is false, reverse to make it descending. - if !sort_commit_files_ascending { - sorted_commit_files.reverse(); - } + // Check that the provided version is less than or equal to the end version if it exists + let lte_end_version = move |version: u64| { + end_version.is_none() || end_version.is_some_and(|end_version| version <= end_version) + }; - Ok(LogSegment { - end_version: version_eff, - log_root, - commit_files: sorted_commit_files, - checkpoint_parts, - }) - } + Ok(fs_client + .list_from(&start_from)? + .map(|meta| ParsedLogPath::try_from(meta?)) + // TODO this filters out .crc files etc which start with "." - how do we want to use these kind of files? + .filter_map_ok(identity) + .take_while(move |x| match x { + Ok(x) => lte_end_version(x.version), + _ => false, + })) } - -/// List all commit and checkpoint files with versions above the provided `version`. If successful, this returns -/// a tuple `(sorted_commit_files_paths, checkpoint_parts): (Vec, Vec)`. -/// The commit files are guaranteed to be sorted in ascending order by version. The elements of -/// `checkpoint_parts` are all the parts of the same checkpoint. Checkpoint parts share the same -/// version. +/// List all commit and checkpoint files with versions above the provided `start_version` (inclusive). +/// If successful, this returns a tuple `(sorted_commit_files_paths, checkpoint_parts)` of type +/// `(Vec, Vec)`. The commit files are guaranteed to be sorted in +/// ascending order by version. The elements of `checkpoint_parts` are all the parts of the same +/// checkpoint. Checkpoint parts share the same version. fn list_log_files_with_version( fs_client: &dyn FileSystemClient, log_root: &Url, - version: Option, + start_version: Option, end_version: Option, ) -> DeltaResult<(Vec, Vec)> { - let begin_version = version.unwrap_or(0); - let version_prefix = format!("{:020}", begin_version); - let start_from = log_root.join(&version_prefix)?; - - let mut max_checkpoint_version = version; + let mut max_checkpoint_version = start_version; let mut checkpoint_parts = vec![]; // We expect 10 commit files per checkpoint, so start with that size. We could adjust this based // on config at some point let mut commit_files = Vec::with_capacity(10); - for meta_res in fs_client.list_from(&start_from)? { - let meta = meta_res?; - // TODO this filters out .crc files etc which start with "." - how do we want to use these kind of files? - let Some(parsed_path) = ParsedLogPath::try_from(meta)? else { - continue; - }; + for parsed_path in get_parsed_log_files_iter(fs_client, log_root, start_version, end_version)? { + let parsed_path = parsed_path?; if parsed_path.is_commit() { commit_files.push(parsed_path); } else if parsed_path.is_checkpoint() { let path_version = parsed_path.version; - if end_version.is_some_and(|end_version| path_version > end_version) { - continue; - } match max_checkpoint_version { None => { checkpoint_parts.push(parsed_path); @@ -338,18 +321,11 @@ fn list_log_files_with_version( } } - debug_assert!( - commit_files - .windows(2) - .all(|cfs| cfs[0].version <= cfs[1].version), - "fs_client.list_from() didn't return a sorted listing! {:?}", - commit_files - ); - Ok((commit_files, checkpoint_parts)) } -/// List all commit and checkpoint files after the provided checkpoint. +/// List all commit and checkpoint files after the provided checkpoint. It is guaranteed that all +/// the returned [`ParsedLogPath`]s will have a version less than or equal to the `end_version`. /// See [`list_log_files_with_version`] for details on the return type. fn list_log_files_with_checkpoint( checkpoint_metadata: &CheckpointMetadata, @@ -398,7 +374,7 @@ mod tests { use crate::engine::default::executor::tokio::TokioBackgroundExecutor; use crate::engine::default::filesystem::ObjectStoreFileSystemClient; use crate::engine::sync::SyncEngine; - use crate::log_segment::LogSegmentBuilder; + use crate::log_segment::LogSegment; use crate::snapshot::CheckpointMetadata; use crate::{FileSystemClient, Table}; use test_utils::delta_path_for_version; @@ -516,18 +492,17 @@ mod tests { Some(&checkpoint_metadata), ); - let log_segment = LogSegmentBuilder::new() - .with_start_checkpoint(checkpoint_metadata) - .build(client.as_ref(), &table_root) - .unwrap(); + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None) + .unwrap(); let (commit_files, checkpoint_parts) = (log_segment.commit_files, log_segment.checkpoint_parts); assert_eq!(checkpoint_parts.len(), 1); assert_eq!(commit_files.len(), 2); assert_eq!(checkpoint_parts[0].version, 5); - assert_eq!(commit_files[0].version, 7); - assert_eq!(commit_files[1].version, 6); + assert_eq!(commit_files[0].version, 6); + assert_eq!(commit_files[1].version, 7); } #[test] fn test_read_log_with_correct_last_checkpoint() { @@ -558,22 +533,21 @@ mod tests { Some(&checkpoint_metadata), ); - let log_segment = LogSegmentBuilder::new() - .with_start_checkpoint(checkpoint_metadata) - .build(client.as_ref(), &table_root) - .unwrap(); + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None) + .unwrap(); let (commit_files, checkpoint_parts) = (log_segment.commit_files, log_segment.checkpoint_parts); assert_eq!(checkpoint_parts.len(), 1); assert_eq!(commit_files.len(), 2); assert_eq!(checkpoint_parts[0].version, 5); - assert_eq!(commit_files[0].version, 7); - assert_eq!(commit_files[1].version, 6); + assert_eq!(commit_files[0].version, 6); + assert_eq!(commit_files[1].version, 7); } #[test] - fn test_builder_witouth_checkpoints() { + fn test_snapshot_without_checkpoints() { let (client, table_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), @@ -590,21 +564,61 @@ mod tests { ], None, ); - let log_segment = LogSegmentBuilder::new() - .without_checkpoint_files() - .build(client.as_ref(), &table_root) - .unwrap(); + + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, None, None).unwrap(); let (commit_files, checkpoint_parts) = (log_segment.commit_files, log_segment.checkpoint_parts); - // Checkpoints should be omitted - assert_eq!(checkpoint_parts.len(), 0); + assert_eq!(checkpoint_parts.len(), 1); + assert_eq!(checkpoint_parts[0].version, 5); // All commit files should still be there let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); - let expected_versions = (0..=7).rev().collect_vec(); + let expected_versions = vec![6, 7]; assert_eq!(versions, expected_versions); } + + #[test] + fn test_builder_with_checkpoint_greater_than_time_travel() { + let checkpoint_metadata = CheckpointMetadata { + version: 5, + size: 10, + parts: None, + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(4, "json"), + delta_path_for_version(5, "json"), + delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + None, + ); + + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, Some(4)) + .unwrap(); + let (commit_files, checkpoint_parts) = + (log_segment.commit_files, log_segment.checkpoint_parts); + + assert_eq!(checkpoint_parts.len(), 1); + assert_eq!(checkpoint_parts[0].version, 3); + + assert_eq!(commit_files.len(), 1); + assert_eq!(commit_files[0].version, 4); + } #[test] fn test_log_segment_commit_versions() { let (client, table_root) = build_log_with_paths_and_checkpoint( @@ -627,13 +641,9 @@ mod tests { // -------------------------------------------------------------------------------- // | Specify start version and end version | // -------------------------------------------------------------------------------- - let log_segment = LogSegmentBuilder::new() - .with_end_version(5) - .with_start_version(2) - .without_checkpoint_files() - .with_sort_commit_files_ascending() - .build(client.as_ref(), &table_root) - .unwrap(); + + let log_segment = + LogSegment::for_table_changes(client.as_ref(), &table_root, 2, 5).unwrap(); let (commit_files, checkpoint_parts) = (log_segment.commit_files, log_segment.checkpoint_parts); @@ -645,14 +655,26 @@ mod tests { let expected_versions = (2..=5).collect_vec(); assert_eq!(versions, expected_versions); + // -------------------------------------------------------------------------------- + // | Start version and end version are the same | + // -------------------------------------------------------------------------------- + let log_segment = + LogSegment::for_table_changes(client.as_ref(), &table_root, 0, Some(0)).unwrap(); + + let (commit_files, checkpoint_parts) = + (log_segment.commit_files, log_segment.checkpoint_parts); + // Checkpoints should be omitted + assert_eq!(checkpoint_parts.len(), 0); + + // There should only be commit version 0 + assert_eq!(commit_files.len(), 1); + assert_eq!(commit_files[0].version, 0); + // -------------------------------------------------------------------------------- // | Specify no start or end version | // -------------------------------------------------------------------------------- - let log_segment = LogSegmentBuilder::new() - .without_checkpoint_files() - .with_sort_commit_files_ascending() - .build(client.as_ref(), &table_root) - .unwrap(); + let log_segment = + LogSegment::for_table_changes(client.as_ref(), &table_root, 0, None).unwrap(); let (commit_files, checkpoint_parts) = (log_segment.commit_files, log_segment.checkpoint_parts); @@ -675,7 +697,14 @@ mod tests { ], None, ); - let log_segment_res = LogSegmentBuilder::new().build(client.as_ref(), &table_root); + let log_segment_res = LogSegment::for_table_changes(client.as_ref(), &table_root, 0, None); + assert!(log_segment_res.is_err()); + + let log_segment_res = LogSegment::for_table_changes(client.as_ref(), &table_root, 1, None); + assert!(log_segment_res.is_err()); + + let log_segment_res = + LogSegment::for_table_changes(client.as_ref(), &table_root, 0, Some(1)); assert!(log_segment_res.is_err()); } @@ -689,49 +718,8 @@ mod tests { ], None, ); - let log_segment_res = LogSegmentBuilder::new() - .with_start_version(1) - .with_end_version(0) - .build(client.as_ref(), &table_root); - assert!(log_segment_res.is_err()); - - let log_segment_res = LogSegmentBuilder::new() - .with_start_version(0) - .with_end_version(0) - .build(client.as_ref(), &table_root); - assert!(log_segment_res.is_ok()); - } - - #[test] - fn test_build_with_start_version_and_checkpoint_fails() { - let checkpoint_metadata = CheckpointMetadata { - version: 3, - size: 10, - parts: None, - size_in_bytes: None, - num_of_add_files: None, - checkpoint_schema: None, - checksum: None, - }; - - let (client, table_root) = build_log_with_paths_and_checkpoint( - &[ - delta_path_for_version(0, "json"), - delta_path_for_version(1, "checkpoint.parquet"), - delta_path_for_version(2, "json"), - delta_path_for_version(3, "checkpoint.parquet"), - delta_path_for_version(4, "json"), - delta_path_for_version(5, "checkpoint.parquet"), - delta_path_for_version(6, "json"), - delta_path_for_version(7, "json"), - ], - Some(&checkpoint_metadata), - ); - - let log_segment_res = LogSegmentBuilder::new() - .with_start_checkpoint(checkpoint_metadata) - .with_start_version(5) - .build(client.as_ref(), &table_root); + let log_segment_res = + LogSegment::for_table_changes(client.as_ref(), &table_root, 1, Some(0)); assert!(log_segment_res.is_err()); } @@ -761,12 +749,10 @@ mod tests { Some(&checkpoint_metadata), ); - let log_segment = LogSegmentBuilder::new() - .with_start_checkpoint(checkpoint_metadata) - .with_end_version(4) - .build(client.as_ref(), &table_root) - .unwrap(); - assert_eq!(log_segment.checkpoint_parts.len(), 1); + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, Some(4)) + .unwrap(); + assert_eq!(log_segment.checkpoint_parts[0].version, 3); assert_eq!(log_segment.commit_files.len(), 1); assert_eq!(log_segment.commit_files[0].version, 4); diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 2305c2421..498d9a841 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -9,7 +9,7 @@ use url::Url; use crate::actions::{Metadata, Protocol}; use crate::features::{ColumnMappingMode, COLUMN_MAPPING_MODE_KEY}; -use crate::log_segment::{LogSegment, LogSegmentBuilder}; +use crate::log_segment::LogSegment; use crate::scan::ScanBuilder; use crate::schema::Schema; use crate::{DeltaResult, Engine, Error, FileSystemClient, Version}; @@ -62,10 +62,9 @@ impl Snapshot { let fs_client = engine.get_file_system_client(); let log_url = table_root.join("_delta_log/").unwrap(); - let log_segment = LogSegmentBuilder::new() - .with_end_version(version) - .with_start_checkpoint(read_last_checkpoint(fs_client.as_ref(), &log_url)?) - .build(fs_client.as_ref(), &table_root)?; + let checkpoint_hint = read_last_checkpoint(fs_client.as_ref(), &log_url)?; + let log_segment = + LogSegment::for_snapshot(fs_client.as_ref(), &table_root, checkpoint_hint, version)?; Self::try_new_from_log_segment(table_root, log_segment, engine) } From 1fbb56358bbb9e66c65d7c9d0b88f9ebd9c8a5f7 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 15:59:26 -0800 Subject: [PATCH 63/88] fix naming --- kernel/src/log_segment.rs | 24 +++++++++++++++--------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 48ecb2adb..5fd151e8a 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -19,16 +19,14 @@ use url::Url; /// A [`LogSegment`] represents a contiguous section of the log and is made up of checkpoint files /// and commit files. It is built with either [`LogSegment::for_table_changes`], or /// [`LogSegment::for_snapshot`], and guarantees the following: -/// 1. Commit/checkpoint file versions will be less than or equal to `end_version`. -/// 2. Commit file versions will not have any gaps between them. -/// 3. If checkpoint(s) is/are present in the range, only commits with versions greater than the most -/// recent checkpoint version are retained. +/// 1. Commit file versions will not have any gaps between them. +/// 2. If checkpoint(s) is/are present in the range, only commits with versions greater than the most +/// recent checkpoint version are retained. There will not be a gap between the checkpoint +/// version and the first commit version. /// /// [`LogSegment`] is used in both [`Snapshot`] and in `TableChanges` to hold commit files and /// checkpoint files. -/// - For a Snapshot at version `n`: This is created using [`LogSegment::for_snapshot`]. Its LogSegment is -/// made up of zero or one checkpoint, and all commits between the checkpoint up to and including the -/// end version `n`. +/// /// [`Snapshot`]: crate::snapshot::Snapshot #[derive(Debug)] #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] @@ -90,6 +88,13 @@ impl LogSegment { }) } + /// Constructs a [`LogSegment`] to be used for [`Snapshot`]. For a Snapshot at version `n`: + /// Its LogSegment is made up of zero or one checkpoint, and all commits between the checkpoint up + /// to and including the end version `n`. + /// + /// This may leverage a `checkpoint_hint` that is read from `_delta_log/_last_checkpoint`. + /// [`Snapshot`]: crate::snapshot::Snapshot + #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) fn for_snapshot( fs_client: &dyn FileSystemClient, table_root: &Url, @@ -125,9 +130,10 @@ impl LogSegment { /// Constructs a [`LogSegment`] to be used for `TableChanges`. For a TableChanges between versions /// `start_version` and `end_version`: Its LogSegment is made up of zero checkpoints and all commits - /// between versions `start_version` and `end_version` (inclusive). If no `end_version` is specified - /// it will be the most recent version by default. + /// between versions `start_version` (inclusive) and `end_version` (inclusive). If no `end_version` + /// is specified it will be the most recent version by default. #[allow(unused)] + #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) fn for_table_changes( fs_client: &dyn FileSystemClient, table_root: &Url, From dd7bd7d925f700d4cab03f9c388da8f945d17b3d Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 16:42:24 -0800 Subject: [PATCH 64/88] add more tests for checkpoint parts --- kernel/src/log_segment.rs | 182 +++++++++++++++++++++++++++++--------- test-utils/src/lib.rs | 6 ++ 2 files changed, 145 insertions(+), 43 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 5fd151e8a..70888b302 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -383,7 +383,7 @@ mod tests { use crate::log_segment::LogSegment; use crate::snapshot::CheckpointMetadata; use crate::{FileSystemClient, Table}; - use test_utils::delta_path_for_version; + use test_utils::{delta_path_for_checkpoint_part, delta_path_for_version}; // NOTE: In addition to testing the meta-predicate for metadata replay, this test also verifies // that the parquet reader properly infers nullcount = rowcount for missing columns. The two @@ -511,11 +511,11 @@ mod tests { assert_eq!(commit_files[1].version, 7); } #[test] - fn test_read_log_with_correct_last_checkpoint() { + fn test_read_log_with_correct_last_multipart_checkpoint() { let checkpoint_metadata = CheckpointMetadata { version: 5, size: 10, - parts: None, + parts: Some(3), size_in_bytes: None, num_of_add_files: None, checkpoint_schema: None, @@ -531,7 +531,9 @@ mod tests { delta_path_for_version(3, "checkpoint.parquet"), delta_path_for_version(3, "json"), delta_path_for_version(4, "json"), - delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_checkpoint_part(5, 1, 3), + delta_path_for_checkpoint_part(5, 2, 3), + delta_path_for_checkpoint_part(5, 3, 3), delta_path_for_version(5, "json"), delta_path_for_version(6, "json"), delta_path_for_version(7, "json"), @@ -545,13 +547,88 @@ mod tests { let (commit_files, checkpoint_parts) = (log_segment.commit_files, log_segment.checkpoint_parts); - assert_eq!(checkpoint_parts.len(), 1); + assert_eq!(checkpoint_parts.len(), 3); assert_eq!(commit_files.len(), 2); assert_eq!(checkpoint_parts[0].version, 5); assert_eq!(commit_files[0].version, 6); assert_eq!(commit_files[1].version, 7); } + #[test] + fn test_read_log_with_missing_checkpoint_part_from_hint() { + let checkpoint_metadata = CheckpointMetadata { + version: 5, + size: 10, + parts: None, + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(1, "json"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(3, "json"), + delta_path_for_version(4, "json"), + delta_path_for_checkpoint_part(5, 1, 3), + // Part 2 is missing! + delta_path_for_checkpoint_part(5, 3, 3), + delta_path_for_version(5, "json"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + Some(&checkpoint_metadata), + ); + + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None); + assert!(log_segment.is_err()) + } + + #[ignore] + #[test] + fn test_read_log_with_missing_checkpoint_part_no_hint() { + // TODO(Oussam): Hande checkpoints correctly so that this test passes + // Part 2 of 3 is missing from checkpoint 5. The Snapshot should be made up of checkpoint + // number 3 and commit files 4 to 7. + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(1, "json"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(3, "json"), + delta_path_for_version(4, "json"), + delta_path_for_checkpoint_part(5, 1, 3), + // Part 2 is missing! + delta_path_for_checkpoint_part(5, 3, 3), + delta_path_for_version(5, "json"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + None, + ); + + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, None, None).unwrap(); + + let (commit_files, checkpoint_parts) = + (log_segment.commit_files, log_segment.checkpoint_parts); + + assert_eq!(checkpoint_parts.len(), 1); + assert_eq!(checkpoint_parts[0].version, 3); + + let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); + let expected_versions = vec![4, 5, 6, 7]; + assert_eq!(versions, expected_versions); + } + #[test] fn test_snapshot_without_checkpoints() { let (client, table_root) = build_log_with_paths_and_checkpoint( @@ -571,6 +648,9 @@ mod tests { None, ); + // -------------------------------------------------------------------------------- + // | Specify no checkpoint or end version | + // -------------------------------------------------------------------------------- let log_segment = LogSegment::for_snapshot(client.as_ref(), &table_root, None, None).unwrap(); let (commit_files, checkpoint_parts) = @@ -583,10 +663,26 @@ mod tests { let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); let expected_versions = vec![6, 7]; assert_eq!(versions, expected_versions); + + // -------------------------------------------------------------------------------- + // | Specify only end version | + // -------------------------------------------------------------------------------- + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, None, Some(2)).unwrap(); + let (commit_files, checkpoint_parts) = + (log_segment.commit_files, log_segment.checkpoint_parts); + + assert_eq!(checkpoint_parts.len(), 1); + assert_eq!(checkpoint_parts[0].version, 1); + + // All commit files should still be there + let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); + let expected_versions = vec![2]; + assert_eq!(versions, expected_versions); } #[test] - fn test_builder_with_checkpoint_greater_than_time_travel() { + fn build_snapshot_with_checkpoint_greater_than_time_travel_version() { let checkpoint_metadata = CheckpointMetadata { version: 5, size: 10, @@ -625,8 +721,43 @@ mod tests { assert_eq!(commit_files.len(), 1); assert_eq!(commit_files[0].version, 4); } + + #[test] + fn build_snapshot_with_start_checkpoint_and_time_travel_version() { + let checkpoint_metadata = CheckpointMetadata { + version: 3, + size: 10, + parts: None, + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(4, "json"), + delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + Some(&checkpoint_metadata), + ); + + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, Some(4)) + .unwrap(); + + assert_eq!(log_segment.checkpoint_parts[0].version, 3); + assert_eq!(log_segment.commit_files.len(), 1); + assert_eq!(log_segment.commit_files[0].version, 4); + } #[test] - fn test_log_segment_commit_versions() { + fn build_table_changes_with_commit_versions() { let (client, table_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), @@ -715,7 +846,7 @@ mod tests { } #[test] - fn test_larger_start_version_is_fail() { + fn table_changes_fails_with_larger_start_version_than_end() { // Commit with version 1 is missing let (client, table_root) = build_log_with_paths_and_checkpoint( &[ @@ -728,39 +859,4 @@ mod tests { LogSegment::for_table_changes(client.as_ref(), &table_root, 1, Some(0)); assert!(log_segment_res.is_err()); } - - #[test] - fn test_build_with_start_checkpoint_and_end_version() { - let checkpoint_metadata = CheckpointMetadata { - version: 3, - size: 10, - parts: None, - size_in_bytes: None, - num_of_add_files: None, - checkpoint_schema: None, - checksum: None, - }; - - let (client, table_root) = build_log_with_paths_and_checkpoint( - &[ - delta_path_for_version(0, "json"), - delta_path_for_version(1, "checkpoint.parquet"), - delta_path_for_version(2, "json"), - delta_path_for_version(3, "checkpoint.parquet"), - delta_path_for_version(4, "json"), - delta_path_for_version(5, "checkpoint.parquet"), - delta_path_for_version(6, "json"), - delta_path_for_version(7, "json"), - ], - Some(&checkpoint_metadata), - ); - - let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, Some(4)) - .unwrap(); - - assert_eq!(log_segment.checkpoint_parts[0].version, 3); - assert_eq!(log_segment.commit_files.len(), 1); - assert_eq!(log_segment.commit_files[0].version, 4); - } } diff --git a/test-utils/src/lib.rs b/test-utils/src/lib.rs index e43e8481c..074ad570e 100644 --- a/test-utils/src/lib.rs +++ b/test-utils/src/lib.rs @@ -88,6 +88,12 @@ pub fn delta_path_for_version(version: u64, suffix: &str) -> Path { Path::from(path.as_str()) } +// get an ObjectStore path for a checkpoint file, based on version, part number, and total number of parts +pub fn delta_path_for_checkpoint_part(version: u64, part_number: u64, total_parts: u64) -> Path { + let path = + format!("_delta_log/{version:020}.checkpoint.{part_number:010}.{total_parts:010}.parquet"); + Path::from(path.as_str()) +} /// put a commit file into the specified object store. pub async fn add_commit( store: &dyn ObjectStore, From 673381f83c25f0f7df612b1724c2d0fdb8979d24 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 16:58:02 -0800 Subject: [PATCH 65/88] make sure parts is checked --- kernel/src/log_segment.rs | 56 ++++++++++++++++++++++++++++++++------- 1 file changed, 46 insertions(+), 10 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 70888b302..c2c273206 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -88,12 +88,13 @@ impl LogSegment { }) } - /// Constructs a [`LogSegment`] to be used for [`Snapshot`]. For a Snapshot at version `n`: + /// Constructs a [`LogSegment`] to be used for Snapshot. For a Snapshot at version `n`: /// Its LogSegment is made up of zero or one checkpoint, and all commits between the checkpoint up /// to and including the end version `n`. /// /// This may leverage a `checkpoint_hint` that is read from `_delta_log/_last_checkpoint`. - /// [`Snapshot`]: crate::snapshot::Snapshot + /// + /// #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) fn for_snapshot( fs_client: &dyn FileSystemClient, @@ -352,14 +353,14 @@ fn list_log_files_with_checkpoint( "Had a _last_checkpoint hint but didn't find any checkpoints", )); }; - if latest_checkpoint.version != checkpoint_metadata.version { warn!( "_last_checkpoint hint is out of date. _last_checkpoint version: {}. Using actual most recent: {}", checkpoint_metadata.version, latest_checkpoint.version ); - } else if checkpoint_parts.len() != checkpoint_metadata.parts.unwrap_or(1) { + } + if checkpoint_parts.len() != checkpoint_metadata.parts.unwrap_or(1) { return Err(Error::Generic(format!( "_last_checkpoint indicated that checkpoint should have {} parts, but it has {}", checkpoint_metadata.parts.unwrap_or(1), @@ -473,7 +474,7 @@ mod tests { } #[test] - fn test_read_log_with_out_of_date_last_checkpoint() { + fn build_snapshot_with_out_of_date_last_checkpoint() { let checkpoint_metadata = CheckpointMetadata { version: 3, size: 10, @@ -511,7 +512,7 @@ mod tests { assert_eq!(commit_files[1].version, 7); } #[test] - fn test_read_log_with_correct_last_multipart_checkpoint() { + fn build_snapshot_with_correct_last_multipart_checkpoint() { let checkpoint_metadata = CheckpointMetadata { version: 5, size: 10, @@ -555,11 +556,11 @@ mod tests { } #[test] - fn test_read_log_with_missing_checkpoint_part_from_hint() { + fn build_snapshot_with_missing_checkpoint_part_from_hint_fails() { let checkpoint_metadata = CheckpointMetadata { version: 5, size: 10, - parts: None, + parts: Some(3), size_in_bytes: None, num_of_add_files: None, checkpoint_schema: None, @@ -589,10 +590,44 @@ mod tests { LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None); assert!(log_segment.is_err()) } + #[test] + fn build_snapshot_with_bad_checkpoint_hint_fails() { + let checkpoint_metadata = CheckpointMetadata { + version: 5, + size: 10, + parts: Some(1), + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(1, "json"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(3, "json"), + delta_path_for_version(4, "json"), + delta_path_for_checkpoint_part(5, 1, 2), + delta_path_for_checkpoint_part(5, 2, 2), + delta_path_for_version(5, "json"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + Some(&checkpoint_metadata), + ); + + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None); + assert!(log_segment.is_err()) + } #[ignore] #[test] - fn test_read_log_with_missing_checkpoint_part_no_hint() { + fn build_snapshot_with_missing_checkpoint_part_no_hint() { // TODO(Oussam): Hande checkpoints correctly so that this test passes // Part 2 of 3 is missing from checkpoint 5. The Snapshot should be made up of checkpoint // number 3 and commit files 4 to 7. @@ -630,7 +665,7 @@ mod tests { } #[test] - fn test_snapshot_without_checkpoints() { + fn build_snapshot_without_checkpoints() { let (client, table_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), @@ -834,6 +869,7 @@ mod tests { ], None, ); + let log_segment_res = LogSegment::for_table_changes(client.as_ref(), &table_root, 0, None); assert!(log_segment_res.is_err()); From 9873380b32e7f5547c9f210dd642d7e0f06f8a8e Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 16:59:58 -0800 Subject: [PATCH 66/88] naming --- kernel/src/log_segment.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index c2c273206..105b7e2d7 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -151,7 +151,7 @@ impl LogSegment { } let sorted_commit_files: Vec<_> = - get_parsed_log_files_iter(fs_client, &log_root, start_version, end_version)? + parsed_log_files_iter(fs_client, &log_root, start_version, end_version)? .filter_ok(|x| x.is_commit()) .try_collect()?; @@ -260,7 +260,7 @@ impl LogSegment { /// included. /// /// Note: this calls [`FileSystemClient::list_from`] to get the list of log files. -fn get_parsed_log_files_iter( +fn parsed_log_files_iter( fs_client: &dyn FileSystemClient, log_root: &Url, start_version: impl Into>, @@ -304,7 +304,7 @@ fn list_log_files_with_version( // on config at some point let mut commit_files = Vec::with_capacity(10); - for parsed_path in get_parsed_log_files_iter(fs_client, log_root, start_version, end_version)? { + for parsed_path in parsed_log_files_iter(fs_client, log_root, start_version, end_version)? { let parsed_path = parsed_path?; if parsed_path.is_commit() { commit_files.push(parsed_path); From 04a7ecf88774f87d1dbc6adb24d392fb9d13680a Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 17:02:17 -0800 Subject: [PATCH 67/88] Fix spacing --- kernel/src/log_segment.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 105b7e2d7..c24bb71a6 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -93,8 +93,6 @@ impl LogSegment { /// to and including the end version `n`. /// /// This may leverage a `checkpoint_hint` that is read from `_delta_log/_last_checkpoint`. - /// - /// #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) fn for_snapshot( fs_client: &dyn FileSystemClient, From a16c71c8987300240e38c27a1fb4dc7f3058e6ff Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 17:12:36 -0800 Subject: [PATCH 68/88] Fix bug --- kernel/src/log_segment.rs | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index c24bb71a6..c93ba1d8c 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -46,7 +46,7 @@ impl LogSegment { log_root: Url, expected_end_version: Option, ) -> DeltaResult { - // We should require that commits that are contiguous. In other words, there must be no gap between commit versions. + // We require that commits that are contiguous. In other words, there must be no gap between commit versions. // There must also be no gap between a checkpoint and the first commit version. require!( sorted_commit_files @@ -90,7 +90,8 @@ impl LogSegment { /// Constructs a [`LogSegment`] to be used for Snapshot. For a Snapshot at version `n`: /// Its LogSegment is made up of zero or one checkpoint, and all commits between the checkpoint up - /// to and including the end version `n`. + /// to and including the end version `n`. Note that a checkpoint may be made up of multiple + /// parts. /// /// This may leverage a `checkpoint_hint` that is read from `_delta_log/_last_checkpoint`. #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] @@ -253,9 +254,9 @@ impl LogSegment { } /// Returns a fallible iterator of [`ParsedLogPath`] that are between the provided `start_version` (inclusive) -/// and `end_version` (inclusive). If `start_version` is not specified, the files will begin from -/// version number 0. If `end_version` is not specified, files up to the most recent version will be -/// included. +/// and `end_version` (inclusive). [`ParsedLogPath`] may be a commit or a checkpoint. If `start_version` is +/// not specified, the files will begin from version number 0. If `end_version` is not specified, files up to +/// the most recent version will be included. /// /// Note: this calls [`FileSystemClient::list_from`] to get the list of log files. fn parsed_log_files_iter( @@ -282,7 +283,7 @@ fn parsed_log_files_iter( .filter_map_ok(identity) .take_while(move |x| match x { Ok(x) => lte_end_version(x.version), - _ => false, + Err(_) => true, })) } /// List all commit and checkpoint files with versions above the provided `start_version` (inclusive). From 8cbf2ff3d323c4fb326578a2f955dd6a364fe5a6 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 17:16:14 -0800 Subject: [PATCH 69/88] Add doc comment explaining properties --- kernel/src/log_segment.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index c93ba1d8c..47013d0ac 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -154,6 +154,10 @@ impl LogSegment { .filter_ok(|x| x.is_commit()) .try_collect()?; + // - Here check that the start version is correct. + // - [`LogSegment::try_new`] will verify that the `end_version` is correct if present. + // - [`LogSegment::try_new`] also checks that there are no gaps between commits. + // If all three are satisfied, this implies that all the desired commits are present. require!( sorted_commit_files .first() From a2ef01433933e39ea7606967a5c75bae212fdf2b Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 17:27:54 -0800 Subject: [PATCH 70/88] Fix more docs --- kernel/src/log_segment.rs | 27 +++++++++++++++------------ 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 47013d0ac..9fc0e708a 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -16,16 +16,15 @@ use std::sync::{Arc, LazyLock}; use tracing::warn; use url::Url; -/// A [`LogSegment`] represents a contiguous section of the log and is made up of checkpoint files -/// and commit files. It is built with either [`LogSegment::for_table_changes`], or -/// [`LogSegment::for_snapshot`], and guarantees the following: +/// A [`LogSegment`] represents a contiguous section of the log and is made of checkpoint files +/// and commit files. , and guarantees the following: /// 1. Commit file versions will not have any gaps between them. /// 2. If checkpoint(s) is/are present in the range, only commits with versions greater than the most /// recent checkpoint version are retained. There will not be a gap between the checkpoint /// version and the first commit version. /// -/// [`LogSegment`] is used in both [`Snapshot`] and in `TableChanges` to hold commit files and -/// checkpoint files. +/// [`LogSegment`] is used in [`Snapshot`] when built with [`LogSegment::for_snapshot`], and +/// and in `TableChanges` when built with [`LogSegment::for_table_changes`]. /// /// [`Snapshot`]: crate::snapshot::Snapshot #[derive(Debug)] @@ -47,7 +46,6 @@ impl LogSegment { expected_end_version: Option, ) -> DeltaResult { // We require that commits that are contiguous. In other words, there must be no gap between commit versions. - // There must also be no gap between a checkpoint and the first commit version. require!( sorted_commit_files .windows(2) @@ -57,6 +55,9 @@ impl LogSegment { sorted_commit_files )) ); + + // There must be no gap between a checkpoint and the first commit version. Note that + // that all checkpoint parts share the same version. if let (Some(checkpoint_file), Some(commit_file)) = (checkpoint_parts.first(), sorted_commit_files.first()) { @@ -89,11 +90,13 @@ impl LogSegment { } /// Constructs a [`LogSegment`] to be used for Snapshot. For a Snapshot at version `n`: - /// Its LogSegment is made up of zero or one checkpoint, and all commits between the checkpoint up - /// to and including the end version `n`. Note that a checkpoint may be made up of multiple - /// parts. + /// Its LogSegment is made of zero or one checkpoint, and all commits between the checkpoint up + /// to and including the end version `n`. Note that a checkpoint may be made of multiple + /// parts. All these parts will have the same checkpoint version. /// - /// This may leverage a `checkpoint_hint` that is read from `_delta_log/_last_checkpoint`. + /// The options for constructing a LogSegment for Snapshot are as follows: + /// - `checkpoint_hint`: a `CheckpointMetadata` to start the log segment from (e.g. from reading the `last_checkpoint` file). + /// - `time_travel_version`: The version of the log that the Snapshot will be at. #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) fn for_snapshot( fs_client: &dyn FileSystemClient, @@ -129,7 +132,7 @@ impl LogSegment { } /// Constructs a [`LogSegment`] to be used for `TableChanges`. For a TableChanges between versions - /// `start_version` and `end_version`: Its LogSegment is made up of zero checkpoints and all commits + /// `start_version` and `end_version`: Its LogSegment is made of zero checkpoints and all commits /// between versions `start_version` (inclusive) and `end_version` (inclusive). If no `end_version` /// is specified it will be the most recent version by default. #[allow(unused)] @@ -632,7 +635,7 @@ mod tests { #[test] fn build_snapshot_with_missing_checkpoint_part_no_hint() { // TODO(Oussam): Hande checkpoints correctly so that this test passes - // Part 2 of 3 is missing from checkpoint 5. The Snapshot should be made up of checkpoint + // Part 2 of 3 is missing from checkpoint 5. The Snapshot should be made of checkpoint // number 3 and commit files 4 to 7. let (client, table_root) = build_log_with_paths_and_checkpoint( &[ From 39955799d6b858a6391a540ef7dd126880972364 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 17:34:53 -0800 Subject: [PATCH 71/88] Remove expected_end_version from try_new --- kernel/src/log_segment.rs | 45 ++++++++++++++++++++++++--------------- 1 file changed, 28 insertions(+), 17 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 9fc0e708a..68a7bf28d 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -43,7 +43,6 @@ impl LogSegment { sorted_commit_files: Vec, checkpoint_parts: Vec, log_root: Url, - expected_end_version: Option, ) -> DeltaResult { // We require that commits that are contiguous. In other words, there must be no gap between commit versions. require!( @@ -71,15 +70,6 @@ impl LogSegment { .or(checkpoint_parts.first()) .ok_or(Error::MissingVersion)? // TODO: A more descriptive error .version; - if let Some(end_version) = expected_end_version { - require!( - version_eff == end_version, - Error::generic(format!( - "version effective not the same as end_version {}, {}", - version_eff, end_version - )) - ); - } Ok(LogSegment { end_version: version_eff, @@ -123,12 +113,20 @@ impl LogSegment { sorted_commit_files.retain(|log_path| checkpoint_file.version < log_path.version); } - LogSegment::try_new( - sorted_commit_files, - checkpoint_parts, - log_root, - time_travel_version, - ) + let log_segment = LogSegment::try_new(sorted_commit_files, checkpoint_parts, log_root)?; + + // Check that the effective version is the same as the time_travel_version we specified + if let Some(time_travel_version) = time_travel_version { + require!( + log_segment.end_version == time_travel_version, + Error::generic(format!( + "LogSegment end version {} not the same as the specified time_travel_version {}", + log_segment.end_version, time_travel_version + )) + ); + } + + Ok(log_segment) } /// Constructs a [`LogSegment`] to be used for `TableChanges`. For a TableChanges between versions @@ -170,7 +168,20 @@ impl LogSegment { start_version )) ); - LogSegment::try_new(sorted_commit_files, vec![], log_root, end_version) + let log_segment = LogSegment::try_new(sorted_commit_files, vec![], log_root)?; + + // Check that the effective version is the same as the time_travel_version we specified + if let Some(end_version) = end_version { + require!( + log_segment.end_version == end_version, + Error::generic(format!( + "LogSegment end version {} not the same as specified end version {}", + log_segment.end_version, end_version + )) + ); + } + + Ok(log_segment) } /// Read a stream of log data from this log segment. /// From fc296a4df317051665bd54c4595143ba4c03104b Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 17:36:05 -0800 Subject: [PATCH 72/88] Fix spacing --- kernel/src/log_segment.rs | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 68a7bf28d..6d4423cbb 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -115,7 +115,6 @@ impl LogSegment { let log_segment = LogSegment::try_new(sorted_commit_files, checkpoint_parts, log_root)?; - // Check that the effective version is the same as the time_travel_version we specified if let Some(time_travel_version) = time_travel_version { require!( log_segment.end_version == time_travel_version, @@ -170,7 +169,6 @@ impl LogSegment { ); let log_segment = LogSegment::try_new(sorted_commit_files, vec![], log_root)?; - // Check that the effective version is the same as the time_travel_version we specified if let Some(end_version) = end_version { require!( log_segment.end_version == end_version, @@ -372,10 +370,10 @@ fn list_log_files_with_checkpoint( }; if latest_checkpoint.version != checkpoint_metadata.version { warn!( - "_last_checkpoint hint is out of date. _last_checkpoint version: {}. Using actual most recent: {}", - checkpoint_metadata.version, - latest_checkpoint.version - ); + "_last_checkpoint hint is out of date. _last_checkpoint version: {}. Using actual most recent: {}", + checkpoint_metadata.version, + latest_checkpoint.version + ); } if checkpoint_parts.len() != checkpoint_metadata.parts.unwrap_or(1) { return Err(Error::Generic(format!( From 6926857cec1507111e2c88deea962de8d923b13a Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 17:42:19 -0800 Subject: [PATCH 73/88] change test util name --- kernel/src/log_segment.rs | 20 ++++++++++---------- test-utils/src/lib.rs | 4 ++-- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 6d4423cbb..b0b9733cc 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -399,7 +399,7 @@ mod tests { use crate::log_segment::LogSegment; use crate::snapshot::CheckpointMetadata; use crate::{FileSystemClient, Table}; - use test_utils::{delta_path_for_checkpoint_part, delta_path_for_version}; + use test_utils::{delta_path_for_multipart_checkpoint, delta_path_for_version}; // NOTE: In addition to testing the meta-predicate for metadata replay, this test also verifies // that the parquet reader properly infers nullcount = rowcount for missing columns. The two @@ -547,9 +547,9 @@ mod tests { delta_path_for_version(3, "checkpoint.parquet"), delta_path_for_version(3, "json"), delta_path_for_version(4, "json"), - delta_path_for_checkpoint_part(5, 1, 3), - delta_path_for_checkpoint_part(5, 2, 3), - delta_path_for_checkpoint_part(5, 3, 3), + delta_path_for_multipart_checkpoint(5, 1, 3), + delta_path_for_multipart_checkpoint(5, 2, 3), + delta_path_for_multipart_checkpoint(5, 3, 3), delta_path_for_version(5, "json"), delta_path_for_version(6, "json"), delta_path_for_version(7, "json"), @@ -591,9 +591,9 @@ mod tests { delta_path_for_version(3, "checkpoint.parquet"), delta_path_for_version(3, "json"), delta_path_for_version(4, "json"), - delta_path_for_checkpoint_part(5, 1, 3), + delta_path_for_multipart_checkpoint(5, 1, 3), // Part 2 is missing! - delta_path_for_checkpoint_part(5, 3, 3), + delta_path_for_multipart_checkpoint(5, 3, 3), delta_path_for_version(5, "json"), delta_path_for_version(6, "json"), delta_path_for_version(7, "json"), @@ -626,8 +626,8 @@ mod tests { delta_path_for_version(3, "checkpoint.parquet"), delta_path_for_version(3, "json"), delta_path_for_version(4, "json"), - delta_path_for_checkpoint_part(5, 1, 2), - delta_path_for_checkpoint_part(5, 2, 2), + delta_path_for_multipart_checkpoint(5, 1, 2), + delta_path_for_multipart_checkpoint(5, 2, 2), delta_path_for_version(5, "json"), delta_path_for_version(6, "json"), delta_path_for_version(7, "json"), @@ -655,9 +655,9 @@ mod tests { delta_path_for_version(3, "checkpoint.parquet"), delta_path_for_version(3, "json"), delta_path_for_version(4, "json"), - delta_path_for_checkpoint_part(5, 1, 3), + delta_path_for_multipart_checkpoint(5, 1, 3), // Part 2 is missing! - delta_path_for_checkpoint_part(5, 3, 3), + delta_path_for_multipart_checkpoint(5, 3, 3), delta_path_for_version(5, "json"), delta_path_for_version(6, "json"), delta_path_for_version(7, "json"), diff --git a/test-utils/src/lib.rs b/test-utils/src/lib.rs index 074ad570e..b7ff42d22 100644 --- a/test-utils/src/lib.rs +++ b/test-utils/src/lib.rs @@ -89,9 +89,9 @@ pub fn delta_path_for_version(version: u64, suffix: &str) -> Path { } // get an ObjectStore path for a checkpoint file, based on version, part number, and total number of parts -pub fn delta_path_for_checkpoint_part(version: u64, part_number: u64, total_parts: u64) -> Path { +pub fn delta_path_for_multipart_checkpoint(version: u64, part_num: u64, num_parts: u64) -> Path { let path = - format!("_delta_log/{version:020}.checkpoint.{part_number:010}.{total_parts:010}.parquet"); + format!("_delta_log/{version:020}.checkpoint.{part_num:010}.{num_parts:010}.parquet"); Path::from(path.as_str()) } /// put a commit file into the specified object store. From e8f34058cdbc4302659e2269401332223ec12e6c Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 20:47:45 -0800 Subject: [PATCH 74/88] Apply suggestions from code review Co-authored-by: Zach Schuermann --- kernel/src/log_segment.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index b0b9733cc..df9887a00 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -17,7 +17,7 @@ use tracing::warn; use url::Url; /// A [`LogSegment`] represents a contiguous section of the log and is made of checkpoint files -/// and commit files. , and guarantees the following: +/// and commit files and guarantees the following: /// 1. Commit file versions will not have any gaps between them. /// 2. If checkpoint(s) is/are present in the range, only commits with versions greater than the most /// recent checkpoint version are retained. There will not be a gap between the checkpoint @@ -32,8 +32,8 @@ use url::Url; pub(crate) struct LogSegment { pub end_version: Version, pub log_root: Url, - /// Sorted commit files in the log segment - pub commit_files: Vec, + /// Sorted commit files in the log segment (ascending) + pub sorted_commit_files: Vec, /// Checkpoint files in the log segment. pub checkpoint_parts: Vec, } @@ -143,9 +143,9 @@ impl LogSegment { let log_root = table_root.join("_delta_log/").unwrap(); let end_version = end_version.into(); - if let (start_version, Some(end_version)) = (start_version, end_version) { + if let Some(end_version) = end_version { if start_version > end_version { - return Err(Error::generic("Failed to build LogSegment: `start_version` cannot be greater than end_version")); + return Err(Error::generic("Failed to build LogSegment: start_version cannot be greater than end_version")); } } @@ -275,7 +275,7 @@ impl LogSegment { /// the most recent version will be included. /// /// Note: this calls [`FileSystemClient::list_from`] to get the list of log files. -fn parsed_log_files_iter( +fn list_log_files( fs_client: &dyn FileSystemClient, log_root: &Url, start_version: impl Into>, From f31ec8a2ad7e0b4eccf7fe464baaa93897c49537 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 20:57:12 -0800 Subject: [PATCH 75/88] Address pr reviews --- kernel/src/log_segment.rs | 114 +++++++++++++++++++++----------------- kernel/src/snapshot.rs | 4 +- 2 files changed, 64 insertions(+), 54 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index df9887a00..2cdf0cb1c 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -43,6 +43,7 @@ impl LogSegment { sorted_commit_files: Vec, checkpoint_parts: Vec, log_root: Url, + end_version: Option, ) -> DeltaResult { // We require that commits that are contiguous. In other words, there must be no gap between commit versions. require!( @@ -70,11 +71,19 @@ impl LogSegment { .or(checkpoint_parts.first()) .ok_or(Error::MissingVersion)? // TODO: A more descriptive error .version; - + if let Some(end_version) = end_version { + require!( + version_eff == end_version, + Error::generic(format!( + "LogSegment end version {} not the same as the specified end version {}", + version_eff, end_version + )) + ); + } Ok(LogSegment { end_version: version_eff, log_root, - commit_files: sorted_commit_files, + sorted_commit_files, checkpoint_parts, }) } @@ -113,19 +122,12 @@ impl LogSegment { sorted_commit_files.retain(|log_path| checkpoint_file.version < log_path.version); } - let log_segment = LogSegment::try_new(sorted_commit_files, checkpoint_parts, log_root)?; - - if let Some(time_travel_version) = time_travel_version { - require!( - log_segment.end_version == time_travel_version, - Error::generic(format!( - "LogSegment end version {} not the same as the specified time_travel_version {}", - log_segment.end_version, time_travel_version - )) - ); - } - - Ok(log_segment) + LogSegment::try_new( + sorted_commit_files, + checkpoint_parts, + log_root, + time_travel_version, + ) } /// Constructs a [`LogSegment`] to be used for `TableChanges`. For a TableChanges between versions @@ -145,12 +147,14 @@ impl LogSegment { let end_version = end_version.into(); if let Some(end_version) = end_version { if start_version > end_version { - return Err(Error::generic("Failed to build LogSegment: start_version cannot be greater than end_version")); + return Err(Error::generic( + "Failed to build LogSegment: start_version cannot be greater than end_version", + )); } } let sorted_commit_files: Vec<_> = - parsed_log_files_iter(fs_client, &log_root, start_version, end_version)? + list_log_files(fs_client, &log_root, start_version, end_version)? .filter_ok(|x| x.is_commit()) .try_collect()?; @@ -167,19 +171,7 @@ impl LogSegment { start_version )) ); - let log_segment = LogSegment::try_new(sorted_commit_files, vec![], log_root)?; - - if let Some(end_version) = end_version { - require!( - log_segment.end_version == end_version, - Error::generic(format!( - "LogSegment end version {} not the same as specified end version {}", - log_segment.end_version, end_version - )) - ); - } - - Ok(log_segment) + LogSegment::try_new(sorted_commit_files, vec![], log_root, end_version) } /// Read a stream of log data from this log segment. /// @@ -203,7 +195,7 @@ impl LogSegment { // `replay` expects commit files to be sorted in descending order, so we reverse the sorted // commit files let commit_files: Vec<_> = self - .commit_files + .sorted_commit_files .iter() .rev() .map(|f| f.location.clone()) @@ -319,7 +311,7 @@ fn list_log_files_with_version( // on config at some point let mut commit_files = Vec::with_capacity(10); - for parsed_path in parsed_log_files_iter(fs_client, log_root, start_version, end_version)? { + for parsed_path in list_log_files(fs_client, log_root, start_version, end_version)? { let parsed_path = parsed_path?; if parsed_path.is_commit() { commit_files.push(parsed_path); @@ -517,8 +509,10 @@ mod tests { let log_segment = LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None) .unwrap(); - let (commit_files, checkpoint_parts) = - (log_segment.commit_files, log_segment.checkpoint_parts); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); assert_eq!(checkpoint_parts.len(), 1); assert_eq!(commit_files.len(), 2); @@ -560,8 +554,10 @@ mod tests { let log_segment = LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None) .unwrap(); - let (commit_files, checkpoint_parts) = - (log_segment.commit_files, log_segment.checkpoint_parts); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); assert_eq!(checkpoint_parts.len(), 3); assert_eq!(commit_files.len(), 2); @@ -668,8 +664,10 @@ mod tests { let log_segment = LogSegment::for_snapshot(client.as_ref(), &table_root, None, None).unwrap(); - let (commit_files, checkpoint_parts) = - (log_segment.commit_files, log_segment.checkpoint_parts); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); assert_eq!(checkpoint_parts.len(), 1); assert_eq!(checkpoint_parts[0].version, 3); @@ -703,8 +701,10 @@ mod tests { // -------------------------------------------------------------------------------- let log_segment = LogSegment::for_snapshot(client.as_ref(), &table_root, None, None).unwrap(); - let (commit_files, checkpoint_parts) = - (log_segment.commit_files, log_segment.checkpoint_parts); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); assert_eq!(checkpoint_parts.len(), 1); assert_eq!(checkpoint_parts[0].version, 5); @@ -719,8 +719,10 @@ mod tests { // -------------------------------------------------------------------------------- let log_segment = LogSegment::for_snapshot(client.as_ref(), &table_root, None, Some(2)).unwrap(); - let (commit_files, checkpoint_parts) = - (log_segment.commit_files, log_segment.checkpoint_parts); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); assert_eq!(checkpoint_parts.len(), 1); assert_eq!(checkpoint_parts[0].version, 1); @@ -762,8 +764,10 @@ mod tests { let log_segment = LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, Some(4)) .unwrap(); - let (commit_files, checkpoint_parts) = - (log_segment.commit_files, log_segment.checkpoint_parts); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); assert_eq!(checkpoint_parts.len(), 1); assert_eq!(checkpoint_parts[0].version, 3); @@ -803,8 +807,8 @@ mod tests { .unwrap(); assert_eq!(log_segment.checkpoint_parts[0].version, 3); - assert_eq!(log_segment.commit_files.len(), 1); - assert_eq!(log_segment.commit_files[0].version, 4); + assert_eq!(log_segment.sorted_commit_files.len(), 1); + assert_eq!(log_segment.sorted_commit_files[0].version, 4); } #[test] fn build_table_changes_with_commit_versions() { @@ -831,8 +835,10 @@ mod tests { let log_segment = LogSegment::for_table_changes(client.as_ref(), &table_root, 2, 5).unwrap(); - let (commit_files, checkpoint_parts) = - (log_segment.commit_files, log_segment.checkpoint_parts); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); // Checkpoints should be omitted assert_eq!(checkpoint_parts.len(), 0); @@ -848,8 +854,10 @@ mod tests { let log_segment = LogSegment::for_table_changes(client.as_ref(), &table_root, 0, Some(0)).unwrap(); - let (commit_files, checkpoint_parts) = - (log_segment.commit_files, log_segment.checkpoint_parts); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); // Checkpoints should be omitted assert_eq!(checkpoint_parts.len(), 0); @@ -862,8 +870,10 @@ mod tests { // -------------------------------------------------------------------------------- let log_segment = LogSegment::for_table_changes(client.as_ref(), &table_root, 0, None).unwrap(); - let (commit_files, checkpoint_parts) = - (log_segment.commit_files, log_segment.checkpoint_parts); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); // Checkpoints should be omitted assert_eq!(checkpoint_parts.len(), 0); diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index a1a8a4026..ad0ae1722 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -319,9 +319,9 @@ mod tests { .version, 2, ); - assert_eq!(snapshot.log_segment.commit_files.len(), 1); + assert_eq!(snapshot.log_segment.sorted_commit_files.len(), 1); assert_eq!( - ParsedLogPath::try_from(snapshot.log_segment.commit_files[0].location.clone()) + ParsedLogPath::try_from(snapshot.log_segment.sorted_commit_files[0].location.clone()) .unwrap() .unwrap() .version, From 634661a1bb74c433f87f7ce4de6a3670795cbaf1 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 23:23:10 -0800 Subject: [PATCH 76/88] Move log segment tests to separate file --- kernel/src/log_segment.rs | 923 -------------------------------- kernel/src/log_segment/mod.rs | 378 +++++++++++++ kernel/src/log_segment/tests.rs | 533 ++++++++++++++++++ 3 files changed, 911 insertions(+), 923 deletions(-) delete mode 100644 kernel/src/log_segment.rs create mode 100644 kernel/src/log_segment/mod.rs create mode 100644 kernel/src/log_segment/tests.rs diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs deleted file mode 100644 index 2cdf0cb1c..000000000 --- a/kernel/src/log_segment.rs +++ /dev/null @@ -1,923 +0,0 @@ -//! Represents a segment of a delta log. [`LogSegment`] wraps a set of checkpoint and commit -//! files. - -use crate::actions::{get_log_schema, Metadata, Protocol, METADATA_NAME, PROTOCOL_NAME}; -use crate::path::ParsedLogPath; -use crate::schema::SchemaRef; -use crate::snapshot::CheckpointMetadata; -use crate::utils::require; -use crate::{ - DeltaResult, Engine, EngineData, Error, Expression, ExpressionRef, FileSystemClient, Version, -}; -use itertools::Itertools; -use std::cmp::Ordering; -use std::convert::identity; -use std::sync::{Arc, LazyLock}; -use tracing::warn; -use url::Url; - -/// A [`LogSegment`] represents a contiguous section of the log and is made of checkpoint files -/// and commit files and guarantees the following: -/// 1. Commit file versions will not have any gaps between them. -/// 2. If checkpoint(s) is/are present in the range, only commits with versions greater than the most -/// recent checkpoint version are retained. There will not be a gap between the checkpoint -/// version and the first commit version. -/// -/// [`LogSegment`] is used in [`Snapshot`] when built with [`LogSegment::for_snapshot`], and -/// and in `TableChanges` when built with [`LogSegment::for_table_changes`]. -/// -/// [`Snapshot`]: crate::snapshot::Snapshot -#[derive(Debug)] -#[cfg_attr(feature = "developer-visibility", visibility::make(pub))] -pub(crate) struct LogSegment { - pub end_version: Version, - pub log_root: Url, - /// Sorted commit files in the log segment (ascending) - pub sorted_commit_files: Vec, - /// Checkpoint files in the log segment. - pub checkpoint_parts: Vec, -} - -impl LogSegment { - fn try_new( - sorted_commit_files: Vec, - checkpoint_parts: Vec, - log_root: Url, - end_version: Option, - ) -> DeltaResult { - // We require that commits that are contiguous. In other words, there must be no gap between commit versions. - require!( - sorted_commit_files - .windows(2) - .all(|cfs| cfs[0].version + 1 == cfs[1].version), - Error::generic(format!( - "Expected ordered contiguous commit files {:?}", - sorted_commit_files - )) - ); - - // There must be no gap between a checkpoint and the first commit version. Note that - // that all checkpoint parts share the same version. - if let (Some(checkpoint_file), Some(commit_file)) = - (checkpoint_parts.first(), sorted_commit_files.first()) - { - require!(checkpoint_file.version + 1 == commit_file.version, - Error::generic(format!("Expected commit file version {} to be next version to checkpoint file version {}", commit_file.version, checkpoint_file.version ))) - } - - // Get the effective version from chosen files - let version_eff = sorted_commit_files - .last() - .or(checkpoint_parts.first()) - .ok_or(Error::MissingVersion)? // TODO: A more descriptive error - .version; - if let Some(end_version) = end_version { - require!( - version_eff == end_version, - Error::generic(format!( - "LogSegment end version {} not the same as the specified end version {}", - version_eff, end_version - )) - ); - } - Ok(LogSegment { - end_version: version_eff, - log_root, - sorted_commit_files, - checkpoint_parts, - }) - } - - /// Constructs a [`LogSegment`] to be used for Snapshot. For a Snapshot at version `n`: - /// Its LogSegment is made of zero or one checkpoint, and all commits between the checkpoint up - /// to and including the end version `n`. Note that a checkpoint may be made of multiple - /// parts. All these parts will have the same checkpoint version. - /// - /// The options for constructing a LogSegment for Snapshot are as follows: - /// - `checkpoint_hint`: a `CheckpointMetadata` to start the log segment from (e.g. from reading the `last_checkpoint` file). - /// - `time_travel_version`: The version of the log that the Snapshot will be at. - #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] - pub(crate) fn for_snapshot( - fs_client: &dyn FileSystemClient, - table_root: &Url, - checkpoint_hint: impl Into>, - time_travel_version: impl Into>, - ) -> DeltaResult { - let time_travel_version = time_travel_version.into(); - let log_root = table_root.join("_delta_log/").unwrap(); - - let (mut sorted_commit_files, checkpoint_parts) = - match (checkpoint_hint.into(), time_travel_version) { - (Some(cp), None) => { - list_log_files_with_checkpoint(&cp, fs_client, &log_root, None)? - } - (Some(cp), Some(end_version)) if cp.version <= end_version => { - list_log_files_with_checkpoint(&cp, fs_client, &log_root, Some(end_version))? - } - _ => list_log_files_with_version(fs_client, &log_root, None, time_travel_version)?, - }; - - // Commit file versions must be greater than the most recent checkpoint version if it exists - if let Some(checkpoint_file) = checkpoint_parts.first() { - sorted_commit_files.retain(|log_path| checkpoint_file.version < log_path.version); - } - - LogSegment::try_new( - sorted_commit_files, - checkpoint_parts, - log_root, - time_travel_version, - ) - } - - /// Constructs a [`LogSegment`] to be used for `TableChanges`. For a TableChanges between versions - /// `start_version` and `end_version`: Its LogSegment is made of zero checkpoints and all commits - /// between versions `start_version` (inclusive) and `end_version` (inclusive). If no `end_version` - /// is specified it will be the most recent version by default. - #[allow(unused)] - #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] - pub(crate) fn for_table_changes( - fs_client: &dyn FileSystemClient, - table_root: &Url, - start_version: Version, - end_version: impl Into>, - ) -> DeltaResult { - let log_root = table_root.join("_delta_log/").unwrap(); - - let end_version = end_version.into(); - if let Some(end_version) = end_version { - if start_version > end_version { - return Err(Error::generic( - "Failed to build LogSegment: start_version cannot be greater than end_version", - )); - } - } - - let sorted_commit_files: Vec<_> = - list_log_files(fs_client, &log_root, start_version, end_version)? - .filter_ok(|x| x.is_commit()) - .try_collect()?; - - // - Here check that the start version is correct. - // - [`LogSegment::try_new`] will verify that the `end_version` is correct if present. - // - [`LogSegment::try_new`] also checks that there are no gaps between commits. - // If all three are satisfied, this implies that all the desired commits are present. - require!( - sorted_commit_files - .first() - .is_some_and(|first_commit| first_commit.version == start_version), - Error::generic(format!( - "Expected the first commit to have version {}", - start_version - )) - ); - LogSegment::try_new(sorted_commit_files, vec![], log_root, end_version) - } - /// Read a stream of log data from this log segment. - /// - /// The log files will be read from most recent to oldest. - /// The boolean flags indicates whether the data was read from - /// a commit file (true) or a checkpoint file (false). - /// - /// `read_schema` is the schema to read the log files with. This can be used - /// to project the log files to a subset of the columns. - /// - /// `meta_predicate` is an optional expression to filter the log files with. It is _NOT_ the - /// query's predicate, but rather a predicate for filtering log files themselves. - #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] - pub(crate) fn replay( - &self, - engine: &dyn Engine, - commit_read_schema: SchemaRef, - checkpoint_read_schema: SchemaRef, - meta_predicate: Option, - ) -> DeltaResult, bool)>> + Send> { - // `replay` expects commit files to be sorted in descending order, so we reverse the sorted - // commit files - let commit_files: Vec<_> = self - .sorted_commit_files - .iter() - .rev() - .map(|f| f.location.clone()) - .collect(); - let commit_stream = engine - .get_json_handler() - .read_json_files(&commit_files, commit_read_schema, meta_predicate.clone())? - .map_ok(|batch| (batch, true)); - - let checkpoint_parts: Vec<_> = self - .checkpoint_parts - .iter() - .map(|f| f.location.clone()) - .collect(); - let checkpoint_stream = engine - .get_parquet_handler() - .read_parquet_files(&checkpoint_parts, checkpoint_read_schema, meta_predicate)? - .map_ok(|batch| (batch, false)); - - Ok(commit_stream.chain(checkpoint_stream)) - } - - // Get the most up-to-date Protocol and Metadata actions - pub(crate) fn read_metadata(&self, engine: &dyn Engine) -> DeltaResult<(Metadata, Protocol)> { - let data_batches = self.replay_for_metadata(engine)?; - let (mut metadata_opt, mut protocol_opt) = (None, None); - for batch in data_batches { - let (batch, _) = batch?; - if metadata_opt.is_none() { - metadata_opt = Metadata::try_new_from_data(batch.as_ref())?; - } - if protocol_opt.is_none() { - protocol_opt = Protocol::try_new_from_data(batch.as_ref())?; - } - if metadata_opt.is_some() && protocol_opt.is_some() { - // we've found both, we can stop - break; - } - } - match (metadata_opt, protocol_opt) { - (Some(m), Some(p)) => Ok((m, p)), - (None, Some(_)) => Err(Error::MissingMetadata), - (Some(_), None) => Err(Error::MissingProtocol), - (None, None) => Err(Error::MissingMetadataAndProtocol), - } - } - - // Replay the commit log, projecting rows to only contain Protocol and Metadata action columns. - fn replay_for_metadata( - &self, - engine: &dyn Engine, - ) -> DeltaResult, bool)>> + Send> { - let schema = get_log_schema().project(&[PROTOCOL_NAME, METADATA_NAME])?; - // filter out log files that do not contain metadata or protocol information - static META_PREDICATE: LazyLock> = LazyLock::new(|| { - Some(Arc::new(Expression::or( - Expression::column([METADATA_NAME, "id"]).is_not_null(), - Expression::column([PROTOCOL_NAME, "minReaderVersion"]).is_not_null(), - ))) - }); - // read the same protocol and metadata schema for both commits and checkpoints - self.replay(engine, schema.clone(), schema, META_PREDICATE.clone()) - } -} - -/// Returns a fallible iterator of [`ParsedLogPath`] that are between the provided `start_version` (inclusive) -/// and `end_version` (inclusive). [`ParsedLogPath`] may be a commit or a checkpoint. If `start_version` is -/// not specified, the files will begin from version number 0. If `end_version` is not specified, files up to -/// the most recent version will be included. -/// -/// Note: this calls [`FileSystemClient::list_from`] to get the list of log files. -fn list_log_files( - fs_client: &dyn FileSystemClient, - log_root: &Url, - start_version: impl Into>, - end_version: impl Into>, -) -> DeltaResult>> { - let start_version = start_version.into().unwrap_or(0); - let end_version = end_version.into(); - - let version_prefix = format!("{:020}", start_version); - let start_from = log_root.join(&version_prefix)?; - - // Check that the provided version is less than or equal to the end version if it exists - let lte_end_version = move |version: u64| { - end_version.is_none() || end_version.is_some_and(|end_version| version <= end_version) - }; - - Ok(fs_client - .list_from(&start_from)? - .map(|meta| ParsedLogPath::try_from(meta?)) - // TODO this filters out .crc files etc which start with "." - how do we want to use these kind of files? - .filter_map_ok(identity) - .take_while(move |x| match x { - Ok(x) => lte_end_version(x.version), - Err(_) => true, - })) -} -/// List all commit and checkpoint files with versions above the provided `start_version` (inclusive). -/// If successful, this returns a tuple `(sorted_commit_files_paths, checkpoint_parts)` of type -/// `(Vec, Vec)`. The commit files are guaranteed to be sorted in -/// ascending order by version. The elements of `checkpoint_parts` are all the parts of the same -/// checkpoint. Checkpoint parts share the same version. -fn list_log_files_with_version( - fs_client: &dyn FileSystemClient, - log_root: &Url, - start_version: Option, - end_version: Option, -) -> DeltaResult<(Vec, Vec)> { - let mut max_checkpoint_version = start_version; - let mut checkpoint_parts = vec![]; - // We expect 10 commit files per checkpoint, so start with that size. We could adjust this based - // on config at some point - let mut commit_files = Vec::with_capacity(10); - - for parsed_path in list_log_files(fs_client, log_root, start_version, end_version)? { - let parsed_path = parsed_path?; - if parsed_path.is_commit() { - commit_files.push(parsed_path); - } else if parsed_path.is_checkpoint() { - let path_version = parsed_path.version; - match max_checkpoint_version { - None => { - checkpoint_parts.push(parsed_path); - max_checkpoint_version = Some(path_version); - } - Some(checkpoint_version) => match path_version.cmp(&checkpoint_version) { - Ordering::Greater => { - max_checkpoint_version = Some(path_version); - checkpoint_parts.clear(); - checkpoint_parts.push(parsed_path); - } - Ordering::Equal => checkpoint_parts.push(parsed_path), - Ordering::Less => {} - }, - } - } - } - - Ok((commit_files, checkpoint_parts)) -} - -/// List all commit and checkpoint files after the provided checkpoint. It is guaranteed that all -/// the returned [`ParsedLogPath`]s will have a version less than or equal to the `end_version`. -/// See [`list_log_files_with_version`] for details on the return type. -fn list_log_files_with_checkpoint( - checkpoint_metadata: &CheckpointMetadata, - fs_client: &dyn FileSystemClient, - log_root: &Url, - end_version: Option, -) -> DeltaResult<(Vec, Vec)> { - let (commit_files, checkpoint_parts) = list_log_files_with_version( - fs_client, - log_root, - Some(checkpoint_metadata.version), - end_version, - )?; - - let Some(latest_checkpoint) = checkpoint_parts.last() else { - // TODO: We could potentially recover here - return Err(Error::generic( - "Had a _last_checkpoint hint but didn't find any checkpoints", - )); - }; - if latest_checkpoint.version != checkpoint_metadata.version { - warn!( - "_last_checkpoint hint is out of date. _last_checkpoint version: {}. Using actual most recent: {}", - checkpoint_metadata.version, - latest_checkpoint.version - ); - } - if checkpoint_parts.len() != checkpoint_metadata.parts.unwrap_or(1) { - return Err(Error::Generic(format!( - "_last_checkpoint indicated that checkpoint should have {} parts, but it has {}", - checkpoint_metadata.parts.unwrap_or(1), - checkpoint_parts.len() - ))); - } - Ok((commit_files, checkpoint_parts)) -} - -#[cfg(test)] -mod tests { - use std::{path::PathBuf, sync::Arc}; - - use itertools::Itertools; - use object_store::{memory::InMemory, path::Path, ObjectStore}; - use url::Url; - - use crate::engine::default::executor::tokio::TokioBackgroundExecutor; - use crate::engine::default::filesystem::ObjectStoreFileSystemClient; - use crate::engine::sync::SyncEngine; - use crate::log_segment::LogSegment; - use crate::snapshot::CheckpointMetadata; - use crate::{FileSystemClient, Table}; - use test_utils::{delta_path_for_multipart_checkpoint, delta_path_for_version}; - - // NOTE: In addition to testing the meta-predicate for metadata replay, this test also verifies - // that the parquet reader properly infers nullcount = rowcount for missing columns. The two - // checkpoint part files that contain transaction app ids have truncated schemas that would - // otherwise fail skipping due to their missing nullcount stat: - // - // Row group 0: count: 1 total(compressed): 111 B total(uncompressed):107 B - // -------------------------------------------------------------------------------- - // type nulls min / max - // txn.appId BINARY 0 "3ae45b72-24e1-865a-a211-3..." / "3ae45b72-24e1-865a-a211-3..." - // txn.version INT64 0 "4390" / "4390" - #[test] - fn test_replay_for_metadata() { - let path = std::fs::canonicalize(PathBuf::from("./tests/data/parquet_row_group_skipping/")); - let url = url::Url::from_directory_path(path.unwrap()).unwrap(); - let engine = SyncEngine::new(); - - let table = Table::new(url); - let snapshot = table.snapshot(&engine, None).unwrap(); - let data: Vec<_> = snapshot - .log_segment - .replay_for_metadata(&engine) - .unwrap() - .try_collect() - .unwrap(); - - // The checkpoint has five parts, each containing one action: - // 1. txn (physically missing P&M columns) - // 2. metaData - // 3. protocol - // 4. add - // 5. txn (physically missing P&M columns) - // - // The parquet reader should skip parts 1, 3, and 5. Note that the actual `read_metadata` - // always skips parts 4 and 5 because it terminates the iteration after finding both P&M. - // - // NOTE: Each checkpoint part is a single-row file -- guaranteed to produce one row group. - // - // WARNING: https://github.com/delta-incubator/delta-kernel-rs/issues/434 -- We currently - // read parts 1 and 5 (4 in all instead of 2) because row group skipping is disabled for - // missing columns, but can still skip part 3 because has valid nullcount stats for P&M. - assert_eq!(data.len(), 4); - } - - // Utility method to build a log using a list of log paths and an optional checkpoint hint. The - // CheckpointMetadata is written to `_delta_log/_last_checkpoint`. - fn build_log_with_paths_and_checkpoint( - paths: &[Path], - checkpoint_metadata: Option<&CheckpointMetadata>, - ) -> (Box, Url) { - let store = Arc::new(InMemory::new()); - - let data = bytes::Bytes::from("kernel-data"); - - // add log files to store - tokio::runtime::Runtime::new() - .expect("create tokio runtime") - .block_on(async { - for path in paths { - store - .put(path, data.clone().into()) - .await - .expect("put log file in store"); - } - if let Some(checkpoint_metadata) = checkpoint_metadata { - let checkpoint_str = - serde_json::to_string(checkpoint_metadata).expect("Serialize checkpoint"); - store - .put( - &Path::from("_delta_log/_last_checkpoint"), - checkpoint_str.into(), - ) - .await - .expect("Write _last_checkpoint"); - } - }); - - let client = ObjectStoreFileSystemClient::new( - store, - false, // don't have ordered listing - Path::from("/"), - Arc::new(TokioBackgroundExecutor::new()), - ); - - let table_root = Url::parse("memory:///").expect("valid url"); - (Box::new(client), table_root) - } - - #[test] - fn build_snapshot_with_out_of_date_last_checkpoint() { - let checkpoint_metadata = CheckpointMetadata { - version: 3, - size: 10, - parts: None, - size_in_bytes: None, - num_of_add_files: None, - checkpoint_schema: None, - checksum: None, - }; - - let (client, table_root) = build_log_with_paths_and_checkpoint( - &[ - delta_path_for_version(0, "json"), - delta_path_for_version(1, "checkpoint.parquet"), - delta_path_for_version(2, "json"), - delta_path_for_version(3, "checkpoint.parquet"), - delta_path_for_version(4, "json"), - delta_path_for_version(5, "checkpoint.parquet"), - delta_path_for_version(6, "json"), - delta_path_for_version(7, "json"), - ], - Some(&checkpoint_metadata), - ); - - let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None) - .unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); - - assert_eq!(checkpoint_parts.len(), 1); - assert_eq!(commit_files.len(), 2); - assert_eq!(checkpoint_parts[0].version, 5); - assert_eq!(commit_files[0].version, 6); - assert_eq!(commit_files[1].version, 7); - } - #[test] - fn build_snapshot_with_correct_last_multipart_checkpoint() { - let checkpoint_metadata = CheckpointMetadata { - version: 5, - size: 10, - parts: Some(3), - size_in_bytes: None, - num_of_add_files: None, - checkpoint_schema: None, - checksum: None, - }; - - let (client, table_root) = build_log_with_paths_and_checkpoint( - &[ - delta_path_for_version(0, "json"), - delta_path_for_version(1, "checkpoint.parquet"), - delta_path_for_version(1, "json"), - delta_path_for_version(2, "json"), - delta_path_for_version(3, "checkpoint.parquet"), - delta_path_for_version(3, "json"), - delta_path_for_version(4, "json"), - delta_path_for_multipart_checkpoint(5, 1, 3), - delta_path_for_multipart_checkpoint(5, 2, 3), - delta_path_for_multipart_checkpoint(5, 3, 3), - delta_path_for_version(5, "json"), - delta_path_for_version(6, "json"), - delta_path_for_version(7, "json"), - ], - Some(&checkpoint_metadata), - ); - - let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None) - .unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); - - assert_eq!(checkpoint_parts.len(), 3); - assert_eq!(commit_files.len(), 2); - assert_eq!(checkpoint_parts[0].version, 5); - assert_eq!(commit_files[0].version, 6); - assert_eq!(commit_files[1].version, 7); - } - - #[test] - fn build_snapshot_with_missing_checkpoint_part_from_hint_fails() { - let checkpoint_metadata = CheckpointMetadata { - version: 5, - size: 10, - parts: Some(3), - size_in_bytes: None, - num_of_add_files: None, - checkpoint_schema: None, - checksum: None, - }; - - let (client, table_root) = build_log_with_paths_and_checkpoint( - &[ - delta_path_for_version(0, "json"), - delta_path_for_version(1, "checkpoint.parquet"), - delta_path_for_version(1, "json"), - delta_path_for_version(2, "json"), - delta_path_for_version(3, "checkpoint.parquet"), - delta_path_for_version(3, "json"), - delta_path_for_version(4, "json"), - delta_path_for_multipart_checkpoint(5, 1, 3), - // Part 2 is missing! - delta_path_for_multipart_checkpoint(5, 3, 3), - delta_path_for_version(5, "json"), - delta_path_for_version(6, "json"), - delta_path_for_version(7, "json"), - ], - Some(&checkpoint_metadata), - ); - - let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None); - assert!(log_segment.is_err()) - } - #[test] - fn build_snapshot_with_bad_checkpoint_hint_fails() { - let checkpoint_metadata = CheckpointMetadata { - version: 5, - size: 10, - parts: Some(1), - size_in_bytes: None, - num_of_add_files: None, - checkpoint_schema: None, - checksum: None, - }; - - let (client, table_root) = build_log_with_paths_and_checkpoint( - &[ - delta_path_for_version(0, "json"), - delta_path_for_version(1, "checkpoint.parquet"), - delta_path_for_version(1, "json"), - delta_path_for_version(2, "json"), - delta_path_for_version(3, "checkpoint.parquet"), - delta_path_for_version(3, "json"), - delta_path_for_version(4, "json"), - delta_path_for_multipart_checkpoint(5, 1, 2), - delta_path_for_multipart_checkpoint(5, 2, 2), - delta_path_for_version(5, "json"), - delta_path_for_version(6, "json"), - delta_path_for_version(7, "json"), - ], - Some(&checkpoint_metadata), - ); - - let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None); - assert!(log_segment.is_err()) - } - - #[ignore] - #[test] - fn build_snapshot_with_missing_checkpoint_part_no_hint() { - // TODO(Oussam): Hande checkpoints correctly so that this test passes - // Part 2 of 3 is missing from checkpoint 5. The Snapshot should be made of checkpoint - // number 3 and commit files 4 to 7. - let (client, table_root) = build_log_with_paths_and_checkpoint( - &[ - delta_path_for_version(0, "json"), - delta_path_for_version(1, "checkpoint.parquet"), - delta_path_for_version(1, "json"), - delta_path_for_version(2, "json"), - delta_path_for_version(3, "checkpoint.parquet"), - delta_path_for_version(3, "json"), - delta_path_for_version(4, "json"), - delta_path_for_multipart_checkpoint(5, 1, 3), - // Part 2 is missing! - delta_path_for_multipart_checkpoint(5, 3, 3), - delta_path_for_version(5, "json"), - delta_path_for_version(6, "json"), - delta_path_for_version(7, "json"), - ], - None, - ); - - let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, None, None).unwrap(); - - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); - - assert_eq!(checkpoint_parts.len(), 1); - assert_eq!(checkpoint_parts[0].version, 3); - - let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); - let expected_versions = vec![4, 5, 6, 7]; - assert_eq!(versions, expected_versions); - } - - #[test] - fn build_snapshot_without_checkpoints() { - let (client, table_root) = build_log_with_paths_and_checkpoint( - &[ - delta_path_for_version(0, "json"), - delta_path_for_version(1, "json"), - delta_path_for_version(1, "checkpoint.parquet"), - delta_path_for_version(2, "json"), - delta_path_for_version(3, "json"), - delta_path_for_version(3, "checkpoint.parquet"), - delta_path_for_version(4, "json"), - delta_path_for_version(5, "json"), - delta_path_for_version(5, "checkpoint.parquet"), - delta_path_for_version(6, "json"), - delta_path_for_version(7, "json"), - ], - None, - ); - - // -------------------------------------------------------------------------------- - // | Specify no checkpoint or end version | - // -------------------------------------------------------------------------------- - let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, None, None).unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); - - assert_eq!(checkpoint_parts.len(), 1); - assert_eq!(checkpoint_parts[0].version, 5); - - // All commit files should still be there - let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); - let expected_versions = vec![6, 7]; - assert_eq!(versions, expected_versions); - - // -------------------------------------------------------------------------------- - // | Specify only end version | - // -------------------------------------------------------------------------------- - let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, None, Some(2)).unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); - - assert_eq!(checkpoint_parts.len(), 1); - assert_eq!(checkpoint_parts[0].version, 1); - - // All commit files should still be there - let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); - let expected_versions = vec![2]; - assert_eq!(versions, expected_versions); - } - - #[test] - fn build_snapshot_with_checkpoint_greater_than_time_travel_version() { - let checkpoint_metadata = CheckpointMetadata { - version: 5, - size: 10, - parts: None, - size_in_bytes: None, - num_of_add_files: None, - checkpoint_schema: None, - checksum: None, - }; - let (client, table_root) = build_log_with_paths_and_checkpoint( - &[ - delta_path_for_version(0, "json"), - delta_path_for_version(1, "json"), - delta_path_for_version(1, "checkpoint.parquet"), - delta_path_for_version(2, "json"), - delta_path_for_version(3, "json"), - delta_path_for_version(3, "checkpoint.parquet"), - delta_path_for_version(4, "json"), - delta_path_for_version(5, "json"), - delta_path_for_version(5, "checkpoint.parquet"), - delta_path_for_version(6, "json"), - delta_path_for_version(7, "json"), - ], - None, - ); - - let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, Some(4)) - .unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); - - assert_eq!(checkpoint_parts.len(), 1); - assert_eq!(checkpoint_parts[0].version, 3); - - assert_eq!(commit_files.len(), 1); - assert_eq!(commit_files[0].version, 4); - } - - #[test] - fn build_snapshot_with_start_checkpoint_and_time_travel_version() { - let checkpoint_metadata = CheckpointMetadata { - version: 3, - size: 10, - parts: None, - size_in_bytes: None, - num_of_add_files: None, - checkpoint_schema: None, - checksum: None, - }; - - let (client, table_root) = build_log_with_paths_and_checkpoint( - &[ - delta_path_for_version(0, "json"), - delta_path_for_version(1, "checkpoint.parquet"), - delta_path_for_version(2, "json"), - delta_path_for_version(3, "checkpoint.parquet"), - delta_path_for_version(4, "json"), - delta_path_for_version(5, "checkpoint.parquet"), - delta_path_for_version(6, "json"), - delta_path_for_version(7, "json"), - ], - Some(&checkpoint_metadata), - ); - - let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, Some(4)) - .unwrap(); - - assert_eq!(log_segment.checkpoint_parts[0].version, 3); - assert_eq!(log_segment.sorted_commit_files.len(), 1); - assert_eq!(log_segment.sorted_commit_files[0].version, 4); - } - #[test] - fn build_table_changes_with_commit_versions() { - let (client, table_root) = build_log_with_paths_and_checkpoint( - &[ - delta_path_for_version(0, "json"), - delta_path_for_version(1, "json"), - delta_path_for_version(1, "checkpoint.parquet"), - delta_path_for_version(2, "json"), - delta_path_for_version(3, "json"), - delta_path_for_version(3, "checkpoint.parquet"), - delta_path_for_version(4, "json"), - delta_path_for_version(5, "json"), - delta_path_for_version(5, "checkpoint.parquet"), - delta_path_for_version(6, "json"), - delta_path_for_version(7, "json"), - ], - None, - ); - - // -------------------------------------------------------------------------------- - // | Specify start version and end version | - // -------------------------------------------------------------------------------- - - let log_segment = - LogSegment::for_table_changes(client.as_ref(), &table_root, 2, 5).unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); - - // Checkpoints should be omitted - assert_eq!(checkpoint_parts.len(), 0); - - // Commits between 2 and 5 (inclusive) should be returned - let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); - let expected_versions = (2..=5).collect_vec(); - assert_eq!(versions, expected_versions); - - // -------------------------------------------------------------------------------- - // | Start version and end version are the same | - // -------------------------------------------------------------------------------- - let log_segment = - LogSegment::for_table_changes(client.as_ref(), &table_root, 0, Some(0)).unwrap(); - - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); - // Checkpoints should be omitted - assert_eq!(checkpoint_parts.len(), 0); - - // There should only be commit version 0 - assert_eq!(commit_files.len(), 1); - assert_eq!(commit_files[0].version, 0); - - // -------------------------------------------------------------------------------- - // | Specify no start or end version | - // -------------------------------------------------------------------------------- - let log_segment = - LogSegment::for_table_changes(client.as_ref(), &table_root, 0, None).unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); - - // Checkpoints should be omitted - assert_eq!(checkpoint_parts.len(), 0); - - // Commits between 2 and 7 (inclusive) should be returned - let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); - let expected_versions = (0..=7).collect_vec(); - assert_eq!(versions, expected_versions); - } - - #[test] - fn test_non_contiguous_log() { - // Commit with version 1 is missing - let (client, table_root) = build_log_with_paths_and_checkpoint( - &[ - delta_path_for_version(0, "json"), - delta_path_for_version(2, "json"), - ], - None, - ); - - let log_segment_res = LogSegment::for_table_changes(client.as_ref(), &table_root, 0, None); - assert!(log_segment_res.is_err()); - - let log_segment_res = LogSegment::for_table_changes(client.as_ref(), &table_root, 1, None); - assert!(log_segment_res.is_err()); - - let log_segment_res = - LogSegment::for_table_changes(client.as_ref(), &table_root, 0, Some(1)); - assert!(log_segment_res.is_err()); - } - - #[test] - fn table_changes_fails_with_larger_start_version_than_end() { - // Commit with version 1 is missing - let (client, table_root) = build_log_with_paths_and_checkpoint( - &[ - delta_path_for_version(0, "json"), - delta_path_for_version(1, "json"), - ], - None, - ); - let log_segment_res = - LogSegment::for_table_changes(client.as_ref(), &table_root, 1, Some(0)); - assert!(log_segment_res.is_err()); - } -} diff --git a/kernel/src/log_segment/mod.rs b/kernel/src/log_segment/mod.rs new file mode 100644 index 000000000..3d4026995 --- /dev/null +++ b/kernel/src/log_segment/mod.rs @@ -0,0 +1,378 @@ +//! Represents a segment of a delta log. [`LogSegment`] wraps a set of checkpoint and commit +//! files. + +use crate::actions::{get_log_schema, Metadata, Protocol, METADATA_NAME, PROTOCOL_NAME}; +use crate::path::ParsedLogPath; +use crate::schema::SchemaRef; +use crate::snapshot::CheckpointMetadata; +use crate::utils::require; +use crate::{ + DeltaResult, Engine, EngineData, Error, Expression, ExpressionRef, FileSystemClient, Version, +}; +use itertools::Itertools; +use std::cmp::Ordering; +use std::convert::identity; +use std::sync::{Arc, LazyLock}; +use tracing::warn; +use url::Url; + +#[cfg(test)] +mod tests; + +/// A [`LogSegment`] represents a contiguous section of the log and is made of checkpoint files +/// and commit files and guarantees the following: +/// 1. Commit file versions will not have any gaps between them. +/// 2. If checkpoint(s) is/are present in the range, only commits with versions greater than the most +/// recent checkpoint version are retained. There will not be a gap between the checkpoint +/// version and the first commit version. +/// 3. All checkpoint_parts in belong to the same checkpoint version, and must form a complete +/// version. Multi-part checkpoints must have all their parts. +/// +/// [`LogSegment`] is used in [`Snapshot`] when built with [`LogSegment::for_snapshot`], and +/// and in `TableChanges` when built with [`LogSegment::for_table_changes`]. +/// +/// [`Snapshot`]: crate::snapshot::Snapshot +#[derive(Debug)] +#[cfg_attr(feature = "developer-visibility", visibility::make(pub))] +pub(crate) struct LogSegment { + pub end_version: Version, + pub log_root: Url, + /// Sorted commit files in the log segment (ascending) + pub sorted_commit_files: Vec, + /// Checkpoint files in the log segment. + pub checkpoint_parts: Vec, +} + +impl LogSegment { + fn try_new( + sorted_commit_files: Vec, + checkpoint_parts: Vec, + log_root: Url, + end_version: Option, + ) -> DeltaResult { + // We require that commits that are contiguous. In other words, there must be no gap between commit versions. + require!( + sorted_commit_files + .windows(2) + .all(|cfs| cfs[0].version + 1 == cfs[1].version), + Error::generic(format!( + "Expected ordered contiguous commit files {:?}", + sorted_commit_files + )) + ); + + // There must be no gap between a checkpoint and the first commit version. Note that + // that all checkpoint parts share the same version. + if let (Some(checkpoint_file), Some(commit_file)) = + (checkpoint_parts.first(), sorted_commit_files.first()) + { + require!(checkpoint_file.version + 1 == commit_file.version, + Error::generic(format!("Expected commit file version {} to be next version to checkpoint file version {}", commit_file.version, checkpoint_file.version ))) + } + + // Get the effective version from chosen files + let version_eff = sorted_commit_files + .last() + .or(checkpoint_parts.first()) + .ok_or(Error::MissingVersion)? // TODO: A more descriptive error + .version; + if let Some(end_version) = end_version { + require!( + version_eff == end_version, + Error::generic(format!( + "LogSegment end version {} not the same as the specified end version {}", + version_eff, end_version + )) + ); + } + Ok(LogSegment { + end_version: version_eff, + log_root, + sorted_commit_files, + checkpoint_parts, + }) + } + + /// Constructs a [`LogSegment`] to be used for Snapshot. For a Snapshot at version `n`: + /// Its LogSegment is made of zero or one checkpoint, and all commits between the checkpoint up + /// to and including the end version `n`. Note that a checkpoint may be made of multiple + /// parts. All these parts will have the same checkpoint version. + /// + /// The options for constructing a LogSegment for Snapshot are as follows: + /// - `checkpoint_hint`: a `CheckpointMetadata` to start the log segment from (e.g. from reading the `last_checkpoint` file). + /// - `time_travel_version`: The version of the log that the Snapshot will be at. + #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] + pub(crate) fn for_snapshot( + fs_client: &dyn FileSystemClient, + table_root: &Url, + checkpoint_hint: impl Into>, + time_travel_version: impl Into>, + ) -> DeltaResult { + let time_travel_version = time_travel_version.into(); + let log_root = table_root.join("_delta_log/")?; + + let (mut sorted_commit_files, checkpoint_parts) = + match (checkpoint_hint.into(), time_travel_version) { + (Some(cp), None) => { + list_log_files_with_checkpoint(&cp, fs_client, &log_root, None)? + } + (Some(cp), Some(end_version)) if cp.version <= end_version => { + list_log_files_with_checkpoint(&cp, fs_client, &log_root, Some(end_version))? + } + _ => list_log_files_with_version(fs_client, &log_root, None, time_travel_version)?, + }; + + // Commit file versions must be greater than the most recent checkpoint version if it exists + if let Some(checkpoint_file) = checkpoint_parts.first() { + sorted_commit_files.retain(|log_path| checkpoint_file.version < log_path.version); + } + + LogSegment::try_new( + sorted_commit_files, + checkpoint_parts, + log_root, + time_travel_version, + ) + } + + /// Constructs a [`LogSegment`] to be used for `TableChanges`. For a TableChanges between versions + /// `start_version` and `end_version`: Its LogSegment is made of zero checkpoints and all commits + /// between versions `start_version` (inclusive) and `end_version` (inclusive). If no `end_version` + /// is specified it will be the most recent version by default. + #[allow(unused)] + #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] + pub(crate) fn for_table_changes( + fs_client: &dyn FileSystemClient, + table_root: &Url, + start_version: Version, + end_version: impl Into>, + ) -> DeltaResult { + let log_root = table_root.join("_delta_log/")?; + + let end_version = end_version.into(); + if let Some(end_version) = end_version { + if start_version > end_version { + return Err(Error::generic( + "Failed to build LogSegment: start_version cannot be greater than end_version", + )); + } + } + + let sorted_commit_files: Vec<_> = + list_log_files(fs_client, &log_root, start_version, end_version)? + .filter_ok(|x| x.is_commit()) + .try_collect()?; + + // - Here check that the start version is correct. + // - [`LogSegment::try_new`] will verify that the `end_version` is correct if present. + // - [`LogSegment::try_new`] also checks that there are no gaps between commits. + // If all three are satisfied, this implies that all the desired commits are present. + require!( + sorted_commit_files + .first() + .is_some_and(|first_commit| first_commit.version == start_version), + Error::generic(format!( + "Expected the first commit to have version {}", + start_version + )) + ); + LogSegment::try_new(sorted_commit_files, vec![], log_root, end_version) + } + /// Read a stream of log data from this log segment. + /// + /// The log files will be read from most recent to oldest. + /// The boolean flags indicates whether the data was read from + /// a commit file (true) or a checkpoint file (false). + /// + /// `read_schema` is the schema to read the log files with. This can be used + /// to project the log files to a subset of the columns. + /// + /// `meta_predicate` is an optional expression to filter the log files with. It is _NOT_ the + /// query's predicate, but rather a predicate for filtering log files themselves. + #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] + pub(crate) fn replay( + &self, + engine: &dyn Engine, + commit_read_schema: SchemaRef, + checkpoint_read_schema: SchemaRef, + meta_predicate: Option, + ) -> DeltaResult, bool)>> + Send> { + // `replay` expects commit files to be sorted in descending order, so we reverse the sorted + // commit files + let commit_files: Vec<_> = self + .sorted_commit_files + .iter() + .rev() + .map(|f| f.location.clone()) + .collect(); + let commit_stream = engine + .get_json_handler() + .read_json_files(&commit_files, commit_read_schema, meta_predicate.clone())? + .map_ok(|batch| (batch, true)); + + let checkpoint_parts: Vec<_> = self + .checkpoint_parts + .iter() + .map(|f| f.location.clone()) + .collect(); + let checkpoint_stream = engine + .get_parquet_handler() + .read_parquet_files(&checkpoint_parts, checkpoint_read_schema, meta_predicate)? + .map_ok(|batch| (batch, false)); + + Ok(commit_stream.chain(checkpoint_stream)) + } + + // Get the most up-to-date Protocol and Metadata actions + pub(crate) fn read_metadata(&self, engine: &dyn Engine) -> DeltaResult<(Metadata, Protocol)> { + let data_batches = self.replay_for_metadata(engine)?; + let (mut metadata_opt, mut protocol_opt) = (None, None); + for batch in data_batches { + let (batch, _) = batch?; + if metadata_opt.is_none() { + metadata_opt = Metadata::try_new_from_data(batch.as_ref())?; + } + if protocol_opt.is_none() { + protocol_opt = Protocol::try_new_from_data(batch.as_ref())?; + } + if metadata_opt.is_some() && protocol_opt.is_some() { + // we've found both, we can stop + break; + } + } + match (metadata_opt, protocol_opt) { + (Some(m), Some(p)) => Ok((m, p)), + (None, Some(_)) => Err(Error::MissingMetadata), + (Some(_), None) => Err(Error::MissingProtocol), + (None, None) => Err(Error::MissingMetadataAndProtocol), + } + } + + // Replay the commit log, projecting rows to only contain Protocol and Metadata action columns. + fn replay_for_metadata( + &self, + engine: &dyn Engine, + ) -> DeltaResult, bool)>> + Send> { + let schema = get_log_schema().project(&[PROTOCOL_NAME, METADATA_NAME])?; + // filter out log files that do not contain metadata or protocol information + static META_PREDICATE: LazyLock> = LazyLock::new(|| { + Some(Arc::new(Expression::or( + Expression::column([METADATA_NAME, "id"]).is_not_null(), + Expression::column([PROTOCOL_NAME, "minReaderVersion"]).is_not_null(), + ))) + }); + // read the same protocol and metadata schema for both commits and checkpoints + self.replay(engine, schema.clone(), schema, META_PREDICATE.clone()) + } +} + +/// Returns a fallible iterator of [`ParsedLogPath`] that are between the provided `start_version` (inclusive) +/// and `end_version` (inclusive). [`ParsedLogPath`] may be a commit or a checkpoint. If `start_version` is +/// not specified, the files will begin from version number 0. If `end_version` is not specified, files up to +/// the most recent version will be included. +/// +/// Note: this calls [`FileSystemClient::list_from`] to get the list of log files. +fn list_log_files( + fs_client: &dyn FileSystemClient, + log_root: &Url, + start_version: impl Into>, + end_version: impl Into>, +) -> DeltaResult>> { + let start_version = start_version.into().unwrap_or(0); + let end_version = end_version.into(); + + let version_prefix = format!("{:020}", start_version); + let start_from = log_root.join(&version_prefix)?; + + Ok(fs_client + .list_from(&start_from)? + .map(|meta| ParsedLogPath::try_from(meta?)) + // TODO this filters out .crc files etc which start with "." - how do we want to use these kind of files? + .filter_map_ok(identity) + .take_while(move |path_res| match path_res { + Ok(path) => end_version.is_none_or(|end_version| path.version <= end_version), + Err(_) => true, + })) +} +/// List all commit and checkpoint files with versions above the provided `start_version` (inclusive). +/// If successful, this returns a tuple `(sorted_commit_files_paths, checkpoint_parts)` of type +/// `(Vec, Vec)`. The commit files are guaranteed to be sorted in +/// ascending order by version. The elements of `checkpoint_parts` are all the parts of the same +/// checkpoint. Checkpoint parts share the same version. +fn list_log_files_with_version( + fs_client: &dyn FileSystemClient, + log_root: &Url, + start_version: Option, + end_version: Option, +) -> DeltaResult<(Vec, Vec)> { + let mut max_checkpoint_version = start_version; + let mut checkpoint_parts = vec![]; + // We expect 10 commit files per checkpoint, so start with that size. We could adjust this based + // on config at some point + let mut commit_files = Vec::with_capacity(10); + + for parsed_path in list_log_files(fs_client, log_root, start_version, end_version)? { + let parsed_path = parsed_path?; + if parsed_path.is_commit() { + commit_files.push(parsed_path); + } else if parsed_path.is_checkpoint() { + let path_version = parsed_path.version; + match max_checkpoint_version { + None => { + checkpoint_parts.push(parsed_path); + max_checkpoint_version = Some(path_version); + } + Some(checkpoint_version) => match path_version.cmp(&checkpoint_version) { + Ordering::Greater => { + max_checkpoint_version = Some(path_version); + checkpoint_parts.clear(); + checkpoint_parts.push(parsed_path); + } + Ordering::Equal => checkpoint_parts.push(parsed_path), + Ordering::Less => {} + }, + } + } + } + + Ok((commit_files, checkpoint_parts)) +} + +/// List all commit and checkpoint files after the provided checkpoint. It is guaranteed that all +/// the returned [`ParsedLogPath`]s will have a version less than or equal to the `end_version`. +/// See [`list_log_files_with_version`] for details on the return type. +fn list_log_files_with_checkpoint( + checkpoint_metadata: &CheckpointMetadata, + fs_client: &dyn FileSystemClient, + log_root: &Url, + end_version: Option, +) -> DeltaResult<(Vec, Vec)> { + let (commit_files, checkpoint_parts) = list_log_files_with_version( + fs_client, + log_root, + Some(checkpoint_metadata.version), + end_version, + )?; + + let Some(latest_checkpoint) = checkpoint_parts.last() else { + // TODO: We could potentially recover here + return Err(Error::generic( + "Had a _last_checkpoint hint but didn't find any checkpoints", + )); + }; + if latest_checkpoint.version != checkpoint_metadata.version { + warn!( + "_last_checkpoint hint is out of date. _last_checkpoint version: {}. Using actual most recent: {}", + checkpoint_metadata.version, + latest_checkpoint.version + ); + } + if checkpoint_parts.len() != checkpoint_metadata.parts.unwrap_or(1) { + return Err(Error::Generic(format!( + "_last_checkpoint indicated that checkpoint should have {} parts, but it has {}", + checkpoint_metadata.parts.unwrap_or(1), + checkpoint_parts.len() + ))); + } + Ok((commit_files, checkpoint_parts)) +} diff --git a/kernel/src/log_segment/tests.rs b/kernel/src/log_segment/tests.rs new file mode 100644 index 000000000..e11d008c8 --- /dev/null +++ b/kernel/src/log_segment/tests.rs @@ -0,0 +1,533 @@ +use std::{path::PathBuf, sync::Arc}; + +use itertools::Itertools; +use object_store::{memory::InMemory, path::Path, ObjectStore}; +use url::Url; + +use crate::engine::default::executor::tokio::TokioBackgroundExecutor; +use crate::engine::default::filesystem::ObjectStoreFileSystemClient; +use crate::engine::sync::SyncEngine; +use crate::log_segment::LogSegment; +use crate::snapshot::CheckpointMetadata; +use crate::{FileSystemClient, Table}; +use test_utils::{delta_path_for_multipart_checkpoint, delta_path_for_version}; + +// NOTE: In addition to testing the meta-predicate for metadata replay, this test also verifies +// that the parquet reader properly infers nullcount = rowcount for missing columns. The two +// checkpoint part files that contain transaction app ids have truncated schemas that would +// otherwise fail skipping due to their missing nullcount stat: +// +// Row group 0: count: 1 total(compressed): 111 B total(uncompressed):107 B +// -------------------------------------------------------------------------------- +// type nulls min / max +// txn.appId BINARY 0 "3ae45b72-24e1-865a-a211-3..." / "3ae45b72-24e1-865a-a211-3..." +// txn.version INT64 0 "4390" / "4390" +#[test] +fn test_replay_for_metadata() { + let path = std::fs::canonicalize(PathBuf::from("./tests/data/parquet_row_group_skipping/")); + let url = url::Url::from_directory_path(path.unwrap()).unwrap(); + let engine = SyncEngine::new(); + + let table = Table::new(url); + let snapshot = table.snapshot(&engine, None).unwrap(); + let data: Vec<_> = snapshot + .log_segment + .replay_for_metadata(&engine) + .unwrap() + .try_collect() + .unwrap(); + + // The checkpoint has five parts, each containing one action: + // 1. txn (physically missing P&M columns) + // 2. metaData + // 3. protocol + // 4. add + // 5. txn (physically missing P&M columns) + // + // The parquet reader should skip parts 1, 3, and 5. Note that the actual `read_metadata` + // always skips parts 4 and 5 because it terminates the iteration after finding both P&M. + // + // NOTE: Each checkpoint part is a single-row file -- guaranteed to produce one row group. + // + // WARNING: https://github.com/delta-incubator/delta-kernel-rs/issues/434 -- We currently + // read parts 1 and 5 (4 in all instead of 2) because row group skipping is disabled for + // missing columns, but can still skip part 3 because has valid nullcount stats for P&M. + assert_eq!(data.len(), 4); +} + +// Utility method to build a log using a list of log paths and an optional checkpoint hint. The +// CheckpointMetadata is written to `_delta_log/_last_checkpoint`. +fn build_log_with_paths_and_checkpoint( + paths: &[Path], + checkpoint_metadata: Option<&CheckpointMetadata>, +) -> (Box, Url) { + let store = Arc::new(InMemory::new()); + + let data = bytes::Bytes::from("kernel-data"); + + // add log files to store + tokio::runtime::Runtime::new() + .expect("create tokio runtime") + .block_on(async { + for path in paths { + store + .put(path, data.clone().into()) + .await + .expect("put log file in store"); + } + if let Some(checkpoint_metadata) = checkpoint_metadata { + let checkpoint_str = + serde_json::to_string(checkpoint_metadata).expect("Serialize checkpoint"); + store + .put( + &Path::from("_delta_log/_last_checkpoint"), + checkpoint_str.into(), + ) + .await + .expect("Write _last_checkpoint"); + } + }); + + let client = ObjectStoreFileSystemClient::new( + store, + false, // don't have ordered listing + Path::from("/"), + Arc::new(TokioBackgroundExecutor::new()), + ); + + let table_root = Url::parse("memory:///").expect("valid url"); + (Box::new(client), table_root) +} + +#[test] +fn build_snapshot_with_out_of_date_last_checkpoint() { + let checkpoint_metadata = CheckpointMetadata { + version: 3, + size: 10, + parts: None, + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(4, "json"), + delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + Some(&checkpoint_metadata), + ); + + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None).unwrap(); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); + + assert_eq!(checkpoint_parts.len(), 1); + assert_eq!(commit_files.len(), 2); + assert_eq!(checkpoint_parts[0].version, 5); + assert_eq!(commit_files[0].version, 6); + assert_eq!(commit_files[1].version, 7); +} +#[test] +fn build_snapshot_with_correct_last_multipart_checkpoint() { + let checkpoint_metadata = CheckpointMetadata { + version: 5, + size: 10, + parts: Some(3), + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(1, "json"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(3, "json"), + delta_path_for_version(4, "json"), + delta_path_for_multipart_checkpoint(5, 1, 3), + delta_path_for_multipart_checkpoint(5, 2, 3), + delta_path_for_multipart_checkpoint(5, 3, 3), + delta_path_for_version(5, "json"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + Some(&checkpoint_metadata), + ); + + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None).unwrap(); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); + + assert_eq!(checkpoint_parts.len(), 3); + assert_eq!(commit_files.len(), 2); + assert_eq!(checkpoint_parts[0].version, 5); + assert_eq!(commit_files[0].version, 6); + assert_eq!(commit_files[1].version, 7); +} + +#[test] +fn build_snapshot_with_missing_checkpoint_part_from_hint_fails() { + let checkpoint_metadata = CheckpointMetadata { + version: 5, + size: 10, + parts: Some(3), + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(1, "json"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(3, "json"), + delta_path_for_version(4, "json"), + delta_path_for_multipart_checkpoint(5, 1, 3), + // Part 2 is missing! + delta_path_for_multipart_checkpoint(5, 3, 3), + delta_path_for_version(5, "json"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + Some(&checkpoint_metadata), + ); + + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None); + assert!(log_segment.is_err()) +} +#[test] +fn build_snapshot_with_bad_checkpoint_hint_fails() { + let checkpoint_metadata = CheckpointMetadata { + version: 5, + size: 10, + parts: Some(1), + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(1, "json"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(3, "json"), + delta_path_for_version(4, "json"), + delta_path_for_multipart_checkpoint(5, 1, 2), + delta_path_for_multipart_checkpoint(5, 2, 2), + delta_path_for_version(5, "json"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + Some(&checkpoint_metadata), + ); + + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None); + assert!(log_segment.is_err()) +} + +#[ignore] +#[test] +fn build_snapshot_with_missing_checkpoint_part_no_hint() { + // TODO(Oussam): Hande checkpoints correctly so that this test passes + // Part 2 of 3 is missing from checkpoint 5. The Snapshot should be made of checkpoint + // number 3 and commit files 4 to 7. + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(1, "json"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(3, "json"), + delta_path_for_version(4, "json"), + delta_path_for_multipart_checkpoint(5, 1, 3), + // Part 2 is missing! + delta_path_for_multipart_checkpoint(5, 3, 3), + delta_path_for_version(5, "json"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + None, + ); + + let log_segment = LogSegment::for_snapshot(client.as_ref(), &table_root, None, None).unwrap(); + + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); + + assert_eq!(checkpoint_parts.len(), 1); + assert_eq!(checkpoint_parts[0].version, 3); + + let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); + let expected_versions = vec![4, 5, 6, 7]; + assert_eq!(versions, expected_versions); +} + +#[test] +fn build_snapshot_without_checkpoints() { + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(4, "json"), + delta_path_for_version(5, "json"), + delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + None, + ); + + // -------------------------------------------------------------------------------- + // | Specify no checkpoint or end version | + // -------------------------------------------------------------------------------- + let log_segment = LogSegment::for_snapshot(client.as_ref(), &table_root, None, None).unwrap(); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); + + assert_eq!(checkpoint_parts.len(), 1); + assert_eq!(checkpoint_parts[0].version, 5); + + // All commit files should still be there + let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); + let expected_versions = vec![6, 7]; + assert_eq!(versions, expected_versions); + + // -------------------------------------------------------------------------------- + // | Specify only end version | + // -------------------------------------------------------------------------------- + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, None, Some(2)).unwrap(); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); + + assert_eq!(checkpoint_parts.len(), 1); + assert_eq!(checkpoint_parts[0].version, 1); + + // All commit files should still be there + let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); + let expected_versions = vec![2]; + assert_eq!(versions, expected_versions); +} + +#[test] +fn build_snapshot_with_checkpoint_greater_than_time_travel_version() { + let checkpoint_metadata = CheckpointMetadata { + version: 5, + size: 10, + parts: None, + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(4, "json"), + delta_path_for_version(5, "json"), + delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + None, + ); + + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, Some(4)) + .unwrap(); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); + + assert_eq!(checkpoint_parts.len(), 1); + assert_eq!(checkpoint_parts[0].version, 3); + + assert_eq!(commit_files.len(), 1); + assert_eq!(commit_files[0].version, 4); +} + +#[test] +fn build_snapshot_with_start_checkpoint_and_time_travel_version() { + let checkpoint_metadata = CheckpointMetadata { + version: 3, + size: 10, + parts: None, + size_in_bytes: None, + num_of_add_files: None, + checkpoint_schema: None, + checksum: None, + }; + + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(4, "json"), + delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + Some(&checkpoint_metadata), + ); + + let log_segment = + LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, Some(4)) + .unwrap(); + + assert_eq!(log_segment.checkpoint_parts[0].version, 3); + assert_eq!(log_segment.sorted_commit_files.len(), 1); + assert_eq!(log_segment.sorted_commit_files[0].version, 4); +} +#[test] +fn build_table_changes_with_commit_versions() { + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "json"), + delta_path_for_version(1, "checkpoint.parquet"), + delta_path_for_version(2, "json"), + delta_path_for_version(3, "json"), + delta_path_for_version(3, "checkpoint.parquet"), + delta_path_for_version(4, "json"), + delta_path_for_version(5, "json"), + delta_path_for_version(5, "checkpoint.parquet"), + delta_path_for_version(6, "json"), + delta_path_for_version(7, "json"), + ], + None, + ); + + // -------------------------------------------------------------------------------- + // | Specify start version and end version | + // -------------------------------------------------------------------------------- + + let log_segment = LogSegment::for_table_changes(client.as_ref(), &table_root, 2, 5).unwrap(); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); + + // Checkpoints should be omitted + assert_eq!(checkpoint_parts.len(), 0); + + // Commits between 2 and 5 (inclusive) should be returned + let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); + let expected_versions = (2..=5).collect_vec(); + assert_eq!(versions, expected_versions); + + // -------------------------------------------------------------------------------- + // | Start version and end version are the same | + // -------------------------------------------------------------------------------- + let log_segment = + LogSegment::for_table_changes(client.as_ref(), &table_root, 0, Some(0)).unwrap(); + + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); + // Checkpoints should be omitted + assert_eq!(checkpoint_parts.len(), 0); + + // There should only be commit version 0 + assert_eq!(commit_files.len(), 1); + assert_eq!(commit_files[0].version, 0); + + // -------------------------------------------------------------------------------- + // | Specify no start or end version | + // -------------------------------------------------------------------------------- + let log_segment = LogSegment::for_table_changes(client.as_ref(), &table_root, 0, None).unwrap(); + let (commit_files, checkpoint_parts) = ( + log_segment.sorted_commit_files, + log_segment.checkpoint_parts, + ); + + // Checkpoints should be omitted + assert_eq!(checkpoint_parts.len(), 0); + + // Commits between 2 and 7 (inclusive) should be returned + let versions = commit_files.into_iter().map(|x| x.version).collect_vec(); + let expected_versions = (0..=7).collect_vec(); + assert_eq!(versions, expected_versions); +} + +#[test] +fn test_non_contiguous_log() { + // Commit with version 1 is missing + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(2, "json"), + ], + None, + ); + + let log_segment_res = LogSegment::for_table_changes(client.as_ref(), &table_root, 0, None); + assert!(log_segment_res.is_err()); + + let log_segment_res = LogSegment::for_table_changes(client.as_ref(), &table_root, 1, None); + assert!(log_segment_res.is_err()); + + let log_segment_res = LogSegment::for_table_changes(client.as_ref(), &table_root, 0, Some(1)); + assert!(log_segment_res.is_err()); +} + +#[test] +fn table_changes_fails_with_larger_start_version_than_end() { + // Commit with version 1 is missing + let (client, table_root) = build_log_with_paths_and_checkpoint( + &[ + delta_path_for_version(0, "json"), + delta_path_for_version(1, "json"), + ], + None, + ); + let log_segment_res = LogSegment::for_table_changes(client.as_ref(), &table_root, 1, Some(0)); + assert!(log_segment_res.is_err()); +} From b94e2dd7bf9b18b3d15cab3271321d73133efc55 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 23:35:40 -0800 Subject: [PATCH 77/88] Address more comments --- kernel/src/log_segment/mod.rs | 9 ++--- kernel/src/log_segment/tests.rs | 62 ++++++++++++++++----------------- kernel/src/snapshot.rs | 5 +-- 3 files changed, 37 insertions(+), 39 deletions(-) diff --git a/kernel/src/log_segment/mod.rs b/kernel/src/log_segment/mod.rs index 3d4026995..05310125f 100644 --- a/kernel/src/log_segment/mod.rs +++ b/kernel/src/log_segment/mod.rs @@ -25,7 +25,7 @@ mod tests; /// 2. If checkpoint(s) is/are present in the range, only commits with versions greater than the most /// recent checkpoint version are retained. There will not be a gap between the checkpoint /// version and the first commit version. -/// 3. All checkpoint_parts in belong to the same checkpoint version, and must form a complete +/// 3. All checkpoint_parts must belong to the same checkpoint version, and must form a complete /// version. Multi-part checkpoints must have all their parts. /// /// [`LogSegment`] is used in [`Snapshot`] when built with [`LogSegment::for_snapshot`], and @@ -104,12 +104,11 @@ impl LogSegment { #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) fn for_snapshot( fs_client: &dyn FileSystemClient, - table_root: &Url, + log_root: Url, checkpoint_hint: impl Into>, time_travel_version: impl Into>, ) -> DeltaResult { let time_travel_version = time_travel_version.into(); - let log_root = table_root.join("_delta_log/")?; let (mut sorted_commit_files, checkpoint_parts) = match (checkpoint_hint.into(), time_travel_version) { @@ -143,12 +142,10 @@ impl LogSegment { #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) fn for_table_changes( fs_client: &dyn FileSystemClient, - table_root: &Url, + log_root: Url, start_version: Version, end_version: impl Into>, ) -> DeltaResult { - let log_root = table_root.join("_delta_log/")?; - let end_version = end_version.into(); if let Some(end_version) = end_version { if start_version > end_version { diff --git a/kernel/src/log_segment/tests.rs b/kernel/src/log_segment/tests.rs index e11d008c8..9b08a5fb7 100644 --- a/kernel/src/log_segment/tests.rs +++ b/kernel/src/log_segment/tests.rs @@ -96,7 +96,8 @@ fn build_log_with_paths_and_checkpoint( ); let table_root = Url::parse("memory:///").expect("valid url"); - (Box::new(client), table_root) + let log_root = table_root.join("_delta_log/").unwrap(); + (Box::new(client), log_root) } #[test] @@ -111,7 +112,7 @@ fn build_snapshot_with_out_of_date_last_checkpoint() { checksum: None, }; - let (client, table_root) = build_log_with_paths_and_checkpoint( + let (client, log_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), delta_path_for_version(1, "checkpoint.parquet"), @@ -126,7 +127,7 @@ fn build_snapshot_with_out_of_date_last_checkpoint() { ); let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None).unwrap(); + LogSegment::for_snapshot(client.as_ref(), log_root, checkpoint_metadata, None).unwrap(); let (commit_files, checkpoint_parts) = ( log_segment.sorted_commit_files, log_segment.checkpoint_parts, @@ -150,7 +151,7 @@ fn build_snapshot_with_correct_last_multipart_checkpoint() { checksum: None, }; - let (client, table_root) = build_log_with_paths_and_checkpoint( + let (client, log_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), delta_path_for_version(1, "checkpoint.parquet"), @@ -170,7 +171,7 @@ fn build_snapshot_with_correct_last_multipart_checkpoint() { ); let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None).unwrap(); + LogSegment::for_snapshot(client.as_ref(), log_root, checkpoint_metadata, None).unwrap(); let (commit_files, checkpoint_parts) = ( log_segment.sorted_commit_files, log_segment.checkpoint_parts, @@ -195,7 +196,7 @@ fn build_snapshot_with_missing_checkpoint_part_from_hint_fails() { checksum: None, }; - let (client, table_root) = build_log_with_paths_and_checkpoint( + let (client, log_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), delta_path_for_version(1, "checkpoint.parquet"), @@ -215,7 +216,7 @@ fn build_snapshot_with_missing_checkpoint_part_from_hint_fails() { ); let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None); + LogSegment::for_snapshot(client.as_ref(), log_root, checkpoint_metadata, None); assert!(log_segment.is_err()) } #[test] @@ -230,7 +231,7 @@ fn build_snapshot_with_bad_checkpoint_hint_fails() { checksum: None, }; - let (client, table_root) = build_log_with_paths_and_checkpoint( + let (client, log_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), delta_path_for_version(1, "checkpoint.parquet"), @@ -249,7 +250,7 @@ fn build_snapshot_with_bad_checkpoint_hint_fails() { ); let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, None); + LogSegment::for_snapshot(client.as_ref(), log_root, checkpoint_metadata, None); assert!(log_segment.is_err()) } @@ -259,7 +260,7 @@ fn build_snapshot_with_missing_checkpoint_part_no_hint() { // TODO(Oussam): Hande checkpoints correctly so that this test passes // Part 2 of 3 is missing from checkpoint 5. The Snapshot should be made of checkpoint // number 3 and commit files 4 to 7. - let (client, table_root) = build_log_with_paths_and_checkpoint( + let (client, log_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), delta_path_for_version(1, "checkpoint.parquet"), @@ -278,7 +279,7 @@ fn build_snapshot_with_missing_checkpoint_part_no_hint() { None, ); - let log_segment = LogSegment::for_snapshot(client.as_ref(), &table_root, None, None).unwrap(); + let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root, None, None).unwrap(); let (commit_files, checkpoint_parts) = ( log_segment.sorted_commit_files, @@ -295,7 +296,7 @@ fn build_snapshot_with_missing_checkpoint_part_no_hint() { #[test] fn build_snapshot_without_checkpoints() { - let (client, table_root) = build_log_with_paths_and_checkpoint( + let (client, log_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), delta_path_for_version(1, "json"), @@ -315,7 +316,8 @@ fn build_snapshot_without_checkpoints() { // -------------------------------------------------------------------------------- // | Specify no checkpoint or end version | // -------------------------------------------------------------------------------- - let log_segment = LogSegment::for_snapshot(client.as_ref(), &table_root, None, None).unwrap(); + let log_segment = + LogSegment::for_snapshot(client.as_ref(), log_root.clone(), None, None).unwrap(); let (commit_files, checkpoint_parts) = ( log_segment.sorted_commit_files, log_segment.checkpoint_parts, @@ -332,8 +334,7 @@ fn build_snapshot_without_checkpoints() { // -------------------------------------------------------------------------------- // | Specify only end version | // -------------------------------------------------------------------------------- - let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, None, Some(2)).unwrap(); + let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root, None, Some(2)).unwrap(); let (commit_files, checkpoint_parts) = ( log_segment.sorted_commit_files, log_segment.checkpoint_parts, @@ -359,7 +360,7 @@ fn build_snapshot_with_checkpoint_greater_than_time_travel_version() { checkpoint_schema: None, checksum: None, }; - let (client, table_root) = build_log_with_paths_and_checkpoint( + let (client, log_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), delta_path_for_version(1, "json"), @@ -377,8 +378,7 @@ fn build_snapshot_with_checkpoint_greater_than_time_travel_version() { ); let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, Some(4)) - .unwrap(); + LogSegment::for_snapshot(client.as_ref(), log_root, checkpoint_metadata, Some(4)).unwrap(); let (commit_files, checkpoint_parts) = ( log_segment.sorted_commit_files, log_segment.checkpoint_parts, @@ -403,7 +403,7 @@ fn build_snapshot_with_start_checkpoint_and_time_travel_version() { checksum: None, }; - let (client, table_root) = build_log_with_paths_and_checkpoint( + let (client, log_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), delta_path_for_version(1, "checkpoint.parquet"), @@ -418,8 +418,7 @@ fn build_snapshot_with_start_checkpoint_and_time_travel_version() { ); let log_segment = - LogSegment::for_snapshot(client.as_ref(), &table_root, checkpoint_metadata, Some(4)) - .unwrap(); + LogSegment::for_snapshot(client.as_ref(), log_root, checkpoint_metadata, Some(4)).unwrap(); assert_eq!(log_segment.checkpoint_parts[0].version, 3); assert_eq!(log_segment.sorted_commit_files.len(), 1); @@ -427,7 +426,7 @@ fn build_snapshot_with_start_checkpoint_and_time_travel_version() { } #[test] fn build_table_changes_with_commit_versions() { - let (client, table_root) = build_log_with_paths_and_checkpoint( + let (client, log_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), delta_path_for_version(1, "json"), @@ -448,7 +447,8 @@ fn build_table_changes_with_commit_versions() { // | Specify start version and end version | // -------------------------------------------------------------------------------- - let log_segment = LogSegment::for_table_changes(client.as_ref(), &table_root, 2, 5).unwrap(); + let log_segment = + LogSegment::for_table_changes(client.as_ref(), log_root.clone(), 2, 5).unwrap(); let (commit_files, checkpoint_parts) = ( log_segment.sorted_commit_files, log_segment.checkpoint_parts, @@ -466,7 +466,7 @@ fn build_table_changes_with_commit_versions() { // | Start version and end version are the same | // -------------------------------------------------------------------------------- let log_segment = - LogSegment::for_table_changes(client.as_ref(), &table_root, 0, Some(0)).unwrap(); + LogSegment::for_table_changes(client.as_ref(), log_root.clone(), 0, Some(0)).unwrap(); let (commit_files, checkpoint_parts) = ( log_segment.sorted_commit_files, @@ -482,7 +482,7 @@ fn build_table_changes_with_commit_versions() { // -------------------------------------------------------------------------------- // | Specify no start or end version | // -------------------------------------------------------------------------------- - let log_segment = LogSegment::for_table_changes(client.as_ref(), &table_root, 0, None).unwrap(); + let log_segment = LogSegment::for_table_changes(client.as_ref(), log_root, 0, None).unwrap(); let (commit_files, checkpoint_parts) = ( log_segment.sorted_commit_files, log_segment.checkpoint_parts, @@ -500,7 +500,7 @@ fn build_table_changes_with_commit_versions() { #[test] fn test_non_contiguous_log() { // Commit with version 1 is missing - let (client, table_root) = build_log_with_paths_and_checkpoint( + let (client, log_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), delta_path_for_version(2, "json"), @@ -508,26 +508,26 @@ fn test_non_contiguous_log() { None, ); - let log_segment_res = LogSegment::for_table_changes(client.as_ref(), &table_root, 0, None); + let log_segment_res = LogSegment::for_table_changes(client.as_ref(), log_root.clone(), 0, None); assert!(log_segment_res.is_err()); - let log_segment_res = LogSegment::for_table_changes(client.as_ref(), &table_root, 1, None); + let log_segment_res = LogSegment::for_table_changes(client.as_ref(), log_root.clone(), 1, None); assert!(log_segment_res.is_err()); - let log_segment_res = LogSegment::for_table_changes(client.as_ref(), &table_root, 0, Some(1)); + let log_segment_res = LogSegment::for_table_changes(client.as_ref(), log_root, 0, Some(1)); assert!(log_segment_res.is_err()); } #[test] fn table_changes_fails_with_larger_start_version_than_end() { // Commit with version 1 is missing - let (client, table_root) = build_log_with_paths_and_checkpoint( + let (client, log_root) = build_log_with_paths_and_checkpoint( &[ delta_path_for_version(0, "json"), delta_path_for_version(1, "json"), ], None, ); - let log_segment_res = LogSegment::for_table_changes(client.as_ref(), &table_root, 1, Some(0)); + let log_segment_res = LogSegment::for_table_changes(client.as_ref(), log_root, 1, Some(0)); assert!(log_segment_res.is_err()); } diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index ad0ae1722..2e0005016 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -59,11 +59,12 @@ impl Snapshot { version: Option, ) -> DeltaResult { let fs_client = engine.get_file_system_client(); - let log_url = table_root.join("_delta_log/").unwrap(); + let log_url = table_root.join("_delta_log/")?; let checkpoint_hint = read_last_checkpoint(fs_client.as_ref(), &log_url)?; + let log_segment = - LogSegment::for_snapshot(fs_client.as_ref(), &table_root, checkpoint_hint, version)?; + LogSegment::for_snapshot(fs_client.as_ref(), log_url, checkpoint_hint, version)?; Self::try_new_from_log_segment(table_root, log_segment, engine) } From ff8ad65cf7b2d1ca9043cc5580a782bf6008df40 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 23:36:15 -0800 Subject: [PATCH 78/88] Make naming consistent --- kernel/src/snapshot.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 2e0005016..2a3e61b1f 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -59,12 +59,12 @@ impl Snapshot { version: Option, ) -> DeltaResult { let fs_client = engine.get_file_system_client(); - let log_url = table_root.join("_delta_log/")?; + let log_root = table_root.join("_delta_log/")?; - let checkpoint_hint = read_last_checkpoint(fs_client.as_ref(), &log_url)?; + let checkpoint_hint = read_last_checkpoint(fs_client.as_ref(), &log_root)?; let log_segment = - LogSegment::for_snapshot(fs_client.as_ref(), log_url, checkpoint_hint, version)?; + LogSegment::for_snapshot(fs_client.as_ref(), log_root, checkpoint_hint, version)?; Self::try_new_from_log_segment(table_root, log_segment, engine) } From 60a8a08c8938ef7ea8e39df6521a8981266a9ee5 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Thu, 14 Nov 2024 23:47:43 -0800 Subject: [PATCH 79/88] Address more nits --- kernel/src/log_segment/mod.rs | 3 +- kernel/src/log_segment/tests.rs | 54 +++++++++++---------------------- 2 files changed, 19 insertions(+), 38 deletions(-) diff --git a/kernel/src/log_segment/mod.rs b/kernel/src/log_segment/mod.rs index 05310125f..292bfff77 100644 --- a/kernel/src/log_segment/mod.rs +++ b/kernel/src/log_segment/mod.rs @@ -363,8 +363,7 @@ fn list_log_files_with_checkpoint( checkpoint_metadata.version, latest_checkpoint.version ); - } - if checkpoint_parts.len() != checkpoint_metadata.parts.unwrap_or(1) { + } else if checkpoint_parts.len() != checkpoint_metadata.parts.unwrap_or(1) { return Err(Error::Generic(format!( "_last_checkpoint indicated that checkpoint should have {} parts, but it has {}", checkpoint_metadata.parts.unwrap_or(1), diff --git a/kernel/src/log_segment/tests.rs b/kernel/src/log_segment/tests.rs index 9b08a5fb7..686690c75 100644 --- a/kernel/src/log_segment/tests.rs +++ b/kernel/src/log_segment/tests.rs @@ -128,10 +128,8 @@ fn build_snapshot_with_out_of_date_last_checkpoint() { let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root, checkpoint_metadata, None).unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); + let commit_files = log_segment.sorted_commit_files; + let checkpoint_parts = log_segment.checkpoint_parts; assert_eq!(checkpoint_parts.len(), 1); assert_eq!(commit_files.len(), 2); @@ -172,10 +170,8 @@ fn build_snapshot_with_correct_last_multipart_checkpoint() { let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root, checkpoint_metadata, None).unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); + let commit_files = log_segment.sorted_commit_files; + let checkpoint_parts = log_segment.checkpoint_parts; assert_eq!(checkpoint_parts.len(), 3); assert_eq!(commit_files.len(), 2); @@ -281,10 +277,8 @@ fn build_snapshot_with_missing_checkpoint_part_no_hint() { let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root, None, None).unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); + let commit_files = log_segment.sorted_commit_files; + let checkpoint_parts = log_segment.checkpoint_parts; assert_eq!(checkpoint_parts.len(), 1); assert_eq!(checkpoint_parts[0].version, 3); @@ -318,10 +312,8 @@ fn build_snapshot_without_checkpoints() { // -------------------------------------------------------------------------------- let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root.clone(), None, None).unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); + let commit_files = log_segment.sorted_commit_files; + let checkpoint_parts = log_segment.checkpoint_parts; assert_eq!(checkpoint_parts.len(), 1); assert_eq!(checkpoint_parts[0].version, 5); @@ -335,10 +327,8 @@ fn build_snapshot_without_checkpoints() { // | Specify only end version | // -------------------------------------------------------------------------------- let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root, None, Some(2)).unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); + let commit_files = log_segment.sorted_commit_files; + let checkpoint_parts = log_segment.checkpoint_parts; assert_eq!(checkpoint_parts.len(), 1); assert_eq!(checkpoint_parts[0].version, 1); @@ -379,10 +369,8 @@ fn build_snapshot_with_checkpoint_greater_than_time_travel_version() { let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root, checkpoint_metadata, Some(4)).unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); + let commit_files = log_segment.sorted_commit_files; + let checkpoint_parts = log_segment.checkpoint_parts; assert_eq!(checkpoint_parts.len(), 1); assert_eq!(checkpoint_parts[0].version, 3); @@ -449,10 +437,8 @@ fn build_table_changes_with_commit_versions() { let log_segment = LogSegment::for_table_changes(client.as_ref(), log_root.clone(), 2, 5).unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); + let commit_files = log_segment.sorted_commit_files; + let checkpoint_parts = log_segment.checkpoint_parts; // Checkpoints should be omitted assert_eq!(checkpoint_parts.len(), 0); @@ -468,10 +454,8 @@ fn build_table_changes_with_commit_versions() { let log_segment = LogSegment::for_table_changes(client.as_ref(), log_root.clone(), 0, Some(0)).unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); + let commit_files = log_segment.sorted_commit_files; + let checkpoint_parts = log_segment.checkpoint_parts; // Checkpoints should be omitted assert_eq!(checkpoint_parts.len(), 0); @@ -483,10 +467,8 @@ fn build_table_changes_with_commit_versions() { // | Specify no start or end version | // -------------------------------------------------------------------------------- let log_segment = LogSegment::for_table_changes(client.as_ref(), log_root, 0, None).unwrap(); - let (commit_files, checkpoint_parts) = ( - log_segment.sorted_commit_files, - log_segment.checkpoint_parts, - ); + let commit_files = log_segment.sorted_commit_files; + let checkpoint_parts = log_segment.checkpoint_parts; // Checkpoints should be omitted assert_eq!(checkpoint_parts.len(), 0); From 2f02d1993cb086fd7fe5368c2d2e7c478f0103ff Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 15 Nov 2024 10:17:13 -0800 Subject: [PATCH 80/88] Update kernel/src/log_segment/mod.rs Co-authored-by: Ryan Johnson --- kernel/src/log_segment/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kernel/src/log_segment/mod.rs b/kernel/src/log_segment/mod.rs index 292bfff77..0a04ae1d5 100644 --- a/kernel/src/log_segment/mod.rs +++ b/kernel/src/log_segment/mod.rs @@ -302,11 +302,11 @@ fn list_log_files_with_version( start_version: Option, end_version: Option, ) -> DeltaResult<(Vec, Vec)> { - let mut max_checkpoint_version = start_version; - let mut checkpoint_parts = vec![]; // We expect 10 commit files per checkpoint, so start with that size. We could adjust this based // on config at some point let mut commit_files = Vec::with_capacity(10); + let mut checkpoint_parts = vec![]; + let mut max_checkpoint_version = start_version; for parsed_path in list_log_files(fs_client, log_root, start_version, end_version)? { let parsed_path = parsed_path?; From 38fb7d35dc1309029703586930afa8554a3ff28b Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 15 Nov 2024 10:18:16 -0800 Subject: [PATCH 81/88] address more pr comments --- kernel/src/log_segment/mod.rs | 46 +++++++++++++++++++-------------- kernel/src/log_segment/tests.rs | 22 ++++++++-------- kernel/src/snapshot.rs | 14 ++++++---- 3 files changed, 46 insertions(+), 36 deletions(-) diff --git a/kernel/src/log_segment/mod.rs b/kernel/src/log_segment/mod.rs index 292bfff77..85d521ae2 100644 --- a/kernel/src/log_segment/mod.rs +++ b/kernel/src/log_segment/mod.rs @@ -38,43 +38,50 @@ pub(crate) struct LogSegment { pub end_version: Version, pub log_root: Url, /// Sorted commit files in the log segment (ascending) - pub sorted_commit_files: Vec, + pub ascending_commit_files: Vec, /// Checkpoint files in the log segment. pub checkpoint_parts: Vec, } impl LogSegment { fn try_new( - sorted_commit_files: Vec, + ascending_commit_files: Vec, checkpoint_parts: Vec, log_root: Url, end_version: Option, ) -> DeltaResult { // We require that commits that are contiguous. In other words, there must be no gap between commit versions. require!( - sorted_commit_files + ascending_commit_files .windows(2) .all(|cfs| cfs[0].version + 1 == cfs[1].version), Error::generic(format!( "Expected ordered contiguous commit files {:?}", - sorted_commit_files + ascending_commit_files )) ); // There must be no gap between a checkpoint and the first commit version. Note that // that all checkpoint parts share the same version. if let (Some(checkpoint_file), Some(commit_file)) = - (checkpoint_parts.first(), sorted_commit_files.first()) + (checkpoint_parts.first(), ascending_commit_files.first()) { - require!(checkpoint_file.version + 1 == commit_file.version, - Error::generic(format!("Expected commit file version {} to be next version to checkpoint file version {}", commit_file.version, checkpoint_file.version ))) + require!( + checkpoint_file.version + 1 == commit_file.version, + Error::generic(format!( + "Gap between checkpoint version {} and next commit {}", + commit_file.version, checkpoint_file.version + )) + ) } // Get the effective version from chosen files - let version_eff = sorted_commit_files + let version_eff = ascending_commit_files .last() .or(checkpoint_parts.first()) - .ok_or(Error::MissingVersion)? // TODO: A more descriptive error + .ok_or(Error::generic( + "Failed to build log segment: No commit or checkpoitn files provided.", + ))? .version; if let Some(end_version) = end_version { require!( @@ -88,7 +95,7 @@ impl LogSegment { Ok(LogSegment { end_version: version_eff, log_root, - sorted_commit_files, + ascending_commit_files, checkpoint_parts, }) } @@ -110,7 +117,7 @@ impl LogSegment { ) -> DeltaResult { let time_travel_version = time_travel_version.into(); - let (mut sorted_commit_files, checkpoint_parts) = + let (mut ascending_commit_files, checkpoint_parts) = match (checkpoint_hint.into(), time_travel_version) { (Some(cp), None) => { list_log_files_with_checkpoint(&cp, fs_client, &log_root, None)? @@ -123,11 +130,11 @@ impl LogSegment { // Commit file versions must be greater than the most recent checkpoint version if it exists if let Some(checkpoint_file) = checkpoint_parts.first() { - sorted_commit_files.retain(|log_path| checkpoint_file.version < log_path.version); + ascending_commit_files.retain(|log_path| checkpoint_file.version < log_path.version); } LogSegment::try_new( - sorted_commit_files, + ascending_commit_files, checkpoint_parts, log_root, time_travel_version, @@ -155,7 +162,7 @@ impl LogSegment { } } - let sorted_commit_files: Vec<_> = + let ascending_commit_files: Vec<_> = list_log_files(fs_client, &log_root, start_version, end_version)? .filter_ok(|x| x.is_commit()) .try_collect()?; @@ -165,7 +172,7 @@ impl LogSegment { // - [`LogSegment::try_new`] also checks that there are no gaps between commits. // If all three are satisfied, this implies that all the desired commits are present. require!( - sorted_commit_files + ascending_commit_files .first() .is_some_and(|first_commit| first_commit.version == start_version), Error::generic(format!( @@ -173,7 +180,7 @@ impl LogSegment { start_version )) ); - LogSegment::try_new(sorted_commit_files, vec![], log_root, end_version) + LogSegment::try_new(ascending_commit_files, vec![], log_root, end_version) } /// Read a stream of log data from this log segment. /// @@ -197,7 +204,7 @@ impl LogSegment { // `replay` expects commit files to be sorted in descending order, so we reverse the sorted // commit files let commit_files: Vec<_> = self - .sorted_commit_files + .ascending_commit_files .iter() .rev() .map(|f| f.location.clone()) @@ -277,7 +284,6 @@ fn list_log_files( ) -> DeltaResult>> { let start_version = start_version.into().unwrap_or(0); let end_version = end_version.into(); - let version_prefix = format!("{:020}", start_version); let start_from = log_root.join(&version_prefix)?; @@ -287,12 +293,12 @@ fn list_log_files( // TODO this filters out .crc files etc which start with "." - how do we want to use these kind of files? .filter_map_ok(identity) .take_while(move |path_res| match path_res { - Ok(path) => end_version.is_none_or(|end_version| path.version <= end_version), + Ok(path) => !end_version.is_some_and(|end_version| end_version < path.version), Err(_) => true, })) } /// List all commit and checkpoint files with versions above the provided `start_version` (inclusive). -/// If successful, this returns a tuple `(sorted_commit_files_paths, checkpoint_parts)` of type +/// If successful, this returns a tuple `(ascending_commit_files, checkpoint_parts)` of type /// `(Vec, Vec)`. The commit files are guaranteed to be sorted in /// ascending order by version. The elements of `checkpoint_parts` are all the parts of the same /// checkpoint. Checkpoint parts share the same version. diff --git a/kernel/src/log_segment/tests.rs b/kernel/src/log_segment/tests.rs index 686690c75..e59b5f318 100644 --- a/kernel/src/log_segment/tests.rs +++ b/kernel/src/log_segment/tests.rs @@ -128,7 +128,7 @@ fn build_snapshot_with_out_of_date_last_checkpoint() { let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root, checkpoint_metadata, None).unwrap(); - let commit_files = log_segment.sorted_commit_files; + let commit_files = log_segment.ascending_commit_files; let checkpoint_parts = log_segment.checkpoint_parts; assert_eq!(checkpoint_parts.len(), 1); @@ -170,7 +170,7 @@ fn build_snapshot_with_correct_last_multipart_checkpoint() { let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root, checkpoint_metadata, None).unwrap(); - let commit_files = log_segment.sorted_commit_files; + let commit_files = log_segment.ascending_commit_files; let checkpoint_parts = log_segment.checkpoint_parts; assert_eq!(checkpoint_parts.len(), 3); @@ -277,7 +277,7 @@ fn build_snapshot_with_missing_checkpoint_part_no_hint() { let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root, None, None).unwrap(); - let commit_files = log_segment.sorted_commit_files; + let commit_files = log_segment.ascending_commit_files; let checkpoint_parts = log_segment.checkpoint_parts; assert_eq!(checkpoint_parts.len(), 1); @@ -312,7 +312,7 @@ fn build_snapshot_without_checkpoints() { // -------------------------------------------------------------------------------- let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root.clone(), None, None).unwrap(); - let commit_files = log_segment.sorted_commit_files; + let commit_files = log_segment.ascending_commit_files; let checkpoint_parts = log_segment.checkpoint_parts; assert_eq!(checkpoint_parts.len(), 1); @@ -327,7 +327,7 @@ fn build_snapshot_without_checkpoints() { // | Specify only end version | // -------------------------------------------------------------------------------- let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root, None, Some(2)).unwrap(); - let commit_files = log_segment.sorted_commit_files; + let commit_files = log_segment.ascending_commit_files; let checkpoint_parts = log_segment.checkpoint_parts; assert_eq!(checkpoint_parts.len(), 1); @@ -369,7 +369,7 @@ fn build_snapshot_with_checkpoint_greater_than_time_travel_version() { let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root, checkpoint_metadata, Some(4)).unwrap(); - let commit_files = log_segment.sorted_commit_files; + let commit_files = log_segment.ascending_commit_files; let checkpoint_parts = log_segment.checkpoint_parts; assert_eq!(checkpoint_parts.len(), 1); @@ -409,8 +409,8 @@ fn build_snapshot_with_start_checkpoint_and_time_travel_version() { LogSegment::for_snapshot(client.as_ref(), log_root, checkpoint_metadata, Some(4)).unwrap(); assert_eq!(log_segment.checkpoint_parts[0].version, 3); - assert_eq!(log_segment.sorted_commit_files.len(), 1); - assert_eq!(log_segment.sorted_commit_files[0].version, 4); + assert_eq!(log_segment.ascending_commit_files.len(), 1); + assert_eq!(log_segment.ascending_commit_files[0].version, 4); } #[test] fn build_table_changes_with_commit_versions() { @@ -437,7 +437,7 @@ fn build_table_changes_with_commit_versions() { let log_segment = LogSegment::for_table_changes(client.as_ref(), log_root.clone(), 2, 5).unwrap(); - let commit_files = log_segment.sorted_commit_files; + let commit_files = log_segment.ascending_commit_files; let checkpoint_parts = log_segment.checkpoint_parts; // Checkpoints should be omitted @@ -454,7 +454,7 @@ fn build_table_changes_with_commit_versions() { let log_segment = LogSegment::for_table_changes(client.as_ref(), log_root.clone(), 0, Some(0)).unwrap(); - let commit_files = log_segment.sorted_commit_files; + let commit_files = log_segment.ascending_commit_files; let checkpoint_parts = log_segment.checkpoint_parts; // Checkpoints should be omitted assert_eq!(checkpoint_parts.len(), 0); @@ -467,7 +467,7 @@ fn build_table_changes_with_commit_versions() { // | Specify no start or end version | // -------------------------------------------------------------------------------- let log_segment = LogSegment::for_table_changes(client.as_ref(), log_root, 0, None).unwrap(); - let commit_files = log_segment.sorted_commit_files; + let commit_files = log_segment.ascending_commit_files; let checkpoint_parts = log_segment.checkpoint_parts; // Checkpoints should be omitted diff --git a/kernel/src/snapshot.rs b/kernel/src/snapshot.rs index 2a3e61b1f..951a44e12 100644 --- a/kernel/src/snapshot.rs +++ b/kernel/src/snapshot.rs @@ -320,12 +320,16 @@ mod tests { .version, 2, ); - assert_eq!(snapshot.log_segment.sorted_commit_files.len(), 1); + assert_eq!(snapshot.log_segment.ascending_commit_files.len(), 1); assert_eq!( - ParsedLogPath::try_from(snapshot.log_segment.sorted_commit_files[0].location.clone()) - .unwrap() - .unwrap() - .version, + ParsedLogPath::try_from( + snapshot.log_segment.ascending_commit_files[0] + .location + .clone() + ) + .unwrap() + .unwrap() + .version, 3, ); } From 13da7080f6742a0ee3f31772c62962a3e98e5bd7 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 15 Nov 2024 11:42:35 -0800 Subject: [PATCH 82/88] Move back log_segment.rs --- kernel/src/{log_segment/mod.rs => log_segment.rs} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename kernel/src/{log_segment/mod.rs => log_segment.rs} (100%) diff --git a/kernel/src/log_segment/mod.rs b/kernel/src/log_segment.rs similarity index 100% rename from kernel/src/log_segment/mod.rs rename to kernel/src/log_segment.rs From 8d20941b099ab6a6b8089bccda9a7b2bbcb7003a Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 15 Nov 2024 11:44:59 -0800 Subject: [PATCH 83/88] fix typo --- kernel/src/log_segment.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 1576c4fda..07e2fc3be 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -79,9 +79,7 @@ impl LogSegment { let version_eff = ascending_commit_files .last() .or(checkpoint_parts.first()) - .ok_or(Error::generic( - "Failed to build log segment: No commit or checkpoitn files provided.", - ))? + .ok_or(Error::generic("No files in log segment"))? .version; if let Some(end_version) = end_version { require!( From 7c45564ef57d93a55c9cc52cd8680f46cd292098 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 15 Nov 2024 13:09:00 -0800 Subject: [PATCH 84/88] Apply suggestions from code review Co-authored-by: Zach Schuermann --- kernel/src/log_segment.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 07e2fc3be..afd753b8b 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -98,7 +98,7 @@ impl LogSegment { }) } - /// Constructs a [`LogSegment`] to be used for Snapshot. For a Snapshot at version `n`: + /// Constructs a [`LogSegment`] to be used for [`Snapshot`]. For a `Snapshot` at version `n`: /// Its LogSegment is made of zero or one checkpoint, and all commits between the checkpoint up /// to and including the end version `n`. Note that a checkpoint may be made of multiple /// parts. All these parts will have the same checkpoint version. @@ -106,6 +106,8 @@ impl LogSegment { /// The options for constructing a LogSegment for Snapshot are as follows: /// - `checkpoint_hint`: a `CheckpointMetadata` to start the log segment from (e.g. from reading the `last_checkpoint` file). /// - `time_travel_version`: The version of the log that the Snapshot will be at. + /// + /// [`Snapshot`]: crate::snapshot::Snapshot #[cfg_attr(feature = "developer-visibility", visibility::make(pub))] pub(crate) fn for_snapshot( fs_client: &dyn FileSystemClient, From 1718d8538bf0b7d64d16d93e723eca6f50fb2abf Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 15 Nov 2024 13:09:52 -0800 Subject: [PATCH 85/88] Apply suggestions from code review Co-authored-by: Zach Schuermann Co-authored-by: Ryan Johnson --- kernel/src/log_segment/tests.rs | 8 ++------ test-utils/src/lib.rs | 2 +- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/kernel/src/log_segment/tests.rs b/kernel/src/log_segment/tests.rs index e59b5f318..cb49d5739 100644 --- a/kernel/src/log_segment/tests.rs +++ b/kernel/src/log_segment/tests.rs @@ -307,9 +307,7 @@ fn build_snapshot_without_checkpoints() { None, ); - // -------------------------------------------------------------------------------- - // | Specify no checkpoint or end version | - // -------------------------------------------------------------------------------- + ///////// Specify no checkpoint or end version ///////// let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root.clone(), None, None).unwrap(); let commit_files = log_segment.ascending_commit_files; @@ -323,9 +321,7 @@ fn build_snapshot_without_checkpoints() { let expected_versions = vec![6, 7]; assert_eq!(versions, expected_versions); - // -------------------------------------------------------------------------------- - // | Specify only end version | - // -------------------------------------------------------------------------------- + ///////// Specify only end version ///////// let log_segment = LogSegment::for_snapshot(client.as_ref(), log_root, None, Some(2)).unwrap(); let commit_files = log_segment.ascending_commit_files; let checkpoint_parts = log_segment.checkpoint_parts; diff --git a/test-utils/src/lib.rs b/test-utils/src/lib.rs index b7ff42d22..f1e81bb77 100644 --- a/test-utils/src/lib.rs +++ b/test-utils/src/lib.rs @@ -89,7 +89,7 @@ pub fn delta_path_for_version(version: u64, suffix: &str) -> Path { } // get an ObjectStore path for a checkpoint file, based on version, part number, and total number of parts -pub fn delta_path_for_multipart_checkpoint(version: u64, part_num: u64, num_parts: u64) -> Path { +pub fn delta_path_for_multipart_checkpoint(version: u64, part_num: u32, num_parts: u32) -> Path { let path = format!("_delta_log/{version:020}.checkpoint.{part_num:010}.{num_parts:010}.parquet"); Path::from(path.as_str()) From 5819e49e08f7c6b4c4817d8e6e3e03a3d5f4f1a4 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 15 Nov 2024 13:10:58 -0800 Subject: [PATCH 86/88] more comments --- kernel/src/log_segment.rs | 2 +- kernel/src/log_segment/tests.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/kernel/src/log_segment.rs b/kernel/src/log_segment.rs index 07e2fc3be..bf1096b21 100644 --- a/kernel/src/log_segment.rs +++ b/kernel/src/log_segment.rs @@ -70,7 +70,7 @@ impl LogSegment { checkpoint_file.version + 1 == commit_file.version, Error::generic(format!( "Gap between checkpoint version {} and next commit {}", - commit_file.version, checkpoint_file.version + checkpoint_file.version, commit_file.version, )) ) } diff --git a/kernel/src/log_segment/tests.rs b/kernel/src/log_segment/tests.rs index e59b5f318..14658d6ff 100644 --- a/kernel/src/log_segment/tests.rs +++ b/kernel/src/log_segment/tests.rs @@ -253,7 +253,7 @@ fn build_snapshot_with_bad_checkpoint_hint_fails() { #[ignore] #[test] fn build_snapshot_with_missing_checkpoint_part_no_hint() { - // TODO(Oussam): Hande checkpoints correctly so that this test passes + // TODO(Oussama): Handel checkpoints correctly so that this test passes // Part 2 of 3 is missing from checkpoint 5. The Snapshot should be made of checkpoint // number 3 and commit files 4 to 7. let (client, log_root) = build_log_with_paths_and_checkpoint( From e49fdfab37efef67bd53d6366ce5dadb54ec4efa Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 15 Nov 2024 13:24:25 -0800 Subject: [PATCH 87/88] Address nits --- kernel/src/log_segment/tests.rs | 24 +++++++++++++----------- test-utils/src/lib.rs | 6 ------ 2 files changed, 13 insertions(+), 17 deletions(-) diff --git a/kernel/src/log_segment/tests.rs b/kernel/src/log_segment/tests.rs index 6c7be76e8..2d9dc6b49 100644 --- a/kernel/src/log_segment/tests.rs +++ b/kernel/src/log_segment/tests.rs @@ -10,7 +10,7 @@ use crate::engine::sync::SyncEngine; use crate::log_segment::LogSegment; use crate::snapshot::CheckpointMetadata; use crate::{FileSystemClient, Table}; -use test_utils::{delta_path_for_multipart_checkpoint, delta_path_for_version}; +use test_utils::delta_path_for_version; // NOTE: In addition to testing the meta-predicate for metadata replay, this test also verifies // that the parquet reader properly infers nullcount = rowcount for missing columns. The two @@ -55,6 +55,13 @@ fn test_replay_for_metadata() { assert_eq!(data.len(), 4); } +// get an ObjectStore path for a checkpoint file, based on version, part number, and total number of parts +pub fn delta_path_for_multipart_checkpoint(version: u64, part_num: u32, num_parts: u32) -> Path { + let path = + format!("_delta_log/{version:020}.checkpoint.{part_num:010}.{num_parts:010}.parquet"); + Path::from(path.as_str()) +} + // Utility method to build a log using a list of log paths and an optional checkpoint hint. The // CheckpointMetadata is written to `_delta_log/_last_checkpoint`. fn build_log_with_paths_and_checkpoint( @@ -253,7 +260,8 @@ fn build_snapshot_with_bad_checkpoint_hint_fails() { #[ignore] #[test] fn build_snapshot_with_missing_checkpoint_part_no_hint() { - // TODO(Oussama): Handel checkpoints correctly so that this test passes + // TODO: Handle checkpoints correctly so that this test passes: https://github.com/delta-incubator/delta-kernel-rs/issues/497 + // Part 2 of 3 is missing from checkpoint 5. The Snapshot should be made of checkpoint // number 3 and commit files 4 to 7. let (client, log_root) = build_log_with_paths_and_checkpoint( @@ -427,9 +435,7 @@ fn build_table_changes_with_commit_versions() { None, ); - // -------------------------------------------------------------------------------- - // | Specify start version and end version | - // -------------------------------------------------------------------------------- + ///////// Specify start version and end version ///////// let log_segment = LogSegment::for_table_changes(client.as_ref(), log_root.clone(), 2, 5).unwrap(); @@ -444,9 +450,7 @@ fn build_table_changes_with_commit_versions() { let expected_versions = (2..=5).collect_vec(); assert_eq!(versions, expected_versions); - // -------------------------------------------------------------------------------- - // | Start version and end version are the same | - // -------------------------------------------------------------------------------- + ///////// Start version and end version are the same ///////// let log_segment = LogSegment::for_table_changes(client.as_ref(), log_root.clone(), 0, Some(0)).unwrap(); @@ -459,9 +463,7 @@ fn build_table_changes_with_commit_versions() { assert_eq!(commit_files.len(), 1); assert_eq!(commit_files[0].version, 0); - // -------------------------------------------------------------------------------- - // | Specify no start or end version | - // -------------------------------------------------------------------------------- + ///////// Specify no start or end version ///////// let log_segment = LogSegment::for_table_changes(client.as_ref(), log_root, 0, None).unwrap(); let commit_files = log_segment.ascending_commit_files; let checkpoint_parts = log_segment.checkpoint_parts; diff --git a/test-utils/src/lib.rs b/test-utils/src/lib.rs index f1e81bb77..e43e8481c 100644 --- a/test-utils/src/lib.rs +++ b/test-utils/src/lib.rs @@ -88,12 +88,6 @@ pub fn delta_path_for_version(version: u64, suffix: &str) -> Path { Path::from(path.as_str()) } -// get an ObjectStore path for a checkpoint file, based on version, part number, and total number of parts -pub fn delta_path_for_multipart_checkpoint(version: u64, part_num: u32, num_parts: u32) -> Path { - let path = - format!("_delta_log/{version:020}.checkpoint.{part_num:010}.{num_parts:010}.parquet"); - Path::from(path.as_str()) -} /// put a commit file into the specified object store. pub async fn add_commit( store: &dyn ObjectStore, From b83dafef093be01bab317a0b0c4f745512c895a7 Mon Sep 17 00:00:00 2001 From: Oussama Saoudi Date: Fri, 15 Nov 2024 13:36:21 -0800 Subject: [PATCH 88/88] fix clippy --- kernel/src/log_segment/tests.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kernel/src/log_segment/tests.rs b/kernel/src/log_segment/tests.rs index 2d9dc6b49..a47a05f16 100644 --- a/kernel/src/log_segment/tests.rs +++ b/kernel/src/log_segment/tests.rs @@ -56,7 +56,7 @@ fn test_replay_for_metadata() { } // get an ObjectStore path for a checkpoint file, based on version, part number, and total number of parts -pub fn delta_path_for_multipart_checkpoint(version: u64, part_num: u32, num_parts: u32) -> Path { +fn delta_path_for_multipart_checkpoint(version: u64, part_num: u32, num_parts: u32) -> Path { let path = format!("_delta_log/{version:020}.checkpoint.{part_num:010}.{num_parts:010}.parquet"); Path::from(path.as_str())